walterddr commented on a change in pull request #8412:
URL: https://github.com/apache/pinot/pull/8412#discussion_r838632119



##########
File path: pinot-common/src/main/proto/mailbox.proto
##########
@@ -0,0 +1,55 @@
+//
+// 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.
+//
+
+/**
+ * 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;
+
+service PinotMailbox {
+  rpc open(stream MailboxContent) returns (stream MailboxStatus);
+}
+
+message MailboxStatus {
+  string mailboxId = 1;

Review comment:
       it is for debuggability purpose. we could use a hash function to compute 
from the mailboxID to a hash if performance is of concern.

##########
File path: pinot-common/src/main/proto/worker.proto
##########
@@ -0,0 +1,75 @@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+
+/**
+ * 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;
+
+service PinotQueryWorker {
+  rpc Submit(QueryRequest) returns (QueryResponse);
+}
+
+message QueryRequest {
+  map<string, string> metadata = 1;
+  QueryPlan queryPlan = 2;
+}
+
+message QueryResponse {
+  map<string, string> metadata = 1;
+  bytes payload = 2;
+}
+
+message QueryPlan {
+  string stageId = 1;
+  string instanceId = 2;

Review comment:
       yes StageId should be INT. I will make that change along with this PR.
   instanceId cannot be as it is a stringify format of the `ServerInstance`

##########
File path: pinot-common/src/main/proto/worker.proto
##########
@@ -0,0 +1,75 @@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+
+/**
+ * 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;
+
+service PinotQueryWorker {
+  rpc Submit(QueryRequest) returns (QueryResponse);
+}
+
+message QueryRequest {
+  map<string, string> metadata = 1;
+  QueryPlan queryPlan = 2;
+}
+
+message QueryResponse {
+  map<string, string> metadata = 1;
+  bytes payload = 2;
+}
+
+message QueryPlan {
+  string stageId = 1;
+  string instanceId = 2;
+  StagePlan stagePlan = 3;
+  map<string, StageMetadata> stageMetadata = 4;
+}
+
+message StagePlan {
+  bytes serializedStagePlan = 1;
+}
+
+message StageMetadata {
+  repeated string instances = 1;
+  repeated string scannedTables = 2;
+  map<string, SegmentMetadata> segmentMetadata = 3;
+}

Review comment:
       this is the same as the current server instance to segment list map. 
   however i can't add a `map<string, repeated string>` in proto so I need to 
wrap the list of segment names into `SegmentMetadata`.
   
   maybe I should rename it to `SegmentNameList`?

##########
File path: pinot-common/src/main/proto/worker.proto
##########
@@ -0,0 +1,75 @@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+
+/**
+ * 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;
+
+service PinotQueryWorker {
+  rpc Submit(QueryRequest) returns (QueryResponse);
+}
+
+message QueryRequest {
+  map<string, string> metadata = 1;
+  QueryPlan queryPlan = 2;
+}
+
+message QueryResponse {
+  map<string, string> metadata = 1;
+  bytes payload = 2;
+}
+
+message QueryPlan {
+  string stageId = 1;

Review comment:
       here QueryPlan is actually StagePlan. renaming it to QueryStagePlan. 

##########
File path: pinot-common/src/main/proto/worker.proto
##########
@@ -0,0 +1,75 @@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+
+/**
+ * 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;
+
+service PinotQueryWorker {
+  rpc Submit(QueryRequest) returns (QueryResponse);
+}
+
+message QueryRequest {
+  map<string, string> metadata = 1;

Review comment:
       not really optional since some of the physical info has to go over 
metadata (like broker address / metrics prefix) but yeah I can make it optional 
since technically you can run without the metadata.

##########
File path: 
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/DataTableBlock.java
##########
@@ -0,0 +1,73 @@
+/**
+ * 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.blocks;
+
+import java.io.IOException;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.Block;
+import org.apache.pinot.core.common.BlockDocIdSet;
+import org.apache.pinot.core.common.BlockDocIdValueSet;
+import org.apache.pinot.core.common.BlockMetadata;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.operator.blocks.InstanceResponseBlock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@code DataTableBlock} is a row-based data block backed by a {@link 
DataTable}.
+ */
+public class DataTableBlock implements Block {

Review comment:
       good idea. let's put a TODO (or design discussion section on this)

##########
File path: pinot-common/src/main/proto/worker.proto
##########
@@ -0,0 +1,75 @@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+
+/**
+ * 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;
+
+service PinotQueryWorker {
+  rpc Submit(QueryRequest) returns (QueryResponse);
+}
+

Review comment:
       so I purposefully not use Broker/Server concept but Worker/Mailbox 
because.
   1. technically speaking a Worker can be Broker (Worker without local 
segments) or Server (Worker with local segments). 
   2. It makes more sense to differentiate whether a proto defines controll 
flow (Worker) or data flow (Mailbox)
   
   I will add more comment to the proto files. 

##########
File path: pinot-common/src/main/proto/worker.proto
##########
@@ -0,0 +1,75 @@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+
+/**
+ * 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;
+
+service PinotQueryWorker {
+  rpc Submit(QueryRequest) returns (QueryResponse);
+}
+
+message QueryRequest {
+  map<string, string> metadata = 1;
+  QueryPlan queryPlan = 2;
+}
+
+message QueryResponse {
+  map<string, string> metadata = 1;
+  bytes payload = 2;
+}
+
+message QueryPlan {
+  string stageId = 1;
+  string instanceId = 2;
+  StagePlan stagePlan = 3;
+  map<string, StageMetadata> stageMetadata = 4;
+}
+
+message StagePlan {
+  bytes serializedStagePlan = 1;
+}
+
+message StageMetadata {
+  repeated string instances = 1;

Review comment:
       not sure what this mean. could you clarify a bit more regarding this 
comment? 
   
   As in my design, once the job has been dispatched. all communication are 
between executors (of different machines)

##########
File path: 
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/DistributedQueryPlan.java
##########
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.plan;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.pinot.core.transport.ServerInstance;
+import org.apache.pinot.query.planner.StageMetadata;
+import org.apache.pinot.query.planner.nodes.StageNode;
+
+
+/**
+ * WorkerQueryRequest is the extended version of the {@link 
org.apache.pinot.core.query.request.ServerQueryRequest}.
+ */
+public class DistributedQueryPlan implements Serializable {
+  private final String _stageId;
+  private ServerInstance _serverInstance;
+  private StageNode _stageRoot;
+  private Map<String, StageMetadata> _metadataMap;

Review comment:
       > I think we need to revisit this abstraction. It looks very confusing 
here
   > 
   > Multi-stage `QueryPlan` built at broker comprises of multiple stages. 
Since each `StageNode` will be executed on some executor / server, that 
information must already be available at the time of building the QueryPlan.
   
   you are absolutely right. 
   so QueryPlan already contains the entire "physical" plan of the multi-stage 
execution. 
   
   
   > 
   > So, then why should the information about `serverInstance` be tracked 
separately ? In fact, we are tracking it in both `StageMetadata` and then again 
outside here in the deserialized plan.
   
   DistributedQueryPlan is actually "ONE STAGE OF" the QueryPlan (or I think we 
should rename it DistributableStagePlan). and yes again you are right this 
should be in some sort of serializable format. 
   
   > 
   > The `StageNode` should be self-contained ideally
   > 
   > * root or leaf stage
   > * stageId
   > * operator tree
   > * executor assignment
   
   all the 4 components you describe above is assigned to a stage, not a 
stageNode. thus the split between StageNode _root; and StageMetadata. 
   
   
   

##########
File path: 
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BroadcastJoinOperator.java
##########
@@ -0,0 +1,162 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.operator;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.BaseOperator;
+import org.apache.pinot.core.query.selection.SelectionOperatorUtils;
+import org.apache.pinot.query.planner.partitioning.KeySelector;
+import org.apache.pinot.query.runtime.blocks.DataTableBlock;
+import org.apache.pinot.query.runtime.blocks.DataTableBlockUtils;
+
+
+/**
+ * This basic {@code BroadcastJoinOperator} implement a basic broadcast join 
algorithm.
+ *
+ * <p>It takes the right table as the broadcast side and materialize a hash 
table. Then for each of the left table row,
+ * it looks up for the corresponding row(s) from the hash table and create a 
joint row.
+ *
+ * <p>For each of the data block received from the left table, it will 
generate a joint data block.
+ */
+public class BroadcastJoinOperator extends BaseOperator<DataTableBlock> {
+
+  private final HashMap<Object, List<Object[]>> _broadcastHashTable;
+  private final BaseOperator<DataTableBlock> _leftTableOperator;
+  private final BaseOperator<DataTableBlock> _rightTableOperator;
+
+  private DataSchema _leftTableSchema;
+  private DataSchema _rightTableSchema;
+  private int _resultRowSize;
+  private boolean _isHashTableBuilt;
+  private KeySelector<Object[], Object> _leftKeySelector;
+  private KeySelector<Object[], Object> _rightKeySelector;
+
+  public BroadcastJoinOperator(BaseOperator<DataTableBlock> leftTableOperator,
+      BaseOperator<DataTableBlock> rightTableOperator, KeySelector<Object[], 
Object> leftKeySelector,
+      KeySelector<Object[], Object> rightKeySelector) {
+    // TODO: this assumes right table is broadcast.
+    _leftKeySelector = leftKeySelector;
+    _rightKeySelector = rightKeySelector;
+    _leftTableOperator = leftTableOperator;
+    _rightTableOperator = rightTableOperator;
+    _isHashTableBuilt = false;
+    _broadcastHashTable = new HashMap<>();
+  }
+
+  @Override
+  public String getOperatorName() {
+    return null;

Review comment:
       good catch will update 

##########
File path: 
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/DistributedQueryPlan.java
##########
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.plan;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.pinot.core.transport.ServerInstance;
+import org.apache.pinot.query.planner.StageMetadata;
+import org.apache.pinot.query.planner.nodes.StageNode;
+
+
+/**
+ * WorkerQueryRequest is the extended version of the {@link 
org.apache.pinot.core.query.request.ServerQueryRequest}.

Review comment:
       actually importing the class ServerQueryRequest but only use it in 
javadoc causes checkstyle to complain. let me see what can be changed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org

Reply via email to