From af2c50662fff5481a62c29db0dbc790c7aa58ea0 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" Date: Tue, 6 Feb 2024 16:43:23 -0800 Subject: [PATCH] [Multi-stage] Optimize query plan serialization --- pinot-common/src/main/proto/mailbox.proto | 18 -- pinot-common/src/main/proto/plan.proto | 2 +- pinot-common/src/main/proto/server.proto | 18 -- pinot-common/src/main/proto/worker.proto | 33 +-- .../explain/PhysicalExplainPlanVisitor.java | 6 +- .../physical/DispatchablePlanContext.java | 9 +- .../physical/MailboxAssignmentVisitor.java | 47 +++-- .../planner/physical/MailboxIdUtils.java | 26 ++- .../pinot/query/routing/MailboxMetadata.java | 63 ++---- .../pinot/query/routing/WorkerMetadata.java | 97 ++++----- .../pinot/query/mailbox/MailboxIdUtils.java | 50 ----- .../pinot/query/runtime/QueryRunner.java | 58 +++--- .../operator/BaseMailboxReceiveOperator.java | 8 +- .../runtime/operator/MailboxSendOperator.java | 20 +- .../runtime/plan/OpChainExecutionContext.java | 17 +- .../query/runtime/plan/StageMetadata.java | 58 +----- ...stributedStagePlan.java => StagePlan.java} | 36 +--- .../pipeline/PipelineBreakerExecutor.java | 19 +- .../plan/serde/QueryPlanSerDeUtils.java | 154 ++++---------- .../plan/server/ServerPlanRequestContext.java | 10 +- .../plan/server/ServerPlanRequestUtils.java | 65 +++--- .../service/dispatch/QueryDispatcher.java | 76 ++++--- .../query/service/server/QueryServer.java | 42 ++-- .../pinot/query/QueryServerEnclosure.java | 7 +- .../query/mailbox/MailboxServiceTest.java | 13 +- .../executor/OpChainSchedulerServiceTest.java | 12 +- .../operator/MailboxReceiveOperatorTest.java | 70 +++---- .../operator/MailboxSendOperatorTest.java | 17 +- .../query/runtime/operator/OpChainTest.java | 56 +++--- .../runtime/operator/OperatorTestUtil.java | 30 +-- .../SortedMailboxReceiveOperatorTest.java | 71 +++---- .../pipeline/PipelineBreakerExecutorTest.java | 100 ++++----- .../plan/serde/QueryPlanSerDeUtilsTest.java | 58 ------ .../runtime/queries/QueryRunnerTestBase.java | 33 ++- .../query/service/server/QueryServerTest.java | 189 +++++++++--------- 35 files changed, 626 insertions(+), 962 deletions(-) delete mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxIdUtils.java rename pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/{DistributedStagePlan.java => StagePlan.java} (53%) delete mode 100644 pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtilsTest.java diff --git a/pinot-common/src/main/proto/mailbox.proto b/pinot-common/src/main/proto/mailbox.proto index 6e1cca9a9279..2ffe923c4dbc 100644 --- a/pinot-common/src/main/proto/mailbox.proto +++ b/pinot-common/src/main/proto/mailbox.proto @@ -17,24 +17,6 @@ // under the License. // -/** - * 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. - */ syntax = "proto3"; package org.apache.pinot.common.proto; diff --git a/pinot-common/src/main/proto/plan.proto b/pinot-common/src/main/proto/plan.proto index b36fa652b0ef..144a6fd7cc69 100644 --- a/pinot-common/src/main/proto/plan.proto +++ b/pinot-common/src/main/proto/plan.proto @@ -78,4 +78,4 @@ message ListField { // The key of the map is a string and the value of the map is a MemberVariableField. message MapField { map content = 1; -} \ No newline at end of file +} diff --git a/pinot-common/src/main/proto/server.proto b/pinot-common/src/main/proto/server.proto index 0239ae125fb6..7781d6f96e0a 100644 --- a/pinot-common/src/main/proto/server.proto +++ b/pinot-common/src/main/proto/server.proto @@ -17,24 +17,6 @@ // under the License. // -/** - * 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. - */ syntax = "proto3"; package org.apache.pinot.common.proto; diff --git a/pinot-common/src/main/proto/worker.proto b/pinot-common/src/main/proto/worker.proto index dfb1cd53eb0c..b7e492fcc5f2 100644 --- a/pinot-common/src/main/proto/worker.proto +++ b/pinot-common/src/main/proto/worker.proto @@ -17,30 +17,10 @@ // under the License. // -/** - * 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. - */ syntax = "proto3"; package org.apache.pinot.common.proto; -import "plan.proto"; - service PinotQueryWorker { // Dispatch a QueryRequest to a PinotQueryWorker rpc Submit(QueryRequest) returns (QueryResponse); @@ -59,7 +39,7 @@ message CancelResponse { // QueryRequest is the dispatched content for all query stages to a physical worker. message QueryRequest { repeated StagePlan stagePlan = 1; - map metadata = 2; + bytes metadata = 2; // Serialized Properties } // QueryResponse is the dispatched response from worker, it doesn't contain actual data, only dispatch status. @@ -70,15 +50,13 @@ message QueryResponse { message StagePlan { int32 stageId = 1; - StageNode stageRoot = 2; + bytes rootNode = 2; // Serialized StageNode StageMetadata stageMetadata = 3; } message StageMetadata { repeated WorkerMetadata workerMetadata = 1; - map customProperty = 2; - string serverAddress = 3; - repeated int32 workerIds = 4; + bytes customProperty = 2; // Serialized Properties } message WorkerMetadata { @@ -90,5 +68,8 @@ message WorkerMetadata { message MailboxMetadata { repeated string mailboxId = 1; repeated string virtualAddress = 2; - map customProperty = 3; +} + +message Properties { + map property = 1; } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PhysicalExplainPlanVisitor.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PhysicalExplainPlanVisitor.java index e1a6ac117637..ea9bef113968 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PhysicalExplainPlanVisitor.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PhysicalExplainPlanVisitor.java @@ -61,7 +61,6 @@ public PhysicalExplainPlanVisitor(DispatchableSubPlan dispatchableSubPlan) { /** * Explains the query plan. * - * @see DispatchableSubPlan#explain() * @param dispatchableSubPlan the queryPlan to explain * @return a String representation of the query plan tree */ @@ -216,9 +215,8 @@ private StringBuilder appendMailboxSend(MailboxSendNode node, Context context) { int receiverStageId = node.getReceiverStageId(); List serverAddressList = - _dispatchableSubPlan.getQueryStageList().get(node.getPlanFragmentId()) - .getWorkerMetadataList().get(context._workerId) - .getMailBoxInfosMap().get(receiverStageId).getVirtualAddressList(); + _dispatchableSubPlan.getQueryStageList().get(node.getPlanFragmentId()).getWorkerMetadataList() + .get(context._workerId).getMailboxMetadataMap().get(receiverStageId).getVirtualAddresses(); List serverInstanceToWorkerIdList = stringifyVirtualServerAddresses(serverAddressList); context._builder.append("->"); String receivers = serverInstanceToWorkerIdList.stream() diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/DispatchablePlanContext.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/DispatchablePlanContext.java index f17f48fd2f86..22744dda0e0d 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/DispatchablePlanContext.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/DispatchablePlanContext.java @@ -108,13 +108,12 @@ public List constructDispatchablePlanFragmentList(Plan int workerId = serverEntry.getKey(); QueryServerInstance queryServerInstance = serverEntry.getValue(); serverInstanceToWorkerIdsMap.computeIfAbsent(queryServerInstance, k -> new ArrayList<>()).add(workerId); - WorkerMetadata.Builder workerMetadataBuilder = new WorkerMetadata.Builder().setVirtualServerAddress( - new VirtualServerAddress(queryServerInstance, workerId)); + WorkerMetadata workerMetadata = new WorkerMetadata(new VirtualServerAddress(queryServerInstance, workerId), + workerIdToMailboxesMap.get(workerId)); if (workerIdToSegmentsMap != null) { - workerMetadataBuilder.addTableSegmentsMap(workerIdToSegmentsMap.get(workerId)); + workerMetadata.setTableSegmentsMap(workerIdToSegmentsMap.get(workerId)); } - workerMetadataBuilder.putAllMailBoxInfosMap(workerIdToMailboxesMap.get(workerId)); - workerMetadataArray[workerId] = workerMetadataBuilder.build(); + workerMetadataArray[workerId] = workerMetadata; } // set the stageMetadata diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/MailboxAssignmentVisitor.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/MailboxAssignmentVisitor.java index 736b443c5374..421e7bbc9c83 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/MailboxAssignmentVisitor.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/MailboxAssignmentVisitor.java @@ -19,8 +19,10 @@ package org.apache.pinot.query.planner.physical; import com.google.common.base.Preconditions; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import org.apache.calcite.rel.RelDistribution; import org.apache.pinot.query.planner.plannode.DefaultPostOrderTraversalVisitor; @@ -63,7 +65,7 @@ public Void process(PlanNode node, DispatchablePlanContext context) { workerId, senderServer, receiverServer); MailboxMetadata mailboxMetadata = new MailboxMetadata(Collections.singletonList( MailboxIdUtils.toPlanMailboxId(senderFragmentId, workerId, receiverFragmentId, workerId)), - Collections.singletonList(new VirtualServerAddress(senderServer, workerId)), Collections.emptyMap()); + Collections.singletonList(new VirtualServerAddress(senderServer, workerId))); senderMailboxesMap.computeIfAbsent(workerId, k -> new HashMap<>()).put(receiverFragmentId, mailboxMetadata); receiverMailboxesMap.computeIfAbsent(workerId, k -> new HashMap<>()).put(senderFragmentId, mailboxMetadata); } @@ -78,11 +80,9 @@ public Void process(PlanNode node, DispatchablePlanContext context) { for (int workerId = 0; workerId < numSenders; workerId++) { String mailboxId = MailboxIdUtils.toPlanMailboxId(senderFragmentId, workerId, receiverFragmentId, workerId); MailboxMetadata serderMailboxMetadata = new MailboxMetadata(Collections.singletonList(mailboxId), - Collections.singletonList(new VirtualServerAddress(receiverServerMap.get(workerId), workerId)), - Collections.emptyMap()); + Collections.singletonList(new VirtualServerAddress(receiverServerMap.get(workerId), workerId))); MailboxMetadata receiverMailboxMetadata = new MailboxMetadata(Collections.singletonList(mailboxId), - Collections.singletonList(new VirtualServerAddress(senderServerMap.get(workerId), workerId)), - Collections.emptyMap()); + Collections.singletonList(new VirtualServerAddress(senderServerMap.get(workerId), workerId))); senderMailboxesMap.computeIfAbsent(workerId, k -> new HashMap<>()) .put(receiverFragmentId, serderMailboxMetadata); receiverMailboxesMap.computeIfAbsent(workerId, k -> new HashMap<>()) @@ -94,22 +94,23 @@ public Void process(PlanNode node, DispatchablePlanContext context) { for (int senderWorkerId = 0; senderWorkerId < numSenders; senderWorkerId++) { VirtualServerAddress senderAddress = new VirtualServerAddress(senderServerMap.get(senderWorkerId), senderWorkerId); - MailboxMetadata senderMailboxMetadata = new MailboxMetadata(); + List receivingMailboxIds = new ArrayList<>(partitionParallelism); + List receivingAddresses = new ArrayList<>(partitionParallelism); + MailboxMetadata senderMailboxMetadata = new MailboxMetadata(receivingMailboxIds, receivingAddresses); senderMailboxesMap.computeIfAbsent(senderWorkerId, k -> new HashMap<>()) .put(receiverFragmentId, senderMailboxMetadata); for (int i = 0; i < partitionParallelism; i++) { - VirtualServerAddress receiverAddress = - new VirtualServerAddress(receiverServerMap.get(receiverWorkerId), receiverWorkerId); String mailboxId = MailboxIdUtils.toPlanMailboxId(senderFragmentId, senderWorkerId, receiverFragmentId, receiverWorkerId); - senderMailboxMetadata.getMailBoxIdList().add(mailboxId); - senderMailboxMetadata.getVirtualAddressList().add(receiverAddress); + receivingMailboxIds.add(mailboxId); + receivingAddresses.add( + new VirtualServerAddress(receiverServerMap.get(receiverWorkerId), receiverWorkerId)); MailboxMetadata receiverMailboxMetadata = receiverMailboxesMap.computeIfAbsent(receiverWorkerId, k -> new HashMap<>()) .computeIfAbsent(senderFragmentId, k -> new MailboxMetadata()); - receiverMailboxMetadata.getMailBoxIdList().add(mailboxId); - receiverMailboxMetadata.getVirtualAddressList().add(senderAddress); + receiverMailboxMetadata.getMailboxIds().add(mailboxId); + receiverMailboxMetadata.getVirtualAddresses().add(senderAddress); receiverWorkerId++; } @@ -123,22 +124,22 @@ public Void process(PlanNode node, DispatchablePlanContext context) { for (int senderWorkerId = 0; senderWorkerId < numSenders; senderWorkerId++) { VirtualServerAddress senderAddress = new VirtualServerAddress(senderServerMap.get(senderWorkerId), senderWorkerId); - MailboxMetadata senderMailboxMetadata = new MailboxMetadata(); + List receivingMailboxIds = new ArrayList<>(numReceivers); + List receivingAddresses = new ArrayList<>(numReceivers); + MailboxMetadata senderMailboxMetadata = new MailboxMetadata(receivingMailboxIds, receivingAddresses); senderMailboxesMap.computeIfAbsent(senderWorkerId, k -> new HashMap<>()) .put(receiverFragmentId, senderMailboxMetadata); for (int receiverWorkerId = 0; receiverWorkerId < numReceivers; receiverWorkerId++) { - VirtualServerAddress receiverAddress = - new VirtualServerAddress(receiverServerMap.get(receiverWorkerId), receiverWorkerId); String mailboxId = MailboxIdUtils.toPlanMailboxId(senderFragmentId, senderWorkerId, receiverFragmentId, receiverWorkerId); - senderMailboxMetadata.getMailBoxIdList().add(mailboxId); - senderMailboxMetadata.getVirtualAddressList().add(receiverAddress); + receivingMailboxIds.add(mailboxId); + receivingAddresses.add(new VirtualServerAddress(receiverServerMap.get(receiverWorkerId), receiverWorkerId)); MailboxMetadata receiverMailboxMetadata = receiverMailboxesMap.computeIfAbsent(receiverWorkerId, k -> new HashMap<>()) .computeIfAbsent(senderFragmentId, k -> new MailboxMetadata()); - receiverMailboxMetadata.getMailBoxIdList().add(mailboxId); - receiverMailboxMetadata.getVirtualAddressList().add(senderAddress); + receiverMailboxMetadata.getMailboxIds().add(mailboxId); + receiverMailboxMetadata.getVirtualAddresses().add(senderAddress); } } } @@ -154,14 +155,12 @@ private boolean isDirectExchangeCompatible(DispatchablePlanMetadata sender, Disp int numReceivers = receiverServerMap.size(); if (sender.getScannedTables().size() > 0 && receiver.getScannedTables().size() == 0) { // leaf-to-intermediate condition - return numSenders * sender.getPartitionParallelism() == numReceivers - && sender.getPartitionFunction() != null + return numSenders * sender.getPartitionParallelism() == numReceivers && sender.getPartitionFunction() != null && sender.getPartitionFunction().equalsIgnoreCase(receiver.getPartitionFunction()); } else { // dynamic-broadcast condition || intermediate-to-intermediate - return numSenders == numReceivers - && sender.getPartitionFunction() != null - && sender.getPartitionFunction().equalsIgnoreCase(receiver.getPartitionFunction()); + return numSenders == numReceivers && sender.getPartitionFunction() != null && sender.getPartitionFunction() + .equalsIgnoreCase(receiver.getPartitionFunction()); } } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/MailboxIdUtils.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/MailboxIdUtils.java index c949c6598fa8..32c7d3197a64 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/MailboxIdUtils.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/physical/MailboxIdUtils.java @@ -18,15 +18,35 @@ */ package org.apache.pinot.query.planner.physical; +import com.google.common.annotations.VisibleForTesting; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.pinot.query.routing.MailboxMetadata; + + public class MailboxIdUtils { private MailboxIdUtils() { } - private static final char SEPARATOR = '|'; + public static final char SEPARATOR = '|'; public static String toPlanMailboxId(int senderStageId, int senderWorkerId, int receiverStageId, int receiverWorkerId) { - return Integer.toString(senderStageId) + SEPARATOR + senderWorkerId + SEPARATOR - + receiverStageId + SEPARATOR + receiverWorkerId; + return Integer.toString(senderStageId) + SEPARATOR + senderWorkerId + SEPARATOR + receiverStageId + SEPARATOR + + receiverWorkerId; + } + + public static String toMailboxId(long requestId, String planMailboxId) { + return Long.toString(requestId) + SEPARATOR + planMailboxId; + } + + public static List toMailboxIds(long requestId, MailboxMetadata mailboxMetadata) { + return mailboxMetadata.getMailboxIds().stream().map(v -> toMailboxId(requestId, v)).collect(Collectors.toList()); + } + + @VisibleForTesting + public static String toMailboxId(long requestId, int senderStageId, int senderWorkerId, int receiverStageId, + int receiverWorkerId) { + return toMailboxId(requestId, toPlanMailboxId(senderStageId, senderWorkerId, receiverStageId, receiverWorkerId)); } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/MailboxMetadata.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/MailboxMetadata.java index dcc46c8271d1..b3484d1a7bc1 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/MailboxMetadata.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/MailboxMetadata.java @@ -19,10 +19,7 @@ package org.apache.pinot.query.routing; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import java.util.Objects; /** @@ -31,68 +28,32 @@ *
    *
  • MailboxId: the unique id of the mailbox
  • *
  • VirtualAddress: the virtual address of the mailbox
  • - *
  • CustomProperties: the custom properties of the mailbox
  • *
