walterddr commented on code in PR #10779:
URL: https://github.com/apache/pinot/pull/10779#discussion_r1198014832


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakOperator.java:
##########
@@ -0,0 +1,118 @@
+/**
+ * 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.pipeline;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.apache.pinot.query.runtime.operator.MultiStageOperator;
+import org.apache.pinot.query.runtime.plan.OpChainExecutionContext;
+
+
+public class PipelineBreakOperator extends MultiStageOperator {
+  private static final String EXPLAIN_NAME = "PIPELINE_BREAKER";
+  private final Deque<Map.Entry<Integer, Operator<TransferableBlock>>> 
_workerEntries;
+  private final Map<Integer, List<TransferableBlock>> _resultMap;
+  private final CountDownLatch _workerDoneLatch;
+  private TransferableBlock _errorBlock;
+
+
+  public PipelineBreakOperator(OpChainExecutionContext context,
+      Map<Integer, Operator<TransferableBlock>> pipelineWorkerMap) {
+    super(context);
+    _resultMap = new HashMap<>();
+    _workerEntries = new ArrayDeque<>();
+    _workerEntries.addAll(pipelineWorkerMap.entrySet());
+    _workerDoneLatch = new CountDownLatch(1);
+  }
+
+  public Map<Integer, List<TransferableBlock>> getResult() {
+    try {
+      boolean isWorkerDone =
+          _workerDoneLatch.await(_context.getDeadlineMs() - 
System.currentTimeMillis(), TimeUnit.MILLISECONDS);
+      if (isWorkerDone && _errorBlock == null) {
+        return _resultMap;
+      }
+    } catch (Exception e) {
+      _errorBlock = TransferableBlockUtils.getErrorTransferableBlock(e);
+    }
+    return Collections.singletonMap(-1, 
Collections.singletonList(_errorBlock));

Review Comment:
   TODO: consider if we should return error here, or throw --> which will be a 
question for the main opChain
   1. if the main OpChain is using this pipeline breaker result as input to 
planVisitor, then we should throw, b/c main OpChain planvisitor will failed, 
and mailbox send operator will not be created, thus the error should be 
returned to broker directly
   2. if the main OpChain is using it as data block for input (e.g. fake the 
mailbox received that has already been received as a sequential access 
operator), then we should retain the error block b/c the error block also 
contain metadata info useful for debugging



-- 
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