*/ public class MailboxMetadata { - private final List _mailBoxIdList; - private final List _virtualAddressList; - private final Map _customProperties; + private final List _mailboxIds; + private final List _virtualAddresses; public MailboxMetadata() { - _mailBoxIdList = new ArrayList<>(); - _virtualAddressList = new ArrayList<>(); - _customProperties = new HashMap<>(); + _mailboxIds = new ArrayList<>(); + _virtualAddresses = new ArrayList<>(); } - public MailboxMetadata(List mailBoxIdList, List virtualAddressList, - Map customProperties) { - _mailBoxIdList = mailBoxIdList; - _virtualAddressList = virtualAddressList; - _customProperties = customProperties; + public MailboxMetadata(List mailboxIds, List virtualAddresses) { + _mailboxIds = mailboxIds; + _virtualAddresses = virtualAddresses; } - public List getMailBoxIdList() { - return _mailBoxIdList; + public List getMailboxIds() { + return _mailboxIds; } - public String getMailBoxId(int index) { - return _mailBoxIdList.get(index); - } - - public List getVirtualAddressList() { - return _virtualAddressList; - } - - public VirtualServerAddress getVirtualAddress(int index) { - return _virtualAddressList.get(index); - } - - public Map getCustomProperties() { - return _customProperties; + public List getVirtualAddresses() { + return _virtualAddresses; } @Override public String toString() { - return _mailBoxIdList + "@" + _virtualAddressList.toString() + "#" + _customProperties.toString(); - } - - @Override - public int hashCode() { - return Objects.hash(_mailBoxIdList, _virtualAddressList, _customProperties); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - MailboxMetadata that = (MailboxMetadata) o; - return Objects.equals(_mailBoxIdList, that._mailBoxIdList) - && Objects.equals(_virtualAddressList, that._virtualAddressList) - && _customProperties.equals(that._customProperties); + return _mailboxIds + "@" + _virtualAddresses; } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerMetadata.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerMetadata.java index 9d92bfb69795..3392261980c8 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerMetadata.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerMetadata.java @@ -24,6 +24,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import javax.annotation.Nullable; import org.apache.pinot.spi.utils.JsonUtils; @@ -41,85 +42,63 @@ * MailboxSendNode and MailboxReceiveNode to derive the info during runtime. this should changed to plan time soon. */ public class WorkerMetadata { - private final VirtualServerAddress _virtualServerAddress; - private final Map _mailBoxInfosMap; + public static final String TABLE_SEGMENTS_MAP_KEY = "tableSegmentsMap"; + + private final VirtualServerAddress _virtualAddress; + private final Map _mailboxMetadataMap; private final Map _customProperties; - private WorkerMetadata(VirtualServerAddress virtualServerAddress, Map mailBoxInfosMap, + public WorkerMetadata(VirtualServerAddress virtualAddress, Map mailboxMetadataMap) { + _virtualAddress = virtualAddress; + _mailboxMetadataMap = mailboxMetadataMap; + _customProperties = new HashMap<>(); + } + + public WorkerMetadata(VirtualServerAddress virtualAddress, Map mailboxMetadataMap, Map customProperties) { - _virtualServerAddress = virtualServerAddress; - _mailBoxInfosMap = mailBoxInfosMap; + _virtualAddress = virtualAddress; + _mailboxMetadataMap = mailboxMetadataMap; _customProperties = customProperties; } - public VirtualServerAddress getVirtualServerAddress() { - return _virtualServerAddress; + public VirtualServerAddress getVirtualAddress() { + return _virtualAddress; } - public Map getMailBoxInfosMap() { - return _mailBoxInfosMap; + public Map getMailboxMetadataMap() { + return _mailboxMetadataMap; } public Map getCustomProperties() { return _customProperties; } - public static class Builder { - public static final String TABLE_SEGMENTS_MAP_KEY = "tableSegmentsMap"; - private VirtualServerAddress _virtualServerAddress; - private Map _mailBoxInfosMap; - private Map _customProperties; - - public Builder() { - _mailBoxInfosMap = new HashMap<>(); - _customProperties = new HashMap<>(); - } - - public Builder setVirtualServerAddress(VirtualServerAddress virtualServerAddress) { - _virtualServerAddress = virtualServerAddress; - return this; - } - - public Builder putAllMailBoxInfosMap(Map mailBoxInfosMap) { - _mailBoxInfosMap.putAll(mailBoxInfosMap); - return this; - } - - public Builder addMailBoxInfoMap(Integer planFragmentId, MailboxMetadata mailBoxMetadata) { - _mailBoxInfosMap.put(planFragmentId, mailBoxMetadata); - return this; - } - - public Builder addTableSegmentsMap(Map> tableSegmentsMap) { - try { - String tableSegmentsMapStr = JsonUtils.objectToString(tableSegmentsMap); - _customProperties.put(TABLE_SEGMENTS_MAP_KEY, tableSegmentsMapStr); - } catch (JsonProcessingException e) { - throw new RuntimeException("Unable to serialize table segments map", e); - } - return this; - } - - public WorkerMetadata build() { - return new WorkerMetadata(_virtualServerAddress, _mailBoxInfosMap, _customProperties); - } - - public void putAllCustomProperties(Map customPropertyMap) { - _customProperties.putAll(customPropertyMap); - } - } - - public static Map> getTableSegmentsMap(WorkerMetadata workerMetadata) { - String tableSegmentKeyStr = workerMetadata.getCustomProperties().get(Builder.TABLE_SEGMENTS_MAP_KEY); - if (tableSegmentKeyStr != null) { + @Nullable + public Map> getTableSegmentsMap() { + String tableSegmentsMapStr = _customProperties.get(TABLE_SEGMENTS_MAP_KEY); + if (tableSegmentsMapStr != null) { try { - return JsonUtils.stringToObject(tableSegmentKeyStr, new TypeReference>>() { + return JsonUtils.stringToObject(tableSegmentsMapStr, new TypeReference>>() { }); } catch (IOException e) { - throw new RuntimeException("Unable to deserialize table segments map", e); + throw new RuntimeException("Unable to deserialize table segments map: " + tableSegmentsMapStr, e); } } else { return null; } } + + public boolean isLeafStageWorker() { + return _customProperties.containsKey(TABLE_SEGMENTS_MAP_KEY); + } + + public void setTableSegmentsMap(Map> tableSegmentsMap) { + String tableSegmentsMapStr; + try { + tableSegmentsMapStr = JsonUtils.objectToString(tableSegmentsMap); + } catch (JsonProcessingException e) { + throw new RuntimeException("Unable to serialize table segments map: " + tableSegmentsMap, e); + } + _customProperties.put(TABLE_SEGMENTS_MAP_KEY, tableSegmentsMapStr); + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxIdUtils.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxIdUtils.java deleted file mode 100644 index 7168afc486b2..000000000000 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxIdUtils.java +++ /dev/null @@ -1,50 +0,0 @@ -/** - * 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.mailbox; - -import com.google.common.annotations.VisibleForTesting; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.pinot.query.routing.MailboxMetadata; - - -// TODO: De-couple mailbox id from query information -public class MailboxIdUtils { - private MailboxIdUtils() { - } - - private static final char SEPARATOR = '|'; - - @VisibleForTesting - public static String toMailboxId(long requestId, int senderStageId, int senderWorkerId, int receiverStageId, - int receiverWorkerId) { - return Long.toString(requestId) + SEPARATOR + senderStageId + SEPARATOR + senderWorkerId + SEPARATOR - + receiverStageId + SEPARATOR + receiverWorkerId; - } - - public static List toMailboxIds(long requestId, MailboxMetadata mailBoxMetadata) { - return toMailboxIds(requestId, mailBoxMetadata.getMailBoxIdList()); - } - - public static List toMailboxIds(long requestId, List mailboxMetadataIdList) { - return mailboxMetadataIdList.stream() - .map(mailboxIdFromBroker -> Long.toString(requestId) + SEPARATOR + mailboxIdFromBroker) - .collect(Collectors.toList()); - } -} 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 526a489fb638..4796383a0bdf 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 @@ -31,17 +31,20 @@ import org.apache.pinot.core.data.manager.InstanceDataManager; import org.apache.pinot.core.query.executor.QueryExecutor; import org.apache.pinot.core.query.executor.ServerQueryExecutorV1Impl; -import org.apache.pinot.query.mailbox.MailboxIdUtils; import org.apache.pinot.query.mailbox.MailboxService; +import org.apache.pinot.query.planner.physical.MailboxIdUtils; import org.apache.pinot.query.planner.plannode.MailboxSendNode; import org.apache.pinot.query.routing.MailboxMetadata; +import org.apache.pinot.query.routing.VirtualServerAddress; +import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.executor.ExecutorServiceUtils; import org.apache.pinot.query.runtime.executor.OpChainSchedulerService; import org.apache.pinot.query.runtime.operator.OpChain; -import org.apache.pinot.query.runtime.plan.DistributedStagePlan; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.query.runtime.plan.PhysicalPlanVisitor; +import org.apache.pinot.query.runtime.plan.StageMetadata; +import org.apache.pinot.query.runtime.plan.StagePlan; import org.apache.pinot.query.runtime.plan.pipeline.PipelineBreakerExecutor; import org.apache.pinot.query.runtime.plan.pipeline.PipelineBreakerResult; import org.apache.pinot.query.runtime.plan.server.ServerPlanRequestUtils; @@ -54,7 +57,7 @@ /** - * {@link QueryRunner} accepts a {@link DistributedStagePlan} and runs it. + * {@link QueryRunner} accepts a {@link StagePlan} and runs it. */ public class QueryRunner { private static final Logger LOGGER = LoggerFactory.getLogger(QueryRunner.class); @@ -139,42 +142,46 @@ public void shutDown() { } /** - * Execute a {@link DistributedStagePlan}. + * Execute a {@link StagePlan}. * *

This execution entry point should be asynchronously called by the request handler and caller should not wait * for results/exceptions.

*/ - public void processQuery(DistributedStagePlan distributedStagePlan, Map requestMetadata) { + public void processQuery(WorkerMetadata workerMetadata, StagePlan stagePlan, Map requestMetadata) { long requestId = Long.parseLong(requestMetadata.get(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID)); long timeoutMs = Long.parseLong(requestMetadata.get(CommonConstants.Broker.Request.QueryOptionKey.TIMEOUT_MS)); - Map opChainMetadata = consolidateMetadata( - distributedStagePlan.getStageMetadata().getCustomProperties(), requestMetadata); long deadlineMs = System.currentTimeMillis() + timeoutMs; + StageMetadata stageMetadata = stagePlan.getStageMetadata(); + Map opChainMetadata = consolidateMetadata(stageMetadata.getCustomProperties(), requestMetadata); + // run pre-stage execution for all pipeline breakers PipelineBreakerResult pipelineBreakerResult = - PipelineBreakerExecutor.executePipelineBreakers(_opChainScheduler, _mailboxService, distributedStagePlan, + PipelineBreakerExecutor.executePipelineBreakers(_opChainScheduler, _mailboxService, workerMetadata, stagePlan, opChainMetadata, requestId, deadlineMs); // Send error block to all the receivers if pipeline breaker fails if (pipelineBreakerResult != null && pipelineBreakerResult.getErrorBlock() != null) { TransferableBlock errorBlock = pipelineBreakerResult.getErrorBlock(); LOGGER.error("Error executing pipeline breaker for request: {}, stage: {}, sending error block: {}", requestId, - distributedStagePlan.getStageId(), errorBlock.getExceptions()); - int receiverStageId = ((MailboxSendNode) distributedStagePlan.getStageRoot()).getReceiverStageId(); - MailboxMetadata mailboxMetadata = distributedStagePlan.getStageMetadata().getWorkerMetadataList() - .get(distributedStagePlan.getServer().workerId()).getMailBoxInfosMap().get(receiverStageId); + stagePlan.getStageId(), errorBlock.getExceptions()); + int receiverStageId = ((MailboxSendNode) stagePlan.getRootNode()).getReceiverStageId(); + MailboxMetadata mailboxMetadata = workerMetadata.getMailboxMetadataMap().get(receiverStageId); List mailboxIds = MailboxIdUtils.toMailboxIds(requestId, mailboxMetadata); - for (int i = 0; i < mailboxIds.size(); i++) { + List virtualAddresses = mailboxMetadata.getVirtualAddresses(); + int numMailboxes = mailboxIds.size(); + for (int i = 0; i < numMailboxes; i++) { + String mailboxId = mailboxIds.get(i); + VirtualServerAddress virtualAddress = virtualAddresses.get(i); try { - _mailboxService.getSendingMailbox(mailboxMetadata.getVirtualAddress(i).hostname(), - mailboxMetadata.getVirtualAddress(i).port(), mailboxIds.get(i), deadlineMs).send(errorBlock); + _mailboxService.getSendingMailbox(virtualAddress.hostname(), virtualAddress.port(), mailboxId, deadlineMs) + .send(errorBlock); } catch (TimeoutException e) { - LOGGER.warn("Timed out sending error block to mailbox: {} for request: {}, stage: {}", mailboxIds.get(i), - requestId, distributedStagePlan.getStageId(), e); + LOGGER.warn("Timed out sending error block to mailbox: {} for request: {}, stage: {}", mailboxId, requestId, + stagePlan.getStageId(), e); } catch (Exception e) { - LOGGER.error("Caught exception sending error block to mailbox: {} for request: {}, stage: {}", - mailboxIds.get(i), requestId, distributedStagePlan.getStageId(), e); + LOGGER.error("Caught exception sending error block to mailbox: {} for request: {}, stage: {}", mailboxId, + requestId, stagePlan.getStageId(), e); } } return; @@ -182,15 +189,14 @@ public void processQuery(DistributedStagePlan distributedStagePlan, Map sendingMailboxIds = MailboxIdUtils.toMailboxIds(requestId, mailboxMetadata); - List sendingMailboxes = new ArrayList<>(sendingMailboxIds.size()); - for (int i = 0; i < sendingMailboxIds.size(); i++) { - sendingMailboxes.add(mailboxService.getSendingMailbox(mailboxMetadata.getVirtualAddress(i).hostname(), - mailboxMetadata.getVirtualAddress(i).port(), sendingMailboxIds.get(i), deadlineMs)); + List sendingAddresses = mailboxMetadata.getVirtualAddresses(); + int numMailboxes = sendingMailboxIds.size(); + List sendingMailboxes = new ArrayList<>(numMailboxes); + for (int i = 0; i < numMailboxes; i++) { + String sendingMailboxId = sendingMailboxIds.get(i); + VirtualServerAddress sendingAddress = sendingAddresses.get(i); + sendingMailboxes.add( + mailboxService.getSendingMailbox(sendingAddress.hostname(), sendingAddress.port(), sendingMailboxId, + deadlineMs)); } return BlockExchange.getExchange(sendingMailboxes, distributionType, distributionKeys, TransferableBlockUtils::splitBlock); 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 10069167d6bf..5059b2f8ec0e 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 @@ -22,6 +22,7 @@ import java.util.Map; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.routing.VirtualServerAddress; +import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.operator.OpChainId; import org.apache.pinot.query.runtime.operator.OpChainStats; import org.apache.pinot.query.runtime.plan.pipeline.PipelineBreakerResult; @@ -38,10 +39,10 @@ public class OpChainExecutionContext { private final MailboxService _mailboxService; private final long _requestId; private final int _stageId; - private final VirtualServerAddress _server; private final long _deadlineMs; private final Map _opChainMetadata; private final StageMetadata _stageMetadata; + private final WorkerMetadata _workerMetadata; private final OpChainId _id; private final OpChainStats _stats; private final PipelineBreakerResult _pipelineBreakerResult; @@ -49,17 +50,17 @@ public class OpChainExecutionContext { private ServerPlanRequestContext _leafStageContext; - public OpChainExecutionContext(MailboxService mailboxService, long requestId, int stageId, - VirtualServerAddress server, long deadlineMs, Map opChainMetadata, StageMetadata stageMetadata, + public OpChainExecutionContext(MailboxService mailboxService, long requestId, int stageId, long deadlineMs, + Map opChainMetadata, StageMetadata stageMetadata, WorkerMetadata workerMetadata, PipelineBreakerResult pipelineBreakerResult) { _mailboxService = mailboxService; _requestId = requestId; _stageId = stageId; - _server = server; _deadlineMs = deadlineMs; _opChainMetadata = Collections.unmodifiableMap(opChainMetadata); _stageMetadata = stageMetadata; - _id = new OpChainId(requestId, server.workerId(), stageId); + _workerMetadata = workerMetadata; + _id = new OpChainId(requestId, workerMetadata.getVirtualAddress().workerId(), stageId); _stats = new OpChainStats(_id.toString()); _pipelineBreakerResult = pipelineBreakerResult; if (pipelineBreakerResult != null && pipelineBreakerResult.getOpChainStats() != null) { @@ -81,7 +82,7 @@ public int getStageId() { } public VirtualServerAddress getServer() { - return _server; + return _workerMetadata.getVirtualAddress(); } public long getDeadlineMs() { @@ -96,6 +97,10 @@ public StageMetadata getStageMetadata() { return _stageMetadata; } + public WorkerMetadata getWorkerMetadata() { + return _workerMetadata; + } + public OpChainId getId() { return _id; } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageMetadata.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageMetadata.java index f2543a3363ea..a07a04a0b7ec 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageMetadata.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageMetadata.java @@ -18,11 +18,10 @@ */ package org.apache.pinot.query.runtime.plan; -import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.pinot.core.routing.TimeBoundaryInfo; +import org.apache.pinot.query.planner.physical.DispatchablePlanFragment; import org.apache.pinot.query.routing.WorkerMetadata; @@ -33,9 +32,9 @@ public class StageMetadata { private final List _workerMetadataList; private final Map _customProperties; - StageMetadata(List workerMetadataList, Map customProperties) { + public StageMetadata(List workerMetadataList, Map customProperties) { _workerMetadataList = workerMetadataList; - _customProperties = Collections.unmodifiableMap(customProperties); + _customProperties = customProperties; } public List getWorkerMetadataList() { @@ -46,54 +45,13 @@ public Map getCustomProperties() { return _customProperties; } - public static class Builder { - public static final String TABLE_NAME_KEY = "tableName"; - public static final String TIME_BOUNDARY_COLUMN_KEY = "timeBoundaryInfo.timeColumn"; - public static final String TIME_BOUNDARY_VALUE_KEY = "timeBoundaryInfo.timeValue"; - private List _workerMetadataList; - private Map _customProperties; - - public Builder() { - _customProperties = new HashMap<>(); - } - - public Builder setWorkerMetadataList(List workerMetadataList) { - _workerMetadataList = workerMetadataList; - return this; - } - - public Builder addTableName(String tableName) { - _customProperties.put(TABLE_NAME_KEY, tableName); - return this; - } - - public Builder addTimeBoundaryInfo(TimeBoundaryInfo timeBoundaryInfo) { - _customProperties.put(TIME_BOUNDARY_COLUMN_KEY, timeBoundaryInfo.getTimeColumn()); - _customProperties.put(TIME_BOUNDARY_VALUE_KEY, timeBoundaryInfo.getTimeValue()); - return this; - } - - public Builder addCustomProperties(Map customPropertyMap) { - _customProperties.putAll(customPropertyMap); - return this; - } - - public StageMetadata build() { - return new StageMetadata(_workerMetadataList, _customProperties); - } - - public void putAllCustomProperties(Map customPropertyMap) { - _customProperties.putAll(customPropertyMap); - } - } - - public static String getTableName(StageMetadata metadata) { - return metadata.getCustomProperties().get(Builder.TABLE_NAME_KEY); + public String getTableName() { + return _customProperties.get(DispatchablePlanFragment.TABLE_NAME_KEY); } - public static TimeBoundaryInfo getTimeBoundary(StageMetadata metadata) { - String timeColumn = metadata.getCustomProperties().get(Builder.TIME_BOUNDARY_COLUMN_KEY); - String timeValue = metadata.getCustomProperties().get(Builder.TIME_BOUNDARY_VALUE_KEY); + public TimeBoundaryInfo getTimeBoundary() { + String timeColumn = _customProperties.get(DispatchablePlanFragment.TIME_BOUNDARY_COLUMN_KEY); + String timeValue = _customProperties.get(DispatchablePlanFragment.TIME_BOUNDARY_VALUE_KEY); return timeColumn != null && timeValue != null ? new TimeBoundaryInfo(timeColumn, timeValue) : null; } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/DistributedStagePlan.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StagePlan.java similarity index 53% rename from pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/DistributedStagePlan.java rename to pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StagePlan.java index 62e8d1925475..a45b48c5de30 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/DistributedStagePlan.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StagePlan.java @@ -18,30 +18,22 @@ */ package org.apache.pinot.query.runtime.plan; -import java.util.List; -import java.util.Map; import org.apache.pinot.query.planner.plannode.PlanNode; -import org.apache.pinot.query.routing.VirtualServerAddress; -import org.apache.pinot.query.routing.WorkerMetadata; /** - * {@code DistributedStagePlan} is the deserialized version of the - * {@link org.apache.pinot.common.proto.Worker.StagePlan}. + * {@code StagePlan} is the deserialized version of the {@link org.apache.pinot.common.proto.Worker.StagePlan}. * *

It is also the extended version of the {@link org.apache.pinot.core.query.request.ServerQueryRequest}. */ -public class DistributedStagePlan { +public class StagePlan { private final int _stageId; - private final VirtualServerAddress _server; - private final PlanNode _stageRoot; + private final PlanNode _rootNode; private final StageMetadata _stageMetadata; - public DistributedStagePlan(int stageId, VirtualServerAddress server, PlanNode stageRoot, - StageMetadata stageMetadata) { + public StagePlan(int stageId, PlanNode rootNode, StageMetadata stageMetadata) { _stageId = stageId; - _server = server; - _stageRoot = stageRoot; + _rootNode = rootNode; _stageMetadata = stageMetadata; } @@ -49,25 +41,11 @@ public int getStageId() { return _stageId; } - public VirtualServerAddress getServer() { - return _server; - } - - public PlanNode getStageRoot() { - return _stageRoot; + public PlanNode getRootNode() { + return _rootNode; } public StageMetadata getStageMetadata() { return _stageMetadata; } - - public WorkerMetadata getCurrentWorkerMetadata() { - return _stageMetadata.getWorkerMetadataList().get(_server.workerId()); - } - - public static boolean isLeafStage(DistributedStagePlan distributedStagePlan) { - WorkerMetadata workerMetadata = distributedStagePlan.getCurrentWorkerMetadata(); - Map> segments = WorkerMetadata.getTableSegmentsMap(workerMetadata); - return segments != null && segments.size() > 0; - } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutor.java index 3db86807d7c2..aec7998e169e 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutor.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutor.java @@ -29,13 +29,14 @@ import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.planner.plannode.MailboxReceiveNode; import org.apache.pinot.query.planner.plannode.PlanNode; +import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.executor.OpChainSchedulerService; import org.apache.pinot.query.runtime.operator.OpChain; -import org.apache.pinot.query.runtime.plan.DistributedStagePlan; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.query.runtime.plan.PhysicalPlanVisitor; +import org.apache.pinot.query.runtime.plan.StagePlan; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,7 +56,8 @@ private PipelineBreakerExecutor() { * * @param scheduler scheduler service to run the pipeline breaker main thread. * @param mailboxService mailbox service to attach the {@link MailboxReceiveNode} against. - * @param distributedStagePlan the distributed stage plan to run pipeline breaker on. + * @param workerMetadata worker metadata for the current worker. + * @param stagePlan the distributed stage plan to run pipeline breaker on. * @param opChainMetadata request metadata, including query options * @param requestId request ID * @param deadlineMs execution deadline @@ -65,23 +67,22 @@ private PipelineBreakerExecutor() { */ @Nullable public static PipelineBreakerResult executePipelineBreakers(OpChainSchedulerService scheduler, - MailboxService mailboxService, DistributedStagePlan distributedStagePlan, Map opChainMetadata, - long requestId, long deadlineMs) { + MailboxService mailboxService, WorkerMetadata workerMetadata, StagePlan stagePlan, + Map opChainMetadata, long requestId, long deadlineMs) { PipelineBreakerContext pipelineBreakerContext = new PipelineBreakerContext(); - PipelineBreakerVisitor.visitPlanRoot(distributedStagePlan.getStageRoot(), pipelineBreakerContext); + PipelineBreakerVisitor.visitPlanRoot(stagePlan.getRootNode(), pipelineBreakerContext); if (!pipelineBreakerContext.getPipelineBreakerMap().isEmpty()) { try { // TODO: This PlanRequestContext needs to indicate it is a pre-stage opChain and only listens to pre-stage // OpChain receive-mail callbacks. // see also: MailboxIdUtils TODOs, de-couple mailbox id from query information OpChainExecutionContext opChainExecutionContext = - new OpChainExecutionContext(mailboxService, requestId, distributedStagePlan.getStageId(), - distributedStagePlan.getServer(), deadlineMs, opChainMetadata, distributedStagePlan.getStageMetadata(), - null); + new OpChainExecutionContext(mailboxService, requestId, stagePlan.getStageId(), deadlineMs, opChainMetadata, + stagePlan.getStageMetadata(), workerMetadata, null); return execute(scheduler, pipelineBreakerContext, opChainExecutionContext); } catch (Exception e) { LOGGER.error("Caught exception executing pipeline breaker for request: {}, stage: {}", requestId, - distributedStagePlan.getStageId(), e); + stagePlan.getStageId(), e); return new PipelineBreakerResult(pipelineBreakerContext.getNodeIdMap(), Collections.emptyMap(), TransferableBlockUtils.getErrorTransferableBlock(e), null); } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtils.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtils.java index f4b34a145a18..fbfb9487b4aa 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtils.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtils.java @@ -18,152 +18,86 @@ */ package org.apache.pinot.query.runtime.plan.serde; -import java.util.ArrayList; -import java.util.HashMap; +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; import java.util.List; import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.commons.lang.StringUtils; +import java.util.stream.Collectors; +import org.apache.pinot.common.proto.Plan; import org.apache.pinot.common.proto.Worker; -import org.apache.pinot.query.planner.physical.DispatchablePlanFragment; import org.apache.pinot.query.planner.plannode.AbstractPlanNode; import org.apache.pinot.query.planner.plannode.StageNodeSerDeUtils; import org.apache.pinot.query.routing.MailboxMetadata; -import org.apache.pinot.query.routing.QueryServerInstance; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.routing.WorkerMetadata; -import org.apache.pinot.query.runtime.plan.DistributedStagePlan; import org.apache.pinot.query.runtime.plan.StageMetadata; +import org.apache.pinot.query.runtime.plan.StagePlan; /** * This utility class serialize/deserialize between {@link Worker.StagePlan} elements to Planner elements. */ public class QueryPlanSerDeUtils { - private static final Pattern VIRTUAL_SERVER_PATTERN = - Pattern.compile("(?[0-9]+)@(?[^:]+):(?[0-9]+)"); - private QueryPlanSerDeUtils() { - // do not instantiate. - } - - public static VirtualServerAddress protoToAddress(String virtualAddressStr) { - Matcher matcher = VIRTUAL_SERVER_PATTERN.matcher(virtualAddressStr); - if (!matcher.matches()) { - throw new IllegalArgumentException("Unexpected virtualAddressStr '" + virtualAddressStr + "'. This might " - + "happen if you are upgrading from an old version of the multistage engine to the current one in a rolling " - + "fashion."); - } - - // Skipped netty and grpc port as they are not used in worker instance. - return new VirtualServerAddress(matcher.group("host"), Integer.parseInt(matcher.group("port")), - Integer.parseInt(matcher.group("virtualid"))); } - public static String addressToProto(VirtualServerAddress serverAddress) { - return String.format("%s@%s:%s", serverAddress.workerId(), serverAddress.hostname(), serverAddress.port()); + public static StagePlan fromProtoStagePlan(Worker.StagePlan protoStagePlan) + throws InvalidProtocolBufferException { + AbstractPlanNode rootNode = + StageNodeSerDeUtils.deserializeStageNode(Plan.StageNode.parseFrom(protoStagePlan.getRootNode())); + StageMetadata stageMetadata = fromProtoStageMetadata(protoStagePlan.getStageMetadata()); + return new StagePlan(protoStagePlan.getStageId(), rootNode, stageMetadata); } - public static List deserializeStagePlan(Worker.StagePlan stagePlan) { - int stageId = stagePlan.getStageId(); - Worker.StageMetadata protoStageMetadata = stagePlan.getStageMetadata(); - String serverAddress = protoStageMetadata.getServerAddress(); - String[] hostPort = StringUtils.split(serverAddress, ':'); - String hostname = hostPort[0]; - int port = Integer.parseInt(hostPort[1]); - AbstractPlanNode stageRoot = StageNodeSerDeUtils.deserializeStageNode(stagePlan.getStageRoot()); - StageMetadata stageMetadata = fromProtoStageMetadata(protoStageMetadata); - List workerIds = protoStageMetadata.getWorkerIdsList(); - List distributedStagePlans = new ArrayList<>(workerIds.size()); - for (int workerId : workerIds) { - distributedStagePlans.add( - new DistributedStagePlan(stageId, new VirtualServerAddress(hostname, port, workerId), stageRoot, - stageMetadata)); - } - return distributedStagePlans; - } - - private static StageMetadata fromProtoStageMetadata(Worker.StageMetadata protoStageMetadata) { - StageMetadata.Builder builder = new StageMetadata.Builder(); - List workerMetadataList = new ArrayList<>(); - for (Worker.WorkerMetadata protoWorkerMetadata : protoStageMetadata.getWorkerMetadataList()) { - workerMetadataList.add(fromProtoWorkerMetadata(protoWorkerMetadata)); - } - builder.setWorkerMetadataList(workerMetadataList); - builder.putAllCustomProperties(protoStageMetadata.getCustomPropertyMap()); - return builder.build(); + private static StageMetadata fromProtoStageMetadata(Worker.StageMetadata protoStageMetadata) + throws InvalidProtocolBufferException { + List workerMetadataList = + protoStageMetadata.getWorkerMetadataList().stream().map(QueryPlanSerDeUtils::fromProtoWorkerMetadata) + .collect(Collectors.toList()); + Map customProperties = fromProtoProperties(protoStageMetadata.getCustomProperty()); + return new StageMetadata(workerMetadataList, customProperties); } private static WorkerMetadata fromProtoWorkerMetadata(Worker.WorkerMetadata protoWorkerMetadata) { - WorkerMetadata.Builder builder = new WorkerMetadata.Builder(); - builder.setVirtualServerAddress(protoToAddress(protoWorkerMetadata.getVirtualAddress())); - builder.putAllMailBoxInfosMap(fromProtoMailboxMetadataMap(protoWorkerMetadata.getMailboxMetadataMap())); - builder.putAllCustomProperties(protoWorkerMetadata.getCustomPropertyMap()); - return builder.build(); - } - - private static Map fromProtoMailboxMetadataMap( - Map mailboxMetadataMap) { - Map mailboxMap = new HashMap<>(); - for (Map.Entry entry : mailboxMetadataMap.entrySet()) { - mailboxMap.put(entry.getKey(), fromProtoMailbox(entry.getValue())); - } - return mailboxMap; + VirtualServerAddress virtualAddress = VirtualServerAddress.parse(protoWorkerMetadata.getVirtualAddress()); + Map mailboxMetadataMap = protoWorkerMetadata.getMailboxMetadataMap().entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> fromProtoMailbox(e.getValue()))); + return new WorkerMetadata(virtualAddress, mailboxMetadataMap, protoWorkerMetadata.getCustomPropertyMap()); } private static MailboxMetadata fromProtoMailbox(Worker.MailboxMetadata protoMailboxMetadata) { - List mailboxIds = new ArrayList<>(); - List virtualAddresses = new ArrayList<>(); - for (int i = 0; i < protoMailboxMetadata.getMailboxIdCount(); i++) { - mailboxIds.add(protoMailboxMetadata.getMailboxId(i)); - virtualAddresses.add(protoToAddress(protoMailboxMetadata.getVirtualAddress(i))); - } - MailboxMetadata mailboxMetadata = - new MailboxMetadata(mailboxIds, virtualAddresses, protoMailboxMetadata.getCustomPropertyMap()); - return mailboxMetadata; + List virtualAddresses = + protoMailboxMetadata.getVirtualAddressList().stream().map(VirtualServerAddress::parse) + .collect(Collectors.toList()); + return new MailboxMetadata(protoMailboxMetadata.getMailboxIdList(), virtualAddresses); } - public static Worker.StageMetadata toProtoStageMetadata(List workerMetadataList, - Map customProperties, QueryServerInstance serverInstance, List workerIds) { - return Worker.StageMetadata.newBuilder().addAllWorkerMetadata(workerMetadataList) - .putAllCustomProperty(customProperties) - .setServerAddress(String.format("%s:%d", serverInstance.getHostname(), serverInstance.getQueryMailboxPort())) - .addAllWorkerIds(workerIds).build(); + public static Map fromProtoProperties(ByteString protoProperties) + throws InvalidProtocolBufferException { + return Worker.Properties.parseFrom(protoProperties).getPropertyMap(); } - public static List toProtoWorkerMetadataList(DispatchablePlanFragment planFragment) { - List workerMetadataList = planFragment.getWorkerMetadataList(); - List protoWorkerMetadataList = new ArrayList<>(workerMetadataList.size()); - for (WorkerMetadata workerMetadata : workerMetadataList) { - protoWorkerMetadataList.add(toProtoWorkerMetadata(workerMetadata)); - } - return protoWorkerMetadataList; + public static List toProtoWorkerMetadataList(List workerMetadataList) { + return workerMetadataList.stream().map(QueryPlanSerDeUtils::toProtoWorkerMetadata).collect(Collectors.toList()); } private static Worker.WorkerMetadata toProtoWorkerMetadata(WorkerMetadata workerMetadata) { - Worker.WorkerMetadata.Builder builder = Worker.WorkerMetadata.newBuilder(); - builder.setVirtualAddress(addressToProto(workerMetadata.getVirtualServerAddress())); - builder.putAllMailboxMetadata(toProtoMailboxMap(workerMetadata.getMailBoxInfosMap())); - builder.putAllCustomProperty(workerMetadata.getCustomProperties()); - return builder.build(); + Map protoMailboxMetadataMap = + workerMetadata.getMailboxMetadataMap().entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> toProtoMailboxMetadata(e.getValue()))); + return Worker.WorkerMetadata.newBuilder().setVirtualAddress(workerMetadata.getVirtualAddress().toString()) + .putAllMailboxMetadata(protoMailboxMetadataMap).putAllCustomProperty(workerMetadata.getCustomProperties()) + .build(); } - private static Map toProtoMailboxMap(Map mailBoxInfosMap) { - Map mailboxMetadataMap = new HashMap<>(); - for (Map.Entry entry : mailBoxInfosMap.entrySet()) { - mailboxMetadataMap.put(entry.getKey(), toProtoMailbox(entry.getValue())); - } - return mailboxMetadataMap; + private static Worker.MailboxMetadata toProtoMailboxMetadata(MailboxMetadata mailboxMetadata) { + List virtualAddresses = + mailboxMetadata.getVirtualAddresses().stream().map(VirtualServerAddress::toString).collect(Collectors.toList()); + return Worker.MailboxMetadata.newBuilder().addAllMailboxId(mailboxMetadata.getMailboxIds()) + .addAllVirtualAddress(virtualAddresses).build(); } - private static Worker.MailboxMetadata toProtoMailbox(MailboxMetadata mailboxMetadata) { - Worker.MailboxMetadata.Builder builder = Worker.MailboxMetadata.newBuilder(); - for (int i = 0; i < mailboxMetadata.getMailBoxIdList().size(); i++) { - builder.addMailboxId(mailboxMetadata.getMailBoxId(i)); - builder.addVirtualAddress(mailboxMetadata.getVirtualAddress(i).toString()); - } - builder.putAllCustomProperty(mailboxMetadata.getCustomProperties()); - return builder.build(); + public static ByteString toProtoProperties(Map properties) { + return Worker.Properties.newBuilder().putAllProperty(properties).build().toByteString(); } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestContext.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestContext.java index 33a955f70908..3c03fa1539df 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestContext.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestContext.java @@ -24,19 +24,19 @@ import org.apache.pinot.core.query.executor.QueryExecutor; import org.apache.pinot.core.query.request.ServerQueryRequest; import org.apache.pinot.query.planner.plannode.PlanNode; -import org.apache.pinot.query.runtime.plan.DistributedStagePlan; +import org.apache.pinot.query.runtime.plan.StagePlan; import org.apache.pinot.query.runtime.plan.pipeline.PipelineBreakerResult; /** - * Context class for converting a {@link org.apache.pinot.query.runtime.plan.DistributedStagePlan} into + * Context class for converting a {@link StagePlan} into * {@link PinotQuery} to execute on server. * * On leaf-stage server node, {@link PlanNode} are split into {@link PinotQuery} part and * {@link org.apache.pinot.query.runtime.operator.OpChain} part. */ public class ServerPlanRequestContext { - private final DistributedStagePlan _stagePlan; + private final StagePlan _stagePlan; private final QueryExecutor _leafQueryExecutor; private final ExecutorService _executorService; private final PipelineBreakerResult _pipelineBreakerResult; @@ -45,7 +45,7 @@ public class ServerPlanRequestContext { private PlanNode _leafStageBoundaryNode; private List _serverQueryRequests; - public ServerPlanRequestContext(DistributedStagePlan stagePlan, QueryExecutor leafQueryExecutor, + public ServerPlanRequestContext(StagePlan stagePlan, QueryExecutor leafQueryExecutor, ExecutorService executorService, PipelineBreakerResult pipelineBreakerResult) { _stagePlan = stagePlan; _leafQueryExecutor = leafQueryExecutor; @@ -54,7 +54,7 @@ public ServerPlanRequestContext(DistributedStagePlan stagePlan, QueryExecutor le _pinotQuery = new PinotQuery(); } - public DistributedStagePlan getStagePlan() { + public StagePlan getStagePlan() { return _stagePlan; } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestUtils.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestUtils.java index 832b2a466677..4c504f71d6bd 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestUtils.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestUtils.java @@ -47,12 +47,11 @@ import org.apache.pinot.core.routing.TimeBoundaryInfo; import org.apache.pinot.query.planner.plannode.JoinNode; import org.apache.pinot.query.planner.plannode.PlanNode; -import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.operator.OpChain; -import org.apache.pinot.query.runtime.plan.DistributedStagePlan; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.query.runtime.plan.PhysicalPlanVisitor; import org.apache.pinot.query.runtime.plan.StageMetadata; +import org.apache.pinot.query.runtime.plan.StagePlan; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.FieldSpec; @@ -64,8 +63,6 @@ import org.apache.pinot.sql.parsers.rewriter.PredicateComparisonRewriter; import org.apache.pinot.sql.parsers.rewriter.QueryRewriter; import org.apache.pinot.sql.parsers.rewriter.QueryRewriterFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class ServerPlanRequestUtils { @@ -73,7 +70,6 @@ private ServerPlanRequestUtils() { } private static final int DEFAULT_LEAF_NODE_LIMIT = Integer.MAX_VALUE; - private static final Logger LOGGER = LoggerFactory.getLogger(ServerPlanRequestUtils.class); private static final List QUERY_REWRITERS_CLASS_NAMES = ImmutableList.of(PredicateComparisonRewriter.class.getName(), NonAggregationGroupByToDistinctQueryRewriter.class.getName()); @@ -82,30 +78,29 @@ private ServerPlanRequestUtils() { private static final QueryOptimizer QUERY_OPTIMIZER = new QueryOptimizer(); /** - * main entry point for compiling leaf-stage {@link DistributedStagePlan}. + * main entry point for compiling leaf-stage {@link StagePlan}. * * @param executionContext the execution context used by the leaf-stage execution engine. - * @param distributedStagePlan the distribute stage plan on the leaf. + * @param stagePlan the distribute stage plan on the leaf. * @return an opChain that executes the leaf-stage, with the leaf-stage execution encapsulated within. */ - public static OpChain compileLeafStage(OpChainExecutionContext executionContext, - DistributedStagePlan distributedStagePlan, HelixManager helixManager, ServerMetrics serverMetrics, - QueryExecutor leafQueryExecutor, ExecutorService executorService) { + public static OpChain compileLeafStage(OpChainExecutionContext executionContext, StagePlan stagePlan, + HelixManager helixManager, ServerMetrics serverMetrics, QueryExecutor leafQueryExecutor, + ExecutorService executorService) { long queryArrivalTimeMs = System.currentTimeMillis(); - ServerPlanRequestContext serverContext = new ServerPlanRequestContext(distributedStagePlan, leafQueryExecutor, - executorService, executionContext.getPipelineBreakerResult()); + ServerPlanRequestContext serverContext = new ServerPlanRequestContext(stagePlan, leafQueryExecutor, executorService, + executionContext.getPipelineBreakerResult()); // 1. compile the PinotQuery constructPinotQueryPlan(serverContext, executionContext.getOpChainMetadata()); // 2. convert PinotQuery into InstanceRequest list (one for each physical table) List instanceRequestList = - ServerPlanRequestUtils.constructServerQueryRequests(executionContext, serverContext, distributedStagePlan, - helixManager.getHelixPropertyStore()); + constructServerQueryRequests(executionContext, serverContext, helixManager.getHelixPropertyStore()); serverContext.setServerQueryRequests(instanceRequestList.stream() .map(instanceRequest -> new ServerQueryRequest(instanceRequest, serverMetrics, queryArrivalTimeMs, true)) .collect(Collectors.toList())); // compile the OpChain executionContext.setLeafStageContext(serverContext); - return PhysicalPlanVisitor.walkPlanNode(distributedStagePlan.getStageRoot(), executionContext); + return PhysicalPlanVisitor.walkPlanNode(stagePlan.getRootNode(), executionContext); } /** @@ -117,18 +112,13 @@ public static OpChain compileLeafStage(OpChainExecutionContext executionContext, */ private static void constructPinotQueryPlan(ServerPlanRequestContext serverContext, Map requestMetadata) { - DistributedStagePlan stagePlan = serverContext.getStagePlan(); + StagePlan stagePlan = serverContext.getStagePlan(); PinotQuery pinotQuery = serverContext.getPinotQuery(); - pinotQuery.setExplain(false); // attach leaf node limit it not set Integer leafNodeLimit = QueryOptionsUtils.getMultiStageLeafLimit(requestMetadata); - if (leafNodeLimit != null) { - pinotQuery.setLimit(leafNodeLimit); - } else { - pinotQuery.setLimit(DEFAULT_LEAF_NODE_LIMIT); - } + pinotQuery.setLimit(leafNodeLimit != null ? leafNodeLimit : DEFAULT_LEAF_NODE_LIMIT); // visit the plan and create PinotQuery and determine the leaf stage boundary PlanNode. - ServerPlanRequestVisitor.walkStageNode(stagePlan.getStageRoot(), serverContext); + ServerPlanRequestVisitor.walkStageNode(stagePlan.getRootNode(), serverContext); } /** @@ -139,17 +129,16 @@ private static void constructPinotQueryPlan(ServerPlanRequestContext serverConte * @return a list of server instance request to be run. */ public static List constructServerQueryRequests(OpChainExecutionContext executionContext, - ServerPlanRequestContext serverContext, DistributedStagePlan distributedStagePlan, - ZkHelixPropertyStore helixPropertyStore) { - StageMetadata stageMetadata = distributedStagePlan.getStageMetadata(); - WorkerMetadata workerMetadata = distributedStagePlan.getCurrentWorkerMetadata(); - String rawTableName = StageMetadata.getTableName(stageMetadata); - int stageId = distributedStagePlan.getStageId(); - Map> tableToSegmentListMap = WorkerMetadata.getTableSegmentsMap(workerMetadata); - List requests = new ArrayList<>(); - for (Map.Entry> tableEntry : tableToSegmentListMap.entrySet()) { - String tableType = tableEntry.getKey(); - List segmentList = tableEntry.getValue(); + ServerPlanRequestContext serverContext, ZkHelixPropertyStore helixPropertyStore) { + int stageId = executionContext.getStageId(); + StageMetadata stageMetadata = executionContext.getStageMetadata(); + String rawTableName = stageMetadata.getTableName(); + Map> tableSegmentsMap = executionContext.getWorkerMetadata().getTableSegmentsMap(); + assert tableSegmentsMap != null; + List requests = new ArrayList<>(tableSegmentsMap.size()); + for (Map.Entry> entry : tableSegmentsMap.entrySet()) { + String tableType = entry.getKey(); + List segments = entry.getValue(); // ZkHelixPropertyStore extends from ZkCacheBaseDataAccessor so it should not cause too much out-of-the-box // network traffic. but there's chance to improve this: // TODO: use TableDataManager: it is already getting tableConfig and Schema when processing segments. @@ -158,15 +147,15 @@ public static List constructServerQueryRequests(OpChainExecutio TableNameBuilder.forType(TableType.OFFLINE).tableNameWithType(rawTableName)); Schema schema = ZKMetadataProvider.getTableSchema(helixPropertyStore, TableNameBuilder.forType(TableType.OFFLINE).tableNameWithType(rawTableName)); - requests.add(ServerPlanRequestUtils.compileInstanceRequest(executionContext, serverContext, stageId, - tableConfig, schema, StageMetadata.getTimeBoundary(stageMetadata), TableType.OFFLINE, segmentList)); + requests.add(compileInstanceRequest(executionContext, serverContext, stageId, tableConfig, schema, + stageMetadata.getTimeBoundary(), TableType.OFFLINE, segments)); } else if (TableType.REALTIME.name().equals(tableType)) { TableConfig tableConfig = ZKMetadataProvider.getTableConfig(helixPropertyStore, TableNameBuilder.forType(TableType.REALTIME).tableNameWithType(rawTableName)); Schema schema = ZKMetadataProvider.getTableSchema(helixPropertyStore, TableNameBuilder.forType(TableType.REALTIME).tableNameWithType(rawTableName)); - requests.add(ServerPlanRequestUtils.compileInstanceRequest(executionContext, serverContext, stageId, - tableConfig, schema, StageMetadata.getTimeBoundary(stageMetadata), TableType.REALTIME, segmentList)); + requests.add(compileInstanceRequest(executionContext, serverContext, stageId, tableConfig, schema, + stageMetadata.getTimeBoundary(), TableType.REALTIME, segments)); } else { throw new IllegalArgumentException("Unsupported table type key: " + tableType); } 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 405f619a9b31..06df28f56129 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 @@ -20,6 +20,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.protobuf.ByteString; import io.grpc.Deadline; import java.util.ArrayList; import java.util.HashMap; @@ -38,7 +39,6 @@ import javax.annotation.Nullable; import org.apache.calcite.util.Pair; import org.apache.pinot.common.datablock.DataBlock; -import org.apache.pinot.common.proto.Plan; import org.apache.pinot.common.proto.Worker; import org.apache.pinot.common.response.broker.ResultTable; import org.apache.pinot.common.utils.DataSchema; @@ -111,41 +111,43 @@ public ResultTable submitAndReduce(RequestContext context, DispatchableSubPlan d void submit(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, Map queryOptions) throws Exception { Deadline deadline = Deadline.after(timeoutMs, TimeUnit.MILLISECONDS); + + // Serialize the stage plans in parallel List stagePlans = dispatchableSubPlan.getQueryStageList(); + Set serverInstances = new HashSet<>(); // Ignore the reduce stage (stage 0) int numStages = stagePlans.size() - 1; - Set serverInstances = new HashSet<>(); - // Serialize the stage plans in parallel - Plan.StageNode[] stageRootNodes = new Plan.StageNode[numStages]; - //noinspection unchecked - List[] stageWorkerMetadataLists = new List[numStages]; - CompletableFuture[] stagePlanSerializationStubs = new CompletableFuture[2 * numStages]; + List> stageInfoFutures = new ArrayList<>(numStages); for (int i = 0; i < numStages; i++) { DispatchablePlanFragment stagePlan = stagePlans.get(i + 1); serverInstances.addAll(stagePlan.getServerInstanceToWorkerIdMap().keySet()); - int finalI = i; - stagePlanSerializationStubs[2 * i] = CompletableFuture.runAsync(() -> stageRootNodes[finalI] = - StageNodeSerDeUtils.serializeStageNode((AbstractPlanNode) stagePlan.getPlanFragment().getFragmentRoot()), - _executorService); - stagePlanSerializationStubs[2 * i + 1] = CompletableFuture.runAsync( - () -> stageWorkerMetadataLists[finalI] = QueryPlanSerDeUtils.toProtoWorkerMetadataList(stagePlan), - _executorService); + stageInfoFutures.add(CompletableFuture.supplyAsync(() -> { + ByteString rootNode = + StageNodeSerDeUtils.serializeStageNode((AbstractPlanNode) stagePlan.getPlanFragment().getFragmentRoot()) + .toByteString(); + ByteString customProperty = QueryPlanSerDeUtils.toProtoProperties(stagePlan.getCustomProperties()); + return new StageInfo(rootNode, customProperty); + }, _executorService)); } + List stageInfos = new ArrayList<>(numStages); try { - CompletableFuture.allOf(stagePlanSerializationStubs) - .get(deadline.timeRemaining(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS); + for (CompletableFuture future : stageInfoFutures) { + stageInfos.add(future.get(deadline.timeRemaining(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS)); + } } finally { - for (CompletableFuture future : stagePlanSerializationStubs) { + for (CompletableFuture future : stageInfoFutures) { if (!future.isDone()) { future.cancel(true); } } } + Map requestMetadata = new HashMap<>(); requestMetadata.put(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID, Long.toString(requestId)); requestMetadata.put(CommonConstants.Broker.Request.QueryOptionKey.TIMEOUT_MS, Long.toString(deadline.timeRemaining(TimeUnit.MILLISECONDS))); requestMetadata.putAll(queryOptions); + ByteString protoRequestMetadata = QueryPlanSerDeUtils.toProtoProperties(requestMetadata); // Submit the query plan to all servers in parallel int numServers = serverInstances.size(); @@ -159,13 +161,23 @@ void submit(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeou DispatchablePlanFragment stagePlan = stagePlans.get(stageId); List workerIds = stagePlan.getServerInstanceToWorkerIdMap().get(serverInstance); if (workerIds != null) { + List stageWorkerMetadataList = stagePlan.getWorkerMetadataList(); + List workerMetadataList = new ArrayList<>(workerIds.size()); + for (int workerId : workerIds) { + workerMetadataList.add(stageWorkerMetadataList.get(workerId)); + } + List protoWorkerMetadataList = + QueryPlanSerDeUtils.toProtoWorkerMetadataList(workerMetadataList); + StageInfo stageInfo = stageInfos.get(i); + Worker.StageMetadata stageMetadata = + Worker.StageMetadata.newBuilder().addAllWorkerMetadata(protoWorkerMetadataList) + .setCustomProperty(stageInfo._customProperty).build(); requestBuilder.addStagePlan( - Worker.StagePlan.newBuilder().setStageId(stageId).setStageRoot(stageRootNodes[i]).setStageMetadata( - QueryPlanSerDeUtils.toProtoStageMetadata(stageWorkerMetadataLists[i], - stagePlan.getCustomProperties(), serverInstance, workerIds)).build()); + Worker.StagePlan.newBuilder().setStageId(stageId).setRootNode(stageInfo._rootNode) + .setStageMetadata(stageMetadata).build()); } } - requestBuilder.putAllMetadata(requestMetadata); + requestBuilder.setMetadata(protoRequestMetadata); getOrCreateDispatchClient(serverInstance).submit(requestBuilder.build(), serverInstance, deadline, dispatchCallbacks::offer); } catch (Throwable t) { @@ -204,6 +216,16 @@ void submit(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeou } } + private static class StageInfo { + final ByteString _rootNode; + final ByteString _customProperty; + + private StageInfo(ByteString rootNode, ByteString customProperty) { + _rootNode = rootNode; + _customProperty = customProperty; + } + } + private void cancel(long requestId, DispatchableSubPlan dispatchableSubPlan) { List stagePlans = dispatchableSubPlan.getQueryStageList(); int numStages = stagePlans.size(); @@ -233,21 +255,19 @@ public static ResultTable runReducer(long requestId, DispatchableSubPlan dispatc Map queryOptions, @Nullable Map statsAggregatorMap, MailboxService mailboxService) { // NOTE: Reduce stage is always stage 0 - DispatchablePlanFragment dispatchablePlanFragment = dispatchableSubPlan.getQueryStageList().get(0); - PlanFragment planFragment = dispatchablePlanFragment.getPlanFragment(); + DispatchablePlanFragment dispatchableStagePlan = dispatchableSubPlan.getQueryStageList().get(0); + PlanFragment planFragment = dispatchableStagePlan.getPlanFragment(); PlanNode rootNode = planFragment.getFragmentRoot(); Preconditions.checkState(rootNode instanceof MailboxReceiveNode, "Expecting mailbox receive node as root of reduce stage, got: %s", rootNode.getClass().getSimpleName()); MailboxReceiveNode receiveNode = (MailboxReceiveNode) rootNode; - List workerMetadataList = dispatchablePlanFragment.getWorkerMetadataList(); + List workerMetadataList = dispatchableStagePlan.getWorkerMetadataList(); Preconditions.checkState(workerMetadataList.size() == 1, "Expecting single worker for reduce stage, got: %s", workerMetadataList.size()); - StageMetadata stageMetadata = new StageMetadata.Builder().setWorkerMetadataList(workerMetadataList) - .addCustomProperties(dispatchablePlanFragment.getCustomProperties()).build(); + StageMetadata stageMetadata = new StageMetadata(workerMetadataList, dispatchableStagePlan.getCustomProperties()); OpChainExecutionContext opChainExecutionContext = new OpChainExecutionContext(mailboxService, requestId, planFragment.getFragmentId(), - workerMetadataList.get(0).getVirtualServerAddress(), System.currentTimeMillis() + timeoutMs, queryOptions, - stageMetadata, null); + System.currentTimeMillis() + timeoutMs, queryOptions, stageMetadata, workerMetadataList.get(0), null); MailboxReceiveOperator receiveOperator = new MailboxReceiveOperator(opChainExecutionContext, receiveNode.getDistributionType(), receiveNode.getSenderStageId()); 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 ecfa9b09f89a..2e52c28a5a57 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 @@ -31,8 +31,10 @@ import org.apache.pinot.common.proto.PinotQueryWorkerGrpc; import org.apache.pinot.common.proto.Worker; import org.apache.pinot.common.utils.NamedThreadFactory; +import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.QueryRunner; -import org.apache.pinot.query.runtime.plan.DistributedStagePlan; +import org.apache.pinot.query.runtime.plan.StageMetadata; +import org.apache.pinot.query.runtime.plan.StagePlan; import org.apache.pinot.query.runtime.plan.serde.QueryPlanSerDeUtils; import org.apache.pinot.query.service.dispatch.QueryDispatcher; import org.apache.pinot.spi.utils.CommonConstants; @@ -95,31 +97,43 @@ public void shutdown() { @Override public void submit(Worker.QueryRequest request, StreamObserver responseObserver) { - Map requestMetadata = request.getMetadataMap(); + Map requestMetadata; + try { + requestMetadata = QueryPlanSerDeUtils.fromProtoProperties(request.getMetadata()); + } catch (Exception e) { + LOGGER.error("Caught exception while deserializing request metadata", e); + responseObserver.onNext(Worker.QueryResponse.newBuilder() + .putMetadata(CommonConstants.Query.Response.ServerResponseStatus.STATUS_ERROR, + QueryException.getTruncatedStackTrace(e)).build()); + responseObserver.onCompleted(); + return; + } long requestId = Long.parseLong(requestMetadata.get(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID)); long timeoutMs = Long.parseLong(requestMetadata.get(CommonConstants.Broker.Request.QueryOptionKey.TIMEOUT_MS)); long deadlineMs = System.currentTimeMillis() + timeoutMs; - List stagePlans = request.getStagePlanList(); - int numStages = stagePlans.size(); + List protoStagePlans = request.getStagePlanList(); + int numStages = protoStagePlans.size(); CompletableFuture[] stageSubmissionStubs = new CompletableFuture[numStages]; for (int i = 0; i < numStages; i++) { - Worker.StagePlan stagePlan = stagePlans.get(i); + Worker.StagePlan protoStagePlan = protoStagePlans.get(i); stageSubmissionStubs[i] = CompletableFuture.runAsync(() -> { - List workerPlans; + StagePlan stagePlan; try { - workerPlans = QueryPlanSerDeUtils.deserializeStagePlan(stagePlan); + stagePlan = QueryPlanSerDeUtils.fromProtoStagePlan(protoStagePlan); } catch (Exception e) { throw new RuntimeException( - String.format("Caught exception while deserializing stage plan for request: %d, stage id: %d", requestId, - stagePlan.getStageId()), e); + String.format("Caught exception while deserializing stage plan for request: %d, stage: %d", requestId, + protoStagePlan.getStageId()), e); } - int numWorkers = workerPlans.size(); + StageMetadata stageMetadata = stagePlan.getStageMetadata(); + List workerMetadataList = stageMetadata.getWorkerMetadataList(); + int numWorkers = workerMetadataList.size(); CompletableFuture[] workerSubmissionStubs = new CompletableFuture[numWorkers]; for (int j = 0; j < numWorkers; j++) { - DistributedStagePlan workerPlan = workerPlans.get(j); + WorkerMetadata workerMetadata = workerMetadataList.get(j); workerSubmissionStubs[j] = - CompletableFuture.runAsync(() -> _queryRunner.processQuery(workerPlan, requestMetadata), + CompletableFuture.runAsync(() -> _queryRunner.processQuery(workerMetadata, stagePlan, requestMetadata), _querySubmissionExecutorService); } try { @@ -127,8 +141,8 @@ public void submit(Worker.QueryRequest request, StreamObserver future : workerSubmissionStubs) { if (!future.isDone()) { diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/QueryServerEnclosure.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/QueryServerEnclosure.java index b4b1dff3cc54..1811218eda9a 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/QueryServerEnclosure.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/QueryServerEnclosure.java @@ -27,8 +27,9 @@ import org.apache.pinot.common.metrics.ServerMetrics; import org.apache.pinot.common.utils.SchemaUtils; import org.apache.pinot.core.data.manager.InstanceDataManager; +import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.QueryRunner; -import org.apache.pinot.query.runtime.plan.DistributedStagePlan; +import org.apache.pinot.query.runtime.plan.StagePlan; import org.apache.pinot.query.testutils.MockInstanceDataManagerFactory; import org.apache.pinot.query.testutils.QueryTestUtils; import org.apache.pinot.spi.data.Schema; @@ -109,9 +110,9 @@ public void shutDown() { _queryRunner.shutDown(); } - public CompletableFuture processQuery(DistributedStagePlan distributedStagePlan, + public CompletableFuture processQuery(WorkerMetadata workerMetadata, StagePlan stagePlan, Map requestMetadataMap) { - return CompletableFuture.runAsync(() -> _queryRunner.processQuery(distributedStagePlan, requestMetadataMap), + return CompletableFuture.runAsync(() -> _queryRunner.processQuery(workerMetadata, stagePlan, requestMetadataMap), _queryRunner.getExecutorService()); } } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/mailbox/MailboxServiceTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/mailbox/MailboxServiceTest.java index 8e6b563ac4f2..3f20d33956c9 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/mailbox/MailboxServiceTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/mailbox/MailboxServiceTest.java @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; +import org.apache.pinot.query.planner.physical.MailboxIdUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.OperatorTestUtil; @@ -299,7 +300,8 @@ public void testLocalEarlyTerminated() SendingMailbox sendingMailbox = _mailboxService1.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); - receivingMailbox.registeredReader(() -> { }); + receivingMailbox.registeredReader(() -> { + }); // send a block sendingMailbox.send(OperatorTestUtil.block(DATA_SCHEMA, new Object[]{0})); @@ -591,15 +593,16 @@ public void testRemoteEarlyTerminated() SendingMailbox sendingMailbox = _mailboxService2.getSendingMailbox("localhost", _mailboxService1.getPort(), mailboxId, Long.MAX_VALUE); ReceivingMailbox receivingMailbox = _mailboxService1.getReceivingMailbox(mailboxId); - receivingMailbox.registeredReader(() -> { }); + receivingMailbox.registeredReader(() -> { + }); // send a block sendingMailbox.send(OperatorTestUtil.block(DATA_SCHEMA, new Object[]{0})); // receiving-side early terminates after pulling the first block TestUtils.waitForCondition(aVoid -> { - TransferableBlock block = receivingMailbox.poll(); - return block != null && block.getNumRows() == 1; - }, 1000L, "Failed to deliver mails"); + TransferableBlock block = receivingMailbox.poll(); + return block != null && block.getNumRows() == 1; + }, 1000L, "Failed to deliver mails"); receivingMailbox.earlyTerminate(); // send another block b/c it doesn't guarantee the next block must be EOS 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 e79f46e671cb..c34c858e7661 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 @@ -18,17 +18,20 @@ */ package org.apache.pinot.query.runtime.executor; -import java.util.Collections; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import org.apache.pinot.common.utils.NamedThreadFactory; import org.apache.pinot.query.routing.VirtualServerAddress; +import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; import org.apache.pinot.query.runtime.operator.MultiStageOperator; import org.apache.pinot.query.runtime.operator.OpChain; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; +import org.apache.pinot.query.runtime.plan.StageMetadata; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; import org.testng.Assert; @@ -67,9 +70,10 @@ public void beforeMethod() { } private OpChain getChain(MultiStageOperator operator) { - VirtualServerAddress address = new VirtualServerAddress("localhost", 1234, 1); - OpChainExecutionContext context = - new OpChainExecutionContext(null, 123L, 1, address, Long.MAX_VALUE, Collections.emptyMap(), null, null); + WorkerMetadata workerMetadata = + new WorkerMetadata(new VirtualServerAddress("localhost", 123, 0), ImmutableMap.of(), ImmutableMap.of()); + OpChainExecutionContext context = new OpChainExecutionContext(null, 123L, 1, Long.MAX_VALUE, ImmutableMap.of(), + new StageMetadata(ImmutableList.of(workerMetadata), ImmutableMap.of()), workerMetadata, null); return new OpChain(context, operator); } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperatorTest.java index 9f5b8dfffe66..1a2949b14286 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperatorTest.java @@ -26,9 +26,9 @@ import org.apache.calcite.rel.RelDistribution; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.utils.DataSchema; -import org.apache.pinot.query.mailbox.MailboxIdUtils; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.mailbox.ReceivingMailbox; +import org.apache.pinot.query.planner.physical.MailboxIdUtils; import org.apache.pinot.query.routing.MailboxMetadata; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.routing.WorkerMetadata; @@ -39,6 +39,7 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -51,12 +52,14 @@ public class MailboxReceiveOperatorTest { - private static final VirtualServerAddress RECEIVER_ADDRESS = new VirtualServerAddress("localhost", 123, 0); private static final DataSchema DATA_SCHEMA = new DataSchema(new String[]{"col1", "col2"}, new DataSchema.ColumnDataType[]{INT, INT}); private static final String MAILBOX_ID_1 = MailboxIdUtils.toMailboxId(0, 1, 0, 0, 0); private static final String MAILBOX_ID_2 = MailboxIdUtils.toMailboxId(0, 1, 1, 0, 0); + private StageMetadata _stageMetadataBoth; + private StageMetadata _stageMetadata1; + private AutoCloseable _mocks; @Mock private MailboxService _mailboxService; @@ -64,40 +67,32 @@ public class MailboxReceiveOperatorTest { private ReceivingMailbox _mailbox1; @Mock private ReceivingMailbox _mailbox2; - private StageMetadata _stageMetadataBoth; - private StageMetadata _stageMetadata1; - @BeforeMethod + @BeforeClass public void setUp() { + VirtualServerAddress server1 = new VirtualServerAddress("localhost", 123, 0); + VirtualServerAddress server2 = new VirtualServerAddress("localhost", 123, 1); + _stageMetadataBoth = new StageMetadata(Stream.of(server1, server2).map(s -> new WorkerMetadata(s, ImmutableMap.of(0, + new MailboxMetadata( + ImmutableList.of(MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0), MailboxIdUtils.toPlanMailboxId(1, 1, 0, 0)), + ImmutableList.of(server1, server2)), 1, new MailboxMetadata( + ImmutableList.of(MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0), MailboxIdUtils.toPlanMailboxId(1, 1, 0, 0)), + ImmutableList.of(server1, server2))), ImmutableMap.of())).collect(Collectors.toList()), ImmutableMap.of()); + _stageMetadata1 = new StageMetadata(ImmutableList.of(new WorkerMetadata(server1, ImmutableMap.of(0, + new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0)), ImmutableList.of(server1)), 1, + new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0)), ImmutableList.of(server1))), + ImmutableMap.of())), ImmutableMap.of()); + } + + @BeforeMethod + public void setUpMethod() { _mocks = MockitoAnnotations.openMocks(this); when(_mailboxService.getHostname()).thenReturn("localhost"); when(_mailboxService.getPort()).thenReturn(123); - VirtualServerAddress server1 = new VirtualServerAddress("localhost", 123, 0); - VirtualServerAddress server2 = new VirtualServerAddress("localhost", 123, 1); - _stageMetadataBoth = new StageMetadata.Builder().setWorkerMetadataList(Stream.of(server1, server2).map( - s -> new WorkerMetadata.Builder().setVirtualServerAddress(s) - .addMailBoxInfoMap(0, new MailboxMetadata( - ImmutableList.of(org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0), - org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 1, 0, 0)), - ImmutableList.of(server1, server2), ImmutableMap.of())) - .addMailBoxInfoMap(1, new MailboxMetadata( - ImmutableList.of(org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0), - org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 1, 0, 0)), - ImmutableList.of(server1, server2), ImmutableMap.of())) - .build()).collect(Collectors.toList())).build(); - _stageMetadata1 = new StageMetadata.Builder().setWorkerMetadataList(Stream.of(server1).map( - s -> new WorkerMetadata.Builder().setVirtualServerAddress(s) - .addMailBoxInfoMap(0, new MailboxMetadata( - ImmutableList.of(org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0)), - ImmutableList.of(server1), ImmutableMap.of())) - .addMailBoxInfoMap(1, new MailboxMetadata( - ImmutableList.of(org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0)), - ImmutableList.of(server1), ImmutableMap.of())) - .build()).collect(Collectors.toList())).build(); } @AfterMethod - public void tearDown() + public void tearDownMethod() throws Exception { _mocks.close(); } @@ -105,7 +100,7 @@ public void tearDown() @Test(expectedExceptions = IllegalStateException.class, expectedExceptionsMessageRegExp = ".*RANGE_DISTRIBUTED.*") public void shouldThrowRangeDistributionNotSupported() { OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadata1); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); //noinspection resource new MailboxReceiveOperator(context, RelDistribution.Type.RANGE_DISTRIBUTED, 1); } @@ -116,8 +111,7 @@ public void shouldTimeout() when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, System.currentTimeMillis() + 1000L, - _stageMetadata1); + OperatorTestUtil.getOpChainContext(_mailboxService, System.currentTimeMillis() + 1000L, _stageMetadata1); try (MailboxReceiveOperator receiveOp = new MailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, 1)) { Thread.sleep(100L); TransferableBlock block = receiveOp.nextBlock(); @@ -132,7 +126,7 @@ public void shouldReceiveEosDirectlyFromSender() { when(_mailbox1.poll()).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadata1); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); try (MailboxReceiveOperator receiveOp = new MailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, 1)) { assertTrue(receiveOp.nextBlock().isEndOfStreamBlock()); } @@ -146,7 +140,7 @@ public void shouldReceiveSingletonMailbox() { TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadata1); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); try (MailboxReceiveOperator receiveOp = new MailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, 1)) { List actualRows = receiveOp.nextBlock().getContainer(); assertEquals(actualRows.size(), 1); @@ -163,7 +157,7 @@ public void shouldReceiveSingletonErrorMailbox() { TransferableBlockUtils.getErrorTransferableBlock(new RuntimeException(errorMessage))); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadata1); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); try (MailboxReceiveOperator receiveOp = new MailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, 1)) { TransferableBlock block = receiveOp.nextBlock(); assertTrue(block.isErrorBlock()); @@ -181,7 +175,7 @@ public void shouldReceiveMailboxFromTwoServersOneNull() { TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadataBoth); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); try (MailboxReceiveOperator receiveOp = new MailboxReceiveOperator(context, RelDistribution.Type.HASH_DISTRIBUTED, 1)) { List actualRows = receiveOp.nextBlock().getContainer(); @@ -204,7 +198,7 @@ public void shouldReceiveMailboxFromTwoServers() { TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadataBoth); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); try (MailboxReceiveOperator receiveOp = new MailboxReceiveOperator(context, RelDistribution.Type.HASH_DISTRIBUTED, 1)) { // Receive first block from server1 @@ -229,7 +223,7 @@ public void shouldGetReceptionReceiveErrorMailbox() { TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadataBoth); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); try (MailboxReceiveOperator receiveOp = new MailboxReceiveOperator(context, RelDistribution.Type.HASH_DISTRIBUTED, 1)) { TransferableBlock block = receiveOp.nextBlock(); @@ -251,7 +245,7 @@ public void shouldEarlyTerminateMailboxesWhenIndicated() { TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadataBoth); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); try (MailboxReceiveOperator receiveOp = new MailboxReceiveOperator(context, RelDistribution.Type.HASH_DISTRIBUTED, 1)) { // Receive first block from server1 diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxSendOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxSendOperatorTest.java index 7a49dcf16a1a..86b2ac0000cb 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxSendOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxSendOperatorTest.java @@ -18,7 +18,8 @@ */ package org.apache.pinot.query.runtime.operator; -import java.util.Collections; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import java.util.List; import java.util.Map; import java.util.concurrent.TimeoutException; @@ -50,7 +51,6 @@ public class MailboxSendOperatorTest { private static final int SENDER_STAGE_ID = 1; private AutoCloseable _mocks; - @Mock private VirtualServerAddress _server; @Mock @@ -61,8 +61,7 @@ public class MailboxSendOperatorTest { private BlockExchange _exchange; @BeforeMethod - public void setUp() - throws Exception { + public void setUpMethod() { _mocks = openMocks(this); when(_server.hostname()).thenReturn("mock"); when(_server.port()).thenReturn(0); @@ -70,7 +69,7 @@ public void setUp() } @AfterMethod - public void tearDown() + public void tearDownMethod() throws Exception { _mocks.close(); } @@ -199,11 +198,11 @@ public void shouldEarlyTerminateWhenUpstreamWhenIndicated() } private MailboxSendOperator getMailboxSendOperator() { - StageMetadata stageMetadata = new StageMetadata.Builder().setWorkerMetadataList( - Collections.singletonList(new WorkerMetadata.Builder().setVirtualServerAddress(_server).build())).build(); + WorkerMetadata workerMetadata = new WorkerMetadata(_server, ImmutableMap.of(), ImmutableMap.of()); + StageMetadata stageMetadata = new StageMetadata(ImmutableList.of(workerMetadata), ImmutableMap.of()); OpChainExecutionContext context = - new OpChainExecutionContext(_mailboxService, 0, SENDER_STAGE_ID, _server, Long.MAX_VALUE, - Collections.emptyMap(), stageMetadata, null); + new OpChainExecutionContext(_mailboxService, 0, SENDER_STAGE_ID, Long.MAX_VALUE, ImmutableMap.of(), + stageMetadata, workerMetadata, null); return new MailboxSendOperator(context, _sourceOperator, _exchange, null, null, false); } } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OpChainTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OpChainTest.java index fb937e623879..a74dec4e6f4d 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OpChainTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OpChainTest.java @@ -22,15 +22,12 @@ import com.google.common.collect.ImmutableMap; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Stack; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; -import java.util.stream.Stream; import javax.annotation.Nullable; import org.apache.calcite.rel.RelDistribution; import org.apache.pinot.common.datatable.DataTable; @@ -77,8 +74,17 @@ public class OpChainTest { private static int _numOperatorsInitialized = 0; private final List _blockList = new ArrayList<>(); - private final ExecutorService _executorService = Executors.newCachedThreadPool(); + private final ExecutorService _executor = Executors.newCachedThreadPool(); private final AtomicReference _leafOpRef = new AtomicReference<>(); + private final VirtualServerAddress _serverAddress = new VirtualServerAddress("localhost", 123, 0); + private final WorkerMetadata _workerMetadata = new WorkerMetadata(_serverAddress, ImmutableMap.of(0, + new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(0, 0, 0, 0)), + ImmutableList.of(_serverAddress)), 1, + new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(0, 0, 0, 0)), + ImmutableList.of(_serverAddress)), 2, + new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(0, 0, 0, 0)), + ImmutableList.of(_serverAddress))), ImmutableMap.of()); + private final StageMetadata _stageMetadata = new StageMetadata(ImmutableList.of(_workerMetadata), ImmutableMap.of()); private AutoCloseable _mocks; @Mock @@ -94,22 +100,9 @@ public class OpChainTest { @Mock private BlockExchange _exchange; - private VirtualServerAddress _serverAddress; - private StageMetadata _receivingStageMetadata; - @BeforeMethod public void setUpMethod() { _mocks = MockitoAnnotations.openMocks(this); - _serverAddress = new VirtualServerAddress("localhost", 123, 0); - _receivingStageMetadata = new StageMetadata.Builder().setWorkerMetadataList(Stream.of(_serverAddress).map( - s -> new WorkerMetadata.Builder().setVirtualServerAddress(s).addMailBoxInfoMap(0, - new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(0, 0, 0, 0)), ImmutableList.of(s), - ImmutableMap.of())).addMailBoxInfoMap(1, - new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(0, 0, 0, 0)), ImmutableList.of(s), - ImmutableMap.of())).addMailBoxInfoMap(2, - new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(0, 0, 0, 0)), ImmutableList.of(s), - ImmutableMap.of())).build()).collect(Collectors.toList())).build(); - when(_mailboxService1.getReceivingMailbox(any())).thenReturn(_mailbox1); when(_mailboxService2.getReceivingMailbox(any())).thenReturn(_mailbox2); @@ -139,7 +132,7 @@ public void tearDownMethod() @AfterClass public void tearDown() { - _executorService.shutdown(); + _executor.shutdown(); } @Test @@ -208,9 +201,8 @@ public void testStatsCollectionTracingEnabledMultipleOperators() { int receivedStageId = 2; int senderStageId = 1; - OpChainExecutionContext context = - new OpChainExecutionContext(_mailboxService1, 1, senderStageId, _serverAddress, Long.MAX_VALUE, - Collections.singletonMap(CommonConstants.Broker.Request.TRACE, "true"), _receivingStageMetadata, null); + OpChainExecutionContext context = new OpChainExecutionContext(_mailboxService1, 1, senderStageId, Long.MAX_VALUE, + ImmutableMap.of(CommonConstants.Broker.Request.TRACE, "true"), _stageMetadata, _workerMetadata, null); Stack operators = getFullOpchain(receivedStageId, senderStageId, context, dummyOperatorWaitTime); @@ -223,8 +215,8 @@ public void testStatsCollectionTracingEnabledMultipleOperators() { opChain.getStats().queued(); OpChainExecutionContext secondStageContext = - new OpChainExecutionContext(_mailboxService2, 1, senderStageId + 1, _serverAddress, Long.MAX_VALUE, - Collections.singletonMap(CommonConstants.Broker.Request.TRACE, "true"), _receivingStageMetadata, null); + new OpChainExecutionContext(_mailboxService2, 1, senderStageId + 1, Long.MAX_VALUE, + ImmutableMap.of(CommonConstants.Broker.Request.TRACE, "true"), _stageMetadata, _workerMetadata, null); MailboxReceiveOperator secondStageReceiveOp = new MailboxReceiveOperator(secondStageContext, RelDistribution.Type.BROADCAST_DISTRIBUTED, senderStageId + 1); @@ -249,8 +241,8 @@ public void testStatsCollectionTracingDisableMultipleOperators() { int receivedStageId = 2; int senderStageId = 1; OpChainExecutionContext context = - new OpChainExecutionContext(_mailboxService1, 1, senderStageId, _serverAddress, Long.MAX_VALUE, - Collections.emptyMap(), _receivingStageMetadata, null); + new OpChainExecutionContext(_mailboxService1, 1, senderStageId, Long.MAX_VALUE, ImmutableMap.of(), + _stageMetadata, _workerMetadata, null); Stack operators = getFullOpchain(receivedStageId, senderStageId, context, dummyOperatorWaitTime); @@ -261,8 +253,8 @@ public void testStatsCollectionTracingDisableMultipleOperators() { opChain.getStats().queued(); OpChainExecutionContext secondStageContext = - new OpChainExecutionContext(_mailboxService2, 1, senderStageId + 1, _serverAddress, Long.MAX_VALUE, - Collections.emptyMap(), _receivingStageMetadata, null); + new OpChainExecutionContext(_mailboxService2, 1, senderStageId + 1, Long.MAX_VALUE, ImmutableMap.of(), + _stageMetadata, _workerMetadata, null); MailboxReceiveOperator secondStageReceiveOp = new MailboxReceiveOperator(secondStageContext, RelDistribution.Type.BROADCAST_DISTRIBUTED, senderStageId); @@ -296,19 +288,19 @@ private Stack getFullOpchain(int receivedStageId, int sender } QueryContext queryContext = QueryContextConverterUtils.getQueryContext("SELECT intCol FROM tbl"); - List dataBlocks = Collections.singletonList( + List dataBlocks = ImmutableList.of( new SelectionResultsBlock(upStreamSchema, Arrays.asList(new Object[]{1}, new Object[]{2}), queryContext)); InstanceResponseBlock metadataBlock = new InstanceResponseBlock(new MetadataResultsBlock()); QueryExecutor queryExecutor = mockQueryExecutor(dataBlocks, metadataBlock); LeafStageTransferableBlockOperator leafOp = - new LeafStageTransferableBlockOperator(context, Collections.singletonList(mock(ServerQueryRequest.class)), - upStreamSchema, queryExecutor, _executorService); + new LeafStageTransferableBlockOperator(context, ImmutableList.of(mock(ServerQueryRequest.class)), + upStreamSchema, queryExecutor, _executor); _leafOpRef.set(leafOp); //Transform operator RexExpression.InputRef ref0 = new RexExpression.InputRef(0); TransformOperator transformOp = - new TransformOperator(context, leafOp, upStreamSchema, Collections.singletonList(ref0), upStreamSchema); + new TransformOperator(context, leafOp, upStreamSchema, ImmutableList.of(ref0), upStreamSchema); //Filter operator RexExpression booleanLiteral = new RexExpression.Literal(ColumnDataType.BOOLEAN, 1); @@ -377,7 +369,7 @@ public DummyMultiStageCallableOperator(OpChainExecutionContext context, MultiSta @Override public List getChildOperators() { - return Collections.singletonList(_upstream); + return ImmutableList.of(_upstream); } @Override diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java index 5f139e554502..3c132269c778 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java @@ -18,15 +18,16 @@ */ package org.apache.pinot.query.runtime.operator; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import org.apache.pinot.common.datablock.DataBlock; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.routing.VirtualServerAddress; +import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.operator.utils.OperatorUtils; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; @@ -38,8 +39,8 @@ public class OperatorTestUtil { // simple key-value collision schema/data test set: "Aa" and "BB" have same hash code in java. private static final List> SIMPLE_KV_DATA_ROWS = - Arrays.asList(Arrays.asList(new Object[]{1, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"}), - Arrays.asList(new Object[]{1, "AA"}, new Object[]{2, "Aa"})); + ImmutableList.of(ImmutableList.of(new Object[]{1, "Aa"}, new Object[]{2, "BB"}, new Object[]{3, "BB"}), + ImmutableList.of(new Object[]{1, "AA"}, new Object[]{2, "Aa"})); private static final MockDataBlockOperatorFactory MOCK_OPERATOR_FACTORY; public static final DataSchema SIMPLE_KV_DATA_SCHEMA = new DataSchema(new String[]{"foo", "bar"}, @@ -75,21 +76,24 @@ public static TransferableBlock block(DataSchema schema, Object[]... rows) { return new TransferableBlock(Arrays.asList(rows), schema, DataBlock.Type.ROW); } - public static OpChainExecutionContext getOpChainContext(MailboxService mailboxService, - VirtualServerAddress receiverAddress, long deadlineMs, StageMetadata stageMetadata) { - return new OpChainExecutionContext(mailboxService, 0, 0, receiverAddress, deadlineMs, Collections.emptyMap(), - stageMetadata, null); + public static OpChainExecutionContext getOpChainContext(MailboxService mailboxService, long deadlineMs, + StageMetadata stageMetadata) { + return new OpChainExecutionContext(mailboxService, 0, 0, deadlineMs, ImmutableMap.of(), stageMetadata, + stageMetadata.getWorkerMetadataList().get(0), null); } public static OpChainExecutionContext getDefaultContext() { - VirtualServerAddress virtualServerAddress = new VirtualServerAddress("mock", 80, 0); - return new OpChainExecutionContext(null, 1, 2, virtualServerAddress, Long.MAX_VALUE, - Collections.singletonMap(CommonConstants.Broker.Request.TRACE, "true"), null, null); + return getDefaultContext(ImmutableMap.of(CommonConstants.Broker.Request.TRACE, "true")); } public static OpChainExecutionContext getDefaultContextWithTracingDisabled() { - VirtualServerAddress virtualServerAddress = new VirtualServerAddress("mock", 80, 0); - return new OpChainExecutionContext(null, 1, 2, virtualServerAddress, Long.MAX_VALUE, Collections.emptyMap(), null, - null); + return getDefaultContext(ImmutableMap.of()); + } + + private static OpChainExecutionContext getDefaultContext(Map opChainMetadata) { + WorkerMetadata workerMetadata = + new WorkerMetadata(new VirtualServerAddress("mock", 80, 0), ImmutableMap.of(), ImmutableMap.of()); + return new OpChainExecutionContext(null, 1, 2, Long.MAX_VALUE, opChainMetadata, + new StageMetadata(ImmutableList.of(workerMetadata), ImmutableMap.of()), workerMetadata, null); } } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperatorTest.java index 4de9e3d4c6db..1e71018215ee 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperatorTest.java @@ -30,10 +30,10 @@ import org.apache.calcite.rel.RelFieldCollation.NullDirection; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.utils.DataSchema; -import org.apache.pinot.query.mailbox.MailboxIdUtils; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.mailbox.ReceivingMailbox; import org.apache.pinot.query.planner.logical.RexExpression; +import org.apache.pinot.query.planner.physical.MailboxIdUtils; import org.apache.pinot.query.routing.MailboxMetadata; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.routing.WorkerMetadata; @@ -44,6 +44,7 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -56,7 +57,6 @@ public class SortedMailboxReceiveOperatorTest { - private static final VirtualServerAddress RECEIVER_ADDRESS = new VirtualServerAddress("localhost", 123, 0); private static final DataSchema DATA_SCHEMA = new DataSchema(new String[]{"col1", "col2"}, new DataSchema.ColumnDataType[]{INT, INT}); private static final List COLLATION_KEYS = Collections.singletonList(new RexExpression.InputRef(0)); @@ -65,6 +65,9 @@ public class SortedMailboxReceiveOperatorTest { private static final String MAILBOX_ID_1 = MailboxIdUtils.toMailboxId(0, 1, 0, 0, 0); private static final String MAILBOX_ID_2 = MailboxIdUtils.toMailboxId(0, 1, 1, 0, 0); + private StageMetadata _stageMetadataBoth; + private StageMetadata _stageMetadata1; + private AutoCloseable _mocks; @Mock private MailboxService _mailboxService; @@ -73,40 +76,31 @@ public class SortedMailboxReceiveOperatorTest { @Mock private ReceivingMailbox _mailbox2; - private StageMetadata _stageMetadataBoth; - private StageMetadata _stageMetadata1; + @BeforeClass + public void setUp() { + VirtualServerAddress server1 = new VirtualServerAddress("localhost", 123, 0); + VirtualServerAddress server2 = new VirtualServerAddress("localhost", 123, 1); + _stageMetadataBoth = new StageMetadata(Stream.of(server1, server2).map(s -> new WorkerMetadata(s, ImmutableMap.of(0, + new MailboxMetadata( + ImmutableList.of(MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0), MailboxIdUtils.toPlanMailboxId(1, 1, 0, 0)), + ImmutableList.of(server1, server2)), 1, new MailboxMetadata( + ImmutableList.of(MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0), MailboxIdUtils.toPlanMailboxId(1, 1, 0, 0)), + ImmutableList.of(server1, server2))), ImmutableMap.of())).collect(Collectors.toList()), ImmutableMap.of()); + _stageMetadata1 = new StageMetadata(ImmutableList.of(new WorkerMetadata(server1, ImmutableMap.of(0, + new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0)), ImmutableList.of(server1)), 1, + new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0)), ImmutableList.of(server1))), + ImmutableMap.of())), ImmutableMap.of()); + } @BeforeMethod - public void setUp() { + public void setUpMethod() { _mocks = MockitoAnnotations.openMocks(this); when(_mailboxService.getHostname()).thenReturn("localhost"); when(_mailboxService.getPort()).thenReturn(123); - VirtualServerAddress server1 = new VirtualServerAddress("localhost", 123, 0); - VirtualServerAddress server2 = new VirtualServerAddress("localhost", 123, 1); - _stageMetadataBoth = new StageMetadata.Builder().setWorkerMetadataList(Stream.of(server1, server2).map( - s -> new WorkerMetadata.Builder().setVirtualServerAddress(s) - .addMailBoxInfoMap(0, new MailboxMetadata( - ImmutableList.of(org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0), - org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 1, 0, 0)), - ImmutableList.of(server1, server2), ImmutableMap.of())) - .addMailBoxInfoMap(1, new MailboxMetadata( - ImmutableList.of(org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0), - org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 1, 0, 0)), - ImmutableList.of(server1, server2), ImmutableMap.of())) - .build()).collect(Collectors.toList())).build(); - _stageMetadata1 = new StageMetadata.Builder().setWorkerMetadataList(Stream.of(server1).map( - s -> new WorkerMetadata.Builder().setVirtualServerAddress(s) - .addMailBoxInfoMap(0, new MailboxMetadata( - ImmutableList.of(org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0)), - ImmutableList.of(server1), ImmutableMap.of())) - .addMailBoxInfoMap(1, new MailboxMetadata( - ImmutableList.of(org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0)), - ImmutableList.of(server1), ImmutableMap.of())) - .build()).collect(Collectors.toList())).build(); } @AfterMethod - public void tearDown() + public void tearDownMethod() throws Exception { _mocks.close(); } @@ -114,7 +108,7 @@ public void tearDown() @Test(expectedExceptions = IllegalStateException.class, expectedExceptionsMessageRegExp = ".*RANGE_DISTRIBUTED.*") public void shouldThrowRangeDistributionNotSupported() { OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadata1); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); //noinspection resource new SortedMailboxReceiveOperator(context, RelDistribution.Type.RANGE_DISTRIBUTED, DATA_SCHEMA, COLLATION_KEYS, COLLATION_DIRECTIONS, COLLATION_NULL_DIRECTIONS, false, 1); @@ -124,7 +118,7 @@ public void shouldThrowRangeDistributionNotSupported() { public void shouldThrowOnEmptyCollationKey() { when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadata1); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); //noinspection resource new SortedMailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, DATA_SCHEMA, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), false, 1); @@ -136,8 +130,7 @@ public void shouldTimeout() when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, System.currentTimeMillis() + 1000L, - _stageMetadata1); + OperatorTestUtil.getOpChainContext(_mailboxService, System.currentTimeMillis() + 1000L, _stageMetadata1); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, DATA_SCHEMA, COLLATION_KEYS, COLLATION_DIRECTIONS, COLLATION_NULL_DIRECTIONS, false, 1)) { @@ -153,7 +146,7 @@ public void shouldReceiveEosDirectlyFromSender() { when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1); when(_mailbox1.poll()).thenReturn(TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadata1); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, DATA_SCHEMA, COLLATION_KEYS, COLLATION_DIRECTIONS, COLLATION_NULL_DIRECTIONS, false, 1)) { @@ -168,7 +161,7 @@ public void shouldReceiveSingletonMailbox() { when(_mailbox1.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row), TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadata1); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, DATA_SCHEMA, COLLATION_KEYS, COLLATION_DIRECTIONS, COLLATION_NULL_DIRECTIONS, false, 1)) { @@ -186,7 +179,7 @@ public void shouldReceiveSingletonErrorMailbox() { when(_mailbox1.poll()).thenReturn( TransferableBlockUtils.getErrorTransferableBlock(new RuntimeException(errorMessage))); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadata1); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadata1); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, DATA_SCHEMA, COLLATION_KEYS, COLLATION_DIRECTIONS, COLLATION_NULL_DIRECTIONS, false, 1)) { @@ -205,7 +198,7 @@ public void shouldReceiveMailboxFromTwoServersOneNull() { when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row), TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadataBoth); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, RelDistribution.Type.HASH_DISTRIBUTED, DATA_SCHEMA, COLLATION_KEYS, COLLATION_DIRECTIONS, COLLATION_NULL_DIRECTIONS, false, 1)) { @@ -230,7 +223,7 @@ public void shouldGetReceptionReceiveErrorMailbox() { when(_mailbox2.poll()).thenReturn(OperatorTestUtil.block(DATA_SCHEMA, row), TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadataBoth); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, RelDistribution.Type.HASH_DISTRIBUTED, DATA_SCHEMA, COLLATION_KEYS, COLLATION_DIRECTIONS, COLLATION_NULL_DIRECTIONS, false, 1)) { @@ -255,7 +248,7 @@ public void shouldReceiveMailboxFromTwoServersWithCollationKey() { OperatorTestUtil.block(DATA_SCHEMA, row4), OperatorTestUtil.block(DATA_SCHEMA, row5), TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadataBoth); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, RelDistribution.Type.HASH_DISTRIBUTED, DATA_SCHEMA, COLLATION_KEYS, COLLATION_DIRECTIONS, COLLATION_NULL_DIRECTIONS, false, 1)) { @@ -286,7 +279,7 @@ public void shouldReceiveMailboxFromTwoServersWithCollationKeyTwoColumns() { TransferableBlockUtils.getEndOfStreamTransferableBlock()); OpChainExecutionContext context = - OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, Long.MAX_VALUE, _stageMetadataBoth); + OperatorTestUtil.getOpChainContext(_mailboxService, Long.MAX_VALUE, _stageMetadataBoth); try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context, RelDistribution.Type.HASH_DISTRIBUTED, dataSchema, collationKeys, collationDirections, collationNullDirections, false, 1)) { diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java index e7fa0e7db29d..94d5e2b8732c 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerExecutorTest.java @@ -20,22 +20,18 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.calcite.rel.RelDistribution; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.logical.PinotRelExchangeType; import org.apache.pinot.common.utils.DataSchema; -import org.apache.pinot.common.utils.NamedThreadFactory; -import org.apache.pinot.query.mailbox.MailboxIdUtils; import org.apache.pinot.query.mailbox.MailboxService; import org.apache.pinot.query.mailbox.ReceivingMailbox; +import org.apache.pinot.query.planner.physical.MailboxIdUtils; import org.apache.pinot.query.planner.plannode.JoinNode; import org.apache.pinot.query.planner.plannode.MailboxReceiveNode; import org.apache.pinot.query.routing.MailboxMetadata; @@ -46,8 +42,8 @@ import org.apache.pinot.query.runtime.executor.ExecutorServiceUtils; import org.apache.pinot.query.runtime.executor.OpChainSchedulerService; import org.apache.pinot.query.runtime.operator.OperatorTestUtil; -import org.apache.pinot.query.runtime.plan.DistributedStagePlan; import org.apache.pinot.query.runtime.plan.StageMetadata; +import org.apache.pinot.query.runtime.plan.StagePlan; import org.mockito.Mock; import org.mockito.MockitoAnnotations; import org.testng.Assert; @@ -61,12 +57,22 @@ public class PipelineBreakerExecutorTest { - private static final VirtualServerAddress RECEIVER_ADDRESS = new VirtualServerAddress("localhost", 123, 0); private static final DataSchema DATA_SCHEMA = new DataSchema(new String[]{"col1", "col2"}, new DataSchema.ColumnDataType[]{INT, INT}); private static final String MAILBOX_ID_1 = MailboxIdUtils.toMailboxId(0, 1, 0, 0, 0); private static final String MAILBOX_ID_2 = MailboxIdUtils.toMailboxId(0, 2, 0, 0, 0); + private final VirtualServerAddress _server = new VirtualServerAddress("localhost", 123, 0); + private final ExecutorService _executor = Executors.newCachedThreadPool(); + private final OpChainSchedulerService _scheduler = new OpChainSchedulerService(_executor); + private final WorkerMetadata _workerMetadata = new WorkerMetadata(_server, ImmutableMap.of(0, new MailboxMetadata( + ImmutableList.of(MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0), MailboxIdUtils.toPlanMailboxId(2, 0, 0, 0)), + ImmutableList.of(_server, _server)), 1, + new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0)), ImmutableList.of(_server)), 2, + new MailboxMetadata(ImmutableList.of(MailboxIdUtils.toPlanMailboxId(2, 0, 0, 0)), ImmutableList.of(_server))), + ImmutableMap.of()); + private final StageMetadata _stageMetadata = new StageMetadata(ImmutableList.of(_workerMetadata), ImmutableMap.of()); + private AutoCloseable _mocks; @Mock private MailboxService _mailboxService; @@ -75,31 +81,8 @@ public class PipelineBreakerExecutorTest { @Mock private ReceivingMailbox _mailbox2; - private VirtualServerAddress _server = new VirtualServerAddress("localhost", 123, 0); - private ExecutorService _executor = Executors.newCachedThreadPool( - new NamedThreadFactory("worker_on_asd_" + getClass().getSimpleName())); - private OpChainSchedulerService _scheduler = new OpChainSchedulerService(_executor); - private StageMetadata _stageMetadata1 = new StageMetadata.Builder().setWorkerMetadataList(Stream.of(_server).map( - s -> new WorkerMetadata.Builder().setVirtualServerAddress(s) - .addMailBoxInfoMap(0, new MailboxMetadata( - ImmutableList.of(org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0), - org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(2, 0, 0, 0)), - ImmutableList.of(_server), ImmutableMap.of())) - .addMailBoxInfoMap(1, new MailboxMetadata( - ImmutableList.of(org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(1, 0, 0, 0)), - ImmutableList.of(_server), ImmutableMap.of())) - .addMailBoxInfoMap(2, new MailboxMetadata( - ImmutableList.of(org.apache.pinot.query.planner.physical.MailboxIdUtils.toPlanMailboxId(2, 0, 0, 0)), - ImmutableList.of(_server), ImmutableMap.of())) - .build()).collect(Collectors.toList())).build(); - - @AfterClass - public void tearDownClass() { - ExecutorServiceUtils.close(_executor); - } - @BeforeMethod - public void setUp() { + public void setUpMethod() { _mocks = MockitoAnnotations.openMocks(this); when(_mailboxService.getHostname()).thenReturn("localhost"); when(_mailboxService.getPort()).thenReturn(123); @@ -109,18 +92,22 @@ public void setUp() { } @AfterMethod - public void tearDown() + public void tearDownMethod() throws Exception { _mocks.close(); } + @AfterClass + public void tearDown() { + ExecutorServiceUtils.close(_executor); + } + @Test public void shouldReturnBlocksUponNormalOperation() { MailboxReceiveNode mailboxReceiveNode = new MailboxReceiveNode(0, DATA_SCHEMA, 1, RelDistribution.Type.SINGLETON, PinotRelExchangeType.PIPELINE_BREAKER, null, null, false, false, null); - DistributedStagePlan distributedStagePlan = - new DistributedStagePlan(0, RECEIVER_ADDRESS, mailboxReceiveNode, _stageMetadata1); + StagePlan stagePlan = new StagePlan(0, mailboxReceiveNode, _stageMetadata); // when when(_mailboxService.getReceivingMailbox(MAILBOX_ID_1)).thenReturn(_mailbox1); @@ -131,8 +118,8 @@ public void shouldReturnBlocksUponNormalOperation() { TransferableBlockUtils.getEndOfStreamTransferableBlock(OperatorTestUtil.getDummyStats(0, 1, _server))); PipelineBreakerResult pipelineBreakerResult = - PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, distributedStagePlan, - Collections.emptyMap(), 0, Long.MAX_VALUE); + PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, _workerMetadata, stagePlan, + ImmutableMap.of(), 0, Long.MAX_VALUE); // then // should have single PB result, receive 2 data blocks, EOS block shouldn't be included @@ -155,11 +142,10 @@ public void shouldWorkWithMultiplePBNodeUponNormalOperation() { new MailboxReceiveNode(0, DATA_SCHEMA, 2, RelDistribution.Type.SINGLETON, PinotRelExchangeType.PIPELINE_BREAKER, null, null, false, false, null); JoinNode joinNode = - new JoinNode(0, DATA_SCHEMA, DATA_SCHEMA, DATA_SCHEMA, JoinRelType.INNER, null, null, Collections.emptyList()); + new JoinNode(0, DATA_SCHEMA, DATA_SCHEMA, DATA_SCHEMA, JoinRelType.INNER, null, null, ImmutableList.of()); joinNode.addInput(mailboxReceiveNode1); joinNode.addInput(mailboxReceiveNode2); - DistributedStagePlan distributedStagePlan = - new DistributedStagePlan(0, RECEIVER_ADDRESS, joinNode, _stageMetadata1); + StagePlan stagePlan = new StagePlan(0, joinNode, _stageMetadata); // when when(_mailboxService.getReceivingMailbox(MAILBOX_ID_1)).thenReturn(_mailbox1); @@ -172,8 +158,8 @@ public void shouldWorkWithMultiplePBNodeUponNormalOperation() { TransferableBlockUtils.getEndOfStreamTransferableBlock(OperatorTestUtil.getDummyStats(0, 2, _server))); PipelineBreakerResult pipelineBreakerResult = - PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, distributedStagePlan, - Collections.emptyMap(), 0, Long.MAX_VALUE); + PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, _workerMetadata, stagePlan, + ImmutableMap.of(), 0, Long.MAX_VALUE); // then // should have two PB result, receive 2 data blocks, one each, EOS block shouldn't be included @@ -195,13 +181,12 @@ public void shouldReturnEmptyBlockWhenPBExecuteWithIncorrectMailboxNode() { MailboxReceiveNode incorrectlyConfiguredMailboxNode = new MailboxReceiveNode(0, DATA_SCHEMA, 3, RelDistribution.Type.SINGLETON, PinotRelExchangeType.PIPELINE_BREAKER, null, null, false, false, null); - DistributedStagePlan distributedStagePlan = - new DistributedStagePlan(0, RECEIVER_ADDRESS, incorrectlyConfiguredMailboxNode, _stageMetadata1); + StagePlan stagePlan = new StagePlan(0, incorrectlyConfiguredMailboxNode, _stageMetadata); // when PipelineBreakerResult pipelineBreakerResult = - PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, distributedStagePlan, - Collections.emptyMap(), 0, Long.MAX_VALUE); + PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, _workerMetadata, stagePlan, + ImmutableMap.of(), 0, Long.MAX_VALUE); // then // should return empty block list @@ -219,8 +204,7 @@ public void shouldReturnErrorBlocksFailureWhenPBTimeout() { MailboxReceiveNode mailboxReceiveNode = new MailboxReceiveNode(0, DATA_SCHEMA, 1, RelDistribution.Type.SINGLETON, PinotRelExchangeType.PIPELINE_BREAKER, null, null, false, false, null); - DistributedStagePlan distributedStagePlan = - new DistributedStagePlan(0, RECEIVER_ADDRESS, mailboxReceiveNode, _stageMetadata1); + StagePlan stagePlan = new StagePlan(0, mailboxReceiveNode, _stageMetadata); // when when(_mailboxService.getReceivingMailbox(MAILBOX_ID_1)).thenReturn(_mailbox1); @@ -231,8 +215,8 @@ public void shouldReturnErrorBlocksFailureWhenPBTimeout() { }); PipelineBreakerResult pipelineBreakerResult = - PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, distributedStagePlan, - Collections.emptyMap(), 0, System.currentTimeMillis() + 100); + PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, _workerMetadata, stagePlan, + ImmutableMap.of(), 0, System.currentTimeMillis() + 100); // then // should contain only failure error blocks @@ -253,11 +237,10 @@ public void shouldReturnWhenAnyPBReturnsEmpty() { new MailboxReceiveNode(0, DATA_SCHEMA, 3, RelDistribution.Type.SINGLETON, PinotRelExchangeType.PIPELINE_BREAKER, null, null, false, false, null); JoinNode joinNode = - new JoinNode(0, DATA_SCHEMA, DATA_SCHEMA, DATA_SCHEMA, JoinRelType.INNER, null, null, Collections.emptyList()); + new JoinNode(0, DATA_SCHEMA, DATA_SCHEMA, DATA_SCHEMA, JoinRelType.INNER, null, null, ImmutableList.of()); joinNode.addInput(mailboxReceiveNode1); joinNode.addInput(incorrectlyConfiguredMailboxNode); - DistributedStagePlan distributedStagePlan = - new DistributedStagePlan(0, RECEIVER_ADDRESS, joinNode, _stageMetadata1); + StagePlan stagePlan = new StagePlan(0, joinNode, _stageMetadata); // when when(_mailboxService.getReceivingMailbox(MAILBOX_ID_1)).thenReturn(_mailbox1); @@ -270,8 +253,8 @@ public void shouldReturnWhenAnyPBReturnsEmpty() { TransferableBlockUtils.getEndOfStreamTransferableBlock()); PipelineBreakerResult pipelineBreakerResult = - PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, distributedStagePlan, - Collections.emptyMap(), 0, Long.MAX_VALUE); + PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, _workerMetadata, stagePlan, + ImmutableMap.of(), 0, Long.MAX_VALUE); // then // should pass when one PB returns result, the other returns empty. @@ -292,11 +275,10 @@ public void shouldReturnErrorBlocksWhenReceivedErrorFromSender() { new MailboxReceiveNode(0, DATA_SCHEMA, 2, RelDistribution.Type.SINGLETON, PinotRelExchangeType.PIPELINE_BREAKER, null, null, false, false, null); JoinNode joinNode = - new JoinNode(0, DATA_SCHEMA, DATA_SCHEMA, DATA_SCHEMA, JoinRelType.INNER, null, null, Collections.emptyList()); + new JoinNode(0, DATA_SCHEMA, DATA_SCHEMA, DATA_SCHEMA, JoinRelType.INNER, null, null, ImmutableList.of()); joinNode.addInput(mailboxReceiveNode1); joinNode.addInput(incorrectlyConfiguredMailboxNode); - DistributedStagePlan distributedStagePlan = - new DistributedStagePlan(0, RECEIVER_ADDRESS, joinNode, _stageMetadata1); + StagePlan stagePlan = new StagePlan(0, joinNode, _stageMetadata); // when when(_mailboxService.getReceivingMailbox(MAILBOX_ID_1)).thenReturn(_mailbox1); @@ -309,8 +291,8 @@ public void shouldReturnErrorBlocksWhenReceivedErrorFromSender() { TransferableBlockUtils.getEndOfStreamTransferableBlock()); PipelineBreakerResult pipelineBreakerResult = - PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, distributedStagePlan, - Collections.emptyMap(), 0, Long.MAX_VALUE); + PipelineBreakerExecutor.executePipelineBreakers(_scheduler, _mailboxService, _workerMetadata, stagePlan, + ImmutableMap.of(), 0, Long.MAX_VALUE); // then // should fail even if one of the 2 PB doesn't contain error block from sender. diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtilsTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtilsTest.java deleted file mode 100644 index 9ca24ebf484b..000000000000 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/serde/QueryPlanSerDeUtilsTest.java +++ /dev/null @@ -1,58 +0,0 @@ -/** - * 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.serde; - -import org.apache.pinot.query.routing.VirtualServerAddress; -import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.Test; - - -public class QueryPlanSerDeUtilsTest { - - @Test - public void shouldSerializeServer() { - // Given: - VirtualServerAddress server = Mockito.mock(VirtualServerAddress.class); - Mockito.when(server.workerId()).thenReturn(1); - Mockito.when(server.hostname()).thenReturn("Server_192.987.1.123"); - Mockito.when(server.port()).thenReturn(80); - - // When: - String serialized = QueryPlanSerDeUtils.addressToProto(server); - - // Then: - Assert.assertEquals(serialized, "1@Server_192.987.1.123:80"); - } - - @Test - public void shouldDeserializeServerString() { - // Given: - String serverString = "1@Server_192.987.1.123:80"; - - // When: - VirtualServerAddress server = QueryPlanSerDeUtils.protoToAddress(serverString); - - // Then: - Assert.assertEquals(server.workerId(), 1); - Assert.assertEquals(server.hostname(), "Server_192.987.1.123"); - Assert.assertEquals(server.port(), 80); - } -} diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java index 33b68f807e77..f1315aa1bf7d 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java @@ -58,9 +58,9 @@ import org.apache.pinot.query.planner.physical.DispatchablePlanFragment; import org.apache.pinot.query.planner.physical.DispatchableSubPlan; import org.apache.pinot.query.routing.QueryServerInstance; -import org.apache.pinot.query.routing.VirtualServerAddress; -import org.apache.pinot.query.runtime.plan.DistributedStagePlan; +import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.plan.StageMetadata; +import org.apache.pinot.query.runtime.plan.StagePlan; import org.apache.pinot.query.service.dispatch.QueryDispatcher; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.Schema; @@ -154,29 +154,24 @@ protected ResultTable queryRunner(String sql, Map> processDistributedStagePlans(DispatchableSubPlan dispatchableSubPlan, int stageId, Map requestMetadataMap) { - Map> serverInstanceToWorkerIdMap = - dispatchableSubPlan.getQueryStageList().get(stageId).getServerInstanceToWorkerIdMap(); + DispatchablePlanFragment dispatchableStagePlan = dispatchableSubPlan.getQueryStageList().get(stageId); + List stageWorkerMetadataList = dispatchableStagePlan.getWorkerMetadataList(); List> submissionStubs = new ArrayList<>(); - for (Map.Entry> entry : serverInstanceToWorkerIdMap.entrySet()) { - QueryServerInstance server = entry.getKey(); - for (int workerId : entry.getValue()) { - DistributedStagePlan distributedStagePlan = - constructDistributedStagePlan(dispatchableSubPlan, stageId, new VirtualServerAddress(server, workerId)); - submissionStubs.add(_servers.get(server).processQuery(distributedStagePlan, requestMetadataMap)); + for (Map.Entry> entry : dispatchableStagePlan.getServerInstanceToWorkerIdMap() + .entrySet()) { + QueryServerEnclosure serverEnclosure = _servers.get(entry.getKey()); + List workerMetadataList = + entry.getValue().stream().map(stageWorkerMetadataList::get).collect(Collectors.toList()); + StageMetadata stageMetadata = new StageMetadata(workerMetadataList, dispatchableStagePlan.getCustomProperties()); + StagePlan stagePlan = + new StagePlan(stageId, dispatchableStagePlan.getPlanFragment().getFragmentRoot(), stageMetadata); + for (WorkerMetadata workerMetadata : workerMetadataList) { + submissionStubs.add(serverEnclosure.processQuery(workerMetadata, stagePlan, requestMetadataMap)); } } return submissionStubs; } - protected static DistributedStagePlan constructDistributedStagePlan(DispatchableSubPlan dispatchableSubPlan, - int stageId, VirtualServerAddress serverAddress) { - return new DistributedStagePlan(stageId, serverAddress, - dispatchableSubPlan.getQueryStageList().get(stageId).getPlanFragment().getFragmentRoot(), - new StageMetadata.Builder().setWorkerMetadataList( - dispatchableSubPlan.getQueryStageList().get(stageId).getWorkerMetadataList()) - .addCustomProperties(dispatchableSubPlan.getQueryStageList().get(stageId).getCustomProperties()).build()); - } - protected List queryH2(String sql) throws Exception { int firstSemi = sql.indexOf(';'); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/server/QueryServerTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/server/QueryServerTest.java index 4e5a0034273c..679f46c60c93 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/server/QueryServerTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/server/QueryServerTest.java @@ -19,6 +19,7 @@ package org.apache.pinot.query.service.server; import com.google.common.collect.Lists; +import com.google.protobuf.ByteString; import io.grpc.Deadline; import io.grpc.ManagedChannel; import io.grpc.ManagedChannelBuilder; @@ -26,6 +27,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Random; import java.util.concurrent.TimeUnit; import org.apache.pinot.common.proto.PinotQueryWorkerGrpc; @@ -49,12 +51,15 @@ import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.EqualityUtils; import org.apache.pinot.util.TestUtils; -import org.mockito.Mockito; -import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.Mockito.*; +import static org.testng.Assert.assertTrue; + public class QueryServerTest extends QueryTestSet { private static final Random RANDOM_REQUEST_ID_GEN = new Random(); @@ -70,10 +75,9 @@ public class QueryServerTest extends QueryTestSet { @BeforeClass public void setUp() throws Exception { - for (int i = 0; i < QUERY_SERVER_COUNT; i++) { int availablePort = QueryTestUtils.getAvailablePort(); - QueryRunner queryRunner = Mockito.mock(QueryRunner.class); + QueryRunner queryRunner = mock(QueryRunner.class); QueryServer queryServer = new QueryServer(availablePort, queryRunner); queryServer.start(); _queryServerMap.put(availablePort, queryServer); @@ -96,85 +100,81 @@ public void tearDown() { } @Test - public void testException() { - DispatchableSubPlan dispatchableSubPlan = _queryEnvironment.planQuery("SELECT * FROM a"); + public void testException() + throws Exception { + DispatchableSubPlan queryPlan = _queryEnvironment.planQuery("SELECT * FROM a"); // only get one worker request out. - Worker.QueryRequest queryRequest = getQueryRequest(dispatchableSubPlan, 1); - QueryRunner mockRunner = - _queryRunnerMap.get(Integer.parseInt(queryRequest.getMetadataOrThrow(KEY_OF_SERVER_INSTANCE_PORT))); - Mockito.doThrow(new RuntimeException("foo")).when(mockRunner).processQuery(Mockito.any(), Mockito.anyMap()); + Worker.QueryRequest queryRequest = getQueryRequest(queryPlan, 1); + Map requestMetadata = QueryPlanSerDeUtils.fromProtoProperties(queryRequest.getMetadata()); + QueryRunner mockRunner = _queryRunnerMap.get(Integer.parseInt(requestMetadata.get(KEY_OF_SERVER_INSTANCE_PORT))); + doThrow(new RuntimeException("foo")).when(mockRunner).processQuery(any(), any(), any()); // submit the request for testing. - Worker.QueryResponse resp = submitRequest(queryRequest); + Worker.QueryResponse resp = submitRequest(queryRequest, requestMetadata); // reset the mock runner before assert. - Mockito.reset(mockRunner); + reset(mockRunner); // should contain error message pattern String errorMessage = resp.getMetadataMap().get(CommonConstants.Query.Response.ServerResponseStatus.STATUS_ERROR); - Assert.assertTrue(errorMessage.contains("foo")); + assertTrue(errorMessage.contains("foo")); } @Test(dataProvider = "testSql") public void testWorkerAcceptsWorkerRequestCorrect(String sql) throws Exception { - DispatchableSubPlan dispatchableSubPlan = _queryEnvironment.planQuery(sql); - - for (int stageId = 0; stageId < dispatchableSubPlan.getQueryStageList().size(); stageId++) { - if (stageId > 0) { // we do not test reduce stage. - // only get one worker request out. - Worker.QueryRequest queryRequest = getQueryRequest(dispatchableSubPlan, stageId); - - // submit the request for testing. - Worker.QueryResponse resp = submitRequest(queryRequest); - Assert.assertNotNull(resp.getMetadataMap().get(CommonConstants.Query.Response.ServerResponseStatus.STATUS_OK)); + DispatchableSubPlan queryPlan = _queryEnvironment.planQuery(sql); + List stagePlans = queryPlan.getQueryStageList(); + int numStages = stagePlans.size(); + // Ignore reduce stage (stage 0) + for (int stageId = 1; stageId < numStages; stageId++) { + // only get one worker request out. + Worker.QueryRequest queryRequest = getQueryRequest(queryPlan, stageId); + Map requestMetadata = QueryPlanSerDeUtils.fromProtoProperties(queryRequest.getMetadata()); - DispatchablePlanFragment dispatchablePlanFragment = dispatchableSubPlan.getQueryStageList().get(stageId); + // submit the request for testing. + Worker.QueryResponse resp = submitRequest(queryRequest, requestMetadata); + assertTrue(resp.getMetadataMap().containsKey(CommonConstants.Query.Response.ServerResponseStatus.STATUS_OK)); - StageMetadata stageMetadata = - new StageMetadata.Builder().setWorkerMetadataList(dispatchablePlanFragment.getWorkerMetadataList()) - .addCustomProperties(dispatchablePlanFragment.getCustomProperties()).build(); + DispatchablePlanFragment dispatchableStagePlan = stagePlans.get(stageId); + List workerMetadataList = dispatchableStagePlan.getWorkerMetadataList(); + StageMetadata stageMetadata = new StageMetadata(workerMetadataList, dispatchableStagePlan.getCustomProperties()); - // ensure mock query runner received correctly deserialized payload. - QueryRunner mockRunner = - _queryRunnerMap.get(Integer.parseInt(queryRequest.getMetadataOrThrow(KEY_OF_SERVER_INSTANCE_PORT))); - String requestIdStr = queryRequest.getMetadataOrThrow(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID); + // ensure mock query runner received correctly deserialized payload. + QueryRunner mockRunner = _queryRunnerMap.get(Integer.parseInt(requestMetadata.get(KEY_OF_SERVER_INSTANCE_PORT))); + String requestId = requestMetadata.get(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID); - // since submitRequest is async, we need to wait for the mockRunner to receive the query payload. - int finalStageId = stageId; - TestUtils.waitForCondition(aVoid -> { - try { - Mockito.verify(mockRunner).processQuery(Mockito.argThat(distributedStagePlan -> { - PlanNode planNode = - dispatchableSubPlan.getQueryStageList().get(finalStageId).getPlanFragment().getFragmentRoot(); - return isStageNodesEqual(planNode, distributedStagePlan.getStageRoot()) && isStageMetadataEqual( - stageMetadata, distributedStagePlan.getStageMetadata()); - }), Mockito.argThat(requestMetadataMap -> requestIdStr.equals( - requestMetadataMap.get(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID)))); - return true; - } catch (Throwable t) { - return false; - } - }, 10000L, "Error verifying mock QueryRunner intercepted query payload!"); + // since submitRequest is async, we need to wait for the mockRunner to receive the query payload. + TestUtils.waitForCondition(aVoid -> { + try { + verify(mockRunner, times(workerMetadataList.size())).processQuery(any(), argThat(stagePlan -> { + PlanNode planNode = dispatchableStagePlan.getPlanFragment().getFragmentRoot(); + return isStageNodesEqual(planNode, stagePlan.getRootNode()) && isStageMetadataEqual(stageMetadata, + stagePlan.getStageMetadata()); + }), argThat(requestMetadataMap -> requestId.equals( + requestMetadataMap.get(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID)))); + return true; + } catch (Throwable t) { + return false; + } + }, 10000L, "Error verifying mock QueryRunner intercepted query payload!"); - // reset the mock runner. - Mockito.reset(mockRunner); - } + // reset the mock runner. + reset(mockRunner); } } private boolean isStageMetadataEqual(StageMetadata expected, StageMetadata actual) { - if (!EqualityUtils.isEqual(StageMetadata.getTableName(expected), StageMetadata.getTableName(actual))) { - return false; - } - TimeBoundaryInfo expectedTimeBoundaryInfo = StageMetadata.getTimeBoundary(expected); - TimeBoundaryInfo actualTimeBoundaryInfo = StageMetadata.getTimeBoundary(actual); - if (expectedTimeBoundaryInfo == null && actualTimeBoundaryInfo != null - || expectedTimeBoundaryInfo != null && actualTimeBoundaryInfo == null) { + if (!Objects.equals(expected.getTableName(), actual.getTableName())) { return false; } - if (expectedTimeBoundaryInfo != null && actualTimeBoundaryInfo != null && ( - !EqualityUtils.isEqual(expectedTimeBoundaryInfo.getTimeColumn(), actualTimeBoundaryInfo.getTimeColumn()) - || !EqualityUtils.isEqual(expectedTimeBoundaryInfo.getTimeValue(), - actualTimeBoundaryInfo.getTimeValue()))) { - return false; + TimeBoundaryInfo expectedTimeBoundaryInfo = expected.getTimeBoundary(); + TimeBoundaryInfo actualTimeBoundaryInfo = actual.getTimeBoundary(); + if (expectedTimeBoundaryInfo != null || actualTimeBoundaryInfo != null) { + if (expectedTimeBoundaryInfo == null || actualTimeBoundaryInfo == null) { + return false; + } + if (!expectedTimeBoundaryInfo.getTimeColumn().equals(actualTimeBoundaryInfo.getTimeColumn()) + || !expectedTimeBoundaryInfo.getTimeValue().equals(actualTimeBoundaryInfo.getTimeValue())) { + return false; + } } List expectedWorkerMetadataList = expected.getWorkerMetadataList(); List actualWorkerMetadataList = actual.getWorkerMetadataList(); @@ -190,13 +190,8 @@ private boolean isStageMetadataEqual(StageMetadata expected, StageMetadata actua } private static boolean isWorkerMetadataEqual(WorkerMetadata expected, WorkerMetadata actual) { - if (!expected.getVirtualServerAddress().hostname().equals(actual.getVirtualServerAddress().hostname()) - || expected.getVirtualServerAddress().port() != actual.getVirtualServerAddress().port() - || expected.getVirtualServerAddress().workerId() != actual.getVirtualServerAddress().workerId()) { - return false; - } - return EqualityUtils.isEqual(WorkerMetadata.getTableSegmentsMap(expected), - WorkerMetadata.getTableSegmentsMap(actual)); + return expected.getVirtualAddress().equals(actual.getVirtualAddress()) && EqualityUtils.isEqual( + expected.getTableSegmentsMap(), actual.getTableSegmentsMap()); } private static boolean isStageNodesEqual(PlanNode left, PlanNode right) { @@ -216,11 +211,10 @@ private static boolean isStageNodesEqual(PlanNode left, PlanNode right) { return true; } - private Worker.QueryResponse submitRequest(Worker.QueryRequest queryRequest) { - String host = queryRequest.getMetadataMap().get(KEY_OF_SERVER_INSTANCE_HOST); - int port = Integer.parseInt(queryRequest.getMetadataMap().get(KEY_OF_SERVER_INSTANCE_PORT)); - long timeoutMs = - Long.parseLong(queryRequest.getMetadataMap().get(CommonConstants.Broker.Request.QueryOptionKey.TIMEOUT_MS)); + private Worker.QueryResponse submitRequest(Worker.QueryRequest queryRequest, Map requestMetadata) { + String host = requestMetadata.get(KEY_OF_SERVER_INSTANCE_HOST); + int port = Integer.parseInt(requestMetadata.get(KEY_OF_SERVER_INSTANCE_PORT)); + long timeoutMs = Long.parseLong(requestMetadata.get(CommonConstants.Broker.Request.QueryOptionKey.TIMEOUT_MS)); ManagedChannel channel = ManagedChannelBuilder.forAddress(host, port).usePlaintext().build(); PinotQueryWorkerGrpc.PinotQueryWorkerBlockingStub stub = PinotQueryWorkerGrpc.newBlockingStub(channel); Worker.QueryResponse resp = @@ -229,32 +223,35 @@ private Worker.QueryResponse submitRequest(Worker.QueryRequest queryRequest) { return resp; } - private Worker.QueryRequest getQueryRequest(DispatchableSubPlan dispatchableSubPlan, int stageId) { - DispatchablePlanFragment planFragment = dispatchableSubPlan.getQueryStageList().get(stageId); - Map> serverInstanceToWorkerIdMap = planFragment.getServerInstanceToWorkerIdMap(); + private Worker.QueryRequest getQueryRequest(DispatchableSubPlan queryPlan, int stageId) { + DispatchablePlanFragment stagePlan = queryPlan.getQueryStageList().get(stageId); + Plan.StageNode rootNode = + StageNodeSerDeUtils.serializeStageNode((AbstractPlanNode) stagePlan.getPlanFragment().getFragmentRoot()); + List workerMetadataList = + QueryPlanSerDeUtils.toProtoWorkerMetadataList(stagePlan.getWorkerMetadataList()); + ByteString customProperty = QueryPlanSerDeUtils.toProtoProperties(stagePlan.getCustomProperties()); + // this particular test set requires the request to have a single QueryServerInstance to dispatch to // as it is not testing the multi-tenancy dispatch (which is in the QueryDispatcherTest) - Map.Entry> entry = serverInstanceToWorkerIdMap.entrySet().iterator().next(); - QueryServerInstance serverInstance = entry.getKey(); - List workerIds = entry.getValue(); - Plan.StageNode stageRoot = - StageNodeSerDeUtils.serializeStageNode((AbstractPlanNode) planFragment.getPlanFragment().getFragmentRoot()); - List protoWorkerMetadataList = QueryPlanSerDeUtils.toProtoWorkerMetadataList(planFragment); + QueryServerInstance serverInstance = stagePlan.getServerInstanceToWorkerIdMap().keySet().iterator().next(); Worker.StageMetadata stageMetadata = - QueryPlanSerDeUtils.toProtoStageMetadata(protoWorkerMetadataList, planFragment.getCustomProperties(), - serverInstance, workerIds); - Worker.StagePlan stagePlan = - Worker.StagePlan.newBuilder().setStageId(stageId).setStageRoot(stageRoot).setStageMetadata(stageMetadata) + Worker.StageMetadata.newBuilder().addAllWorkerMetadata(workerMetadataList).setCustomProperty(customProperty) .build(); + Worker.StagePlan protoStagePlan = + Worker.StagePlan.newBuilder().setStageId(stageId).setRootNode(rootNode.toByteString()) + .setStageMetadata(stageMetadata).build(); + + Map requestMetadata = new HashMap<>(); + // the default configurations that must exist. + requestMetadata.put(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID, + String.valueOf(RANDOM_REQUEST_ID_GEN.nextLong())); + requestMetadata.put(CommonConstants.Broker.Request.QueryOptionKey.TIMEOUT_MS, + String.valueOf(CommonConstants.Broker.DEFAULT_BROKER_TIMEOUT_MS)); + // extra configurations we want to test also parsed out correctly. + requestMetadata.put(KEY_OF_SERVER_INSTANCE_HOST, serverInstance.getHostname()); + requestMetadata.put(KEY_OF_SERVER_INSTANCE_PORT, Integer.toString(serverInstance.getQueryServicePort())); - return Worker.QueryRequest.newBuilder().addStagePlan(stagePlan) - // the default configurations that must exist. - .putMetadata(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID, - String.valueOf(RANDOM_REQUEST_ID_GEN.nextLong())) - .putMetadata(CommonConstants.Broker.Request.QueryOptionKey.TIMEOUT_MS, - String.valueOf(CommonConstants.Broker.DEFAULT_BROKER_TIMEOUT_MS)) - // extra configurations we want to test also parsed out correctly. - .putMetadata(KEY_OF_SERVER_INSTANCE_HOST, serverInstance.getHostname()) - .putMetadata(KEY_OF_SERVER_INSTANCE_PORT, String.valueOf(serverInstance.getQueryServicePort())).build(); + return Worker.QueryRequest.newBuilder().addStagePlan(protoStagePlan) + .setMetadata(QueryPlanSerDeUtils.toProtoProperties(requestMetadata)).build(); } }