You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "gortiz (via GitHub)" <gi...@apache.org> on 2023/07/28 11:45:32 UTC

[GitHub] [pinot] gortiz opened a new pull request, #11205: [Draft] Remove noop

gortiz opened a new pull request, #11205:
URL: https://github.com/apache/pinot/pull/11205

   This PR includes a deep change in V2 scheduler in order to do not use NOOP blocks. The NOOP implementation used to create a coroutine system in V2 has been proven to be very error prone and even if it is possible to implent the same idea in a safer way, we think it is better to use a more imperative and easy to understand scheduler.
   
   The main disadvantage is that we need to have one thread per active OpChain, but don't expect to have thousand of running OpChains, so it shouldn't be a problem. In short term, we may be able to use Loom, which would make the code even easier.
   
   Even if Git thinks I'm the author of these changes, most of the ideas and even lines were originally created by @walterddr in its own fork. Given that it was very difficult to rebase these changes, I've manually applied these changes to a more recent master branch.


-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289872541


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultiStageOperator.java:
##########
@@ -44,6 +45,7 @@ public MultiStageOperator(OpChainExecutionContext context) {
     _opChainStats = _context.getStats();
   }
 
+  @Nonnull

Review Comment:
   I don't know if I told you, but I was used to work in kotlin, when nullability is checked at compile time. That is why I prefer to have these warnings and deal with them at compile time.
   
   But I understand that is not something we would like to add here. I've just add the annotations trying to get some help from the IDE to solve some NPE I've found while doing the PR.
   
   Removing them



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1290051875


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/SchedulerService.java:
##########
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.executor;
+
+import org.apache.pinot.query.runtime.operator.OpChain;
+
+
+public interface SchedulerService {

Review Comment:
   Removed in ace781b6a7a1f500bd51fc7bcf7879d39b4b6439



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1280755616


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -79,7 +94,11 @@ public boolean offer(TransferableBlock block, long timeoutMs) {
     try {
       if (_blocks.offer(block, timeoutMs, TimeUnit.MILLISECONDS)) {
         if (_errorBlock.get() == null) {
+          if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("==[MAILBOX]== Block " + block + " ready to read from mailbox: " + _id);
+          }
           _receiveMailCallback.accept(MailboxIdUtils.toOpChainId(_id));
+          notifyReader();

Review Comment:
   tagging @Jackie-Jiang to take a look at the mailbox mechanism



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281397920


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);

Review Comment:
   We need to remove the entry when closing/cancelling the `OpChain`



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. for leaf stage this cannot be a dangling opchain b/c they will eventually be cleared up
+    // via query timeout.
+    List<OpChainId> opChainIdsToCancel = _submittedOpChainMap.keySet()

Review Comment:
   Use `entrySet().iterator()` to iterate should be cheaper and create less garbage. We also need to remove the entry because there is no guarantee that the future already starts executing



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {

Review Comment:
   (format) The indentation in this file is incorrect



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {

Review Comment:
   We probably want a different API to directly cancel an `OpChain` (or `OpChainId`) because that is much cheaper than cancelling a request id



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);

Review Comment:
   ```suggestion
                   thrown = new RuntimeException("Error block: " + returnedErrorBlock.getDataBlock().getExceptions());
   ```



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java:
##########
@@ -115,12 +115,12 @@ public MultiStageBrokerRequestHandler(PinotConfiguration config, String brokerId
 
     long releaseMs = config.getProperty(QueryConfig.KEY_OF_SCHEDULER_RELEASE_TIMEOUT_MS,
         QueryConfig.DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS);
-    _reducerScheduler = new OpChainSchedulerService(new RoundRobinScheduler(releaseMs),
-        Executors.newCachedThreadPool(new NamedThreadFactory("query_broker_reducer_" + _reducerPort + "_port")));
-    _mailboxService = new MailboxService(_reducerHostname, _reducerPort, config, _reducerScheduler::onDataAvailable);
+    //TODO: make this configurable
+    _opChainExecutor = new OpChainExecutor(new NamedThreadFactory("op_chain_worker_on_" + _reducerPort + "_port"));

Review Comment:
   It is not closed in `shutDown()`



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/SchedulerService.java:
##########
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.executor;
+
+import org.apache.pinot.query.runtime.operator.OpChain;
+
+
+public interface SchedulerService {

Review Comment:
   +1



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/OpChainExecutor.java:
##########
@@ -0,0 +1,129 @@
+/**
+ * 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;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class OpChainExecutor implements ExecutorService, AutoCloseable {

Review Comment:
   I don't really see much value for this wrapper class. Suggest removing it and directly creating an `ExecutorService`



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. for leaf stage this cannot be a dangling opchain b/c they will eventually be cleared up
+    // via query timeout.
+    List<OpChainId> opChainIdsToCancel = _submittedOpChainMap.keySet()
+        .stream().filter(opChainId -> opChainId.getRequestId() == requestId).collect(Collectors.toList());
+    for (OpChainId opChainId : opChainIdsToCancel) {
+      Future<?> future = _submittedOpChainMap.get(opChainId);
+      if (future != null) {
+        future.cancel(true);
+      }
+    }
   }
 
   private void closeOpChain(OpChain opChain) {
-    try {
       opChain.close();
-    } finally {
-      _scheduler.deregister(opChain);
-    }
   }
 
-  private void cancelOpChain(OpChain opChain, Throwable e) {
-    try {
-      opChain.cancel(e);
-    } finally {
-      _scheduler.deregister(opChain);
-    }
+  private void cancelOpChain(OpChain opChain, @Nullable Throwable t) {

Review Comment:
   With the null handling above, `t` will no longer be `null`, and we can remove the annotation and special null handling in all the subsequent classes



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1283318547


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. for leaf stage this cannot be a dangling opchain b/c they will eventually be cleared up
+    // via query timeout.
+    List<OpChainId> opChainIdsToCancel = _submittedOpChainMap.keySet()

Review Comment:
   Jackie suggested to remove the entry once the task finish and I modified the code to do so



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1284940784


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -51,16 +51,30 @@ public class ReceivingMailbox {
   // TODO: Revisit if this is the correct way to apply back pressure
   private final BlockingQueue<TransferableBlock> _blocks = new ArrayBlockingQueue<>(DEFAULT_MAX_PENDING_BLOCKS);
   private final AtomicReference<TransferableBlock> _errorBlock = new AtomicReference<>();
+  @Nullable
+  private volatile Reader _reader;
 
   public ReceivingMailbox(String id, Consumer<OpChainId> receiveMailCallback) {
     _id = id;
     _receiveMailCallback = receiveMailCallback;
   }
 
+  public void registeredReader(Reader reader) {

Review Comment:
   In `InMemorySendingMailbox`, we directly get the receiving mailbox and put blocks into it. At that moment the reader is not registered but there are contents within the mailbox. Our use pattern is not standard, so not sure if we should put a general framework.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1283308899


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {

Review Comment:
   but we should still use a timeout cache or something as a final fallback. b/c the grpc cancel signal can also get lost in the wire (we don't retry cancel or monitor cancel grpc call status)



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281390603


##########
pinot-core/src/main/java/org/apache/pinot/core/common/Operator.java:
##########
@@ -39,6 +39,7 @@ public interface Operator<T extends Block> {
    * @throws EarlyTerminationException if the operator is early-terminated (interrupted) before processing the next
    *         block of data. Operator can early terminated when the query times out, or is already satisfied.
    */
+  @Nullable

Review Comment:
   Should be okay. Both V1 and V2 can return null block



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281474548


##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java:
##########
@@ -115,12 +115,12 @@ public MultiStageBrokerRequestHandler(PinotConfiguration config, String brokerId
 
     long releaseMs = config.getProperty(QueryConfig.KEY_OF_SCHEDULER_RELEASE_TIMEOUT_MS,
         QueryConfig.DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS);
-    _reducerScheduler = new OpChainSchedulerService(new RoundRobinScheduler(releaseMs),
-        Executors.newCachedThreadPool(new NamedThreadFactory("query_broker_reducer_" + _reducerPort + "_port")));
-    _mailboxService = new MailboxService(_reducerHostname, _reducerPort, config, _reducerScheduler::onDataAvailable);
+    //TODO: make this configurable
+    _opChainExecutor = new OpChainExecutor(new NamedThreadFactory("op_chain_worker_on_" + _reducerPort + "_port"));

Review Comment:
   Fixed on  https://github.com/apache/pinot/commit/306edff063e2c6ff63921feefefef728c23b5727



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289872541


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultiStageOperator.java:
##########
@@ -44,6 +45,7 @@ public MultiStageOperator(OpChainExecutionContext context) {
     _opChainStats = _context.getStats();
   }
 
+  @Nonnull

Review Comment:
   I don't know if I told you, but I was used to work in kotlin, when nullability is checked at compile time. That is why I prefer to have these warnings and deal with them at compile time.
   
   But I understand that is not something we would like to add here. I've just add the annotations trying to get some help from the IDE to solve some NPE I've found while doing the PR.
   
   Removing them in a26c79a7edb3a0d67da0979de548d695f810e1f0



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1295575927


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java:
##########
@@ -0,0 +1,246 @@
+/**
+ * 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.utils;
+
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BlockingMultiStreamConsumer<E> implements AutoCloseable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlockingMultiStreamConsumer.class);
+  private final Object _id;
+  protected final List<? extends AsyncStream<E>> _mailboxes;
+  protected final ArrayBlockingQueue<Boolean> _newDataReady = new ArrayBlockingQueue<>(1);
+  private final long _deadlineMs;
+  /**
+   * An index that used to calculate where do we are going to start reading.
+   * The invariant is that we are always going to start reading from {@code _lastRead + 1}.
+   * Therefore {@link #_lastRead} must be in the range {@code [-1, mailbox.size() - 1]}
+   */
+  protected int _lastRead;
+  private E _errorBlock = null;
+
+  public BlockingMultiStreamConsumer(Object id, long deadlineMs, List<? extends AsyncStream<E>> asyncProducers) {
+    _id = id;
+    _deadlineMs = deadlineMs;
+    AsyncStream.OnNewData onNewData = this::onData;
+    _mailboxes = asyncProducers;
+    _mailboxes.forEach(blockProducer -> blockProducer.addOnNewDataListener(onNewData));
+    _lastRead = _mailboxes.size() - 1;
+  }
+
+  protected abstract boolean isError(E element);
+
+  protected abstract boolean isEos(E element);
+
+  protected abstract E onTimeout();
+
+  protected abstract E onException(Exception e);
+
+  protected abstract E onEos();
+
+  @Override
+  public void close() {
+    cancelRemainingMailboxes();
+  }
+
+  public void cancel(Throwable t) {
+    cancelRemainingMailboxes();
+  }
+
+  protected void cancelRemainingMailboxes() {
+    for (AsyncStream<E> mailbox : _mailboxes) {
+      mailbox.cancel();
+    }
+  }
+
+  public void onData() {
+    if (_newDataReady.offer(Boolean.TRUE)) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("New data notification delivered on " + _id + ". " + System.identityHashCode(_newDataReady));
+      }
+    } else if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("New data notification ignored on " + _id + ". " + System.identityHashCode(_newDataReady));
+    }
+  }
+
+  /**
+   * Reads the next block for any ready mailbox or blocks until some of them is ready.
+   *
+   * The method implements a sequential read semantic. Meaning that:
+   * <ol>
+   *   <li>EOS is only returned when all mailboxes already emitted EOS or there are no mailboxes</li>
+   *   <li>If an error is read from a mailbox, the error is returned</li>
+   *   <li>If data is read from a mailbox, that data block is returned</li>
+   *   <li>If no mailbox is ready, the calling thread is blocked</li>
+   * </ol>
+   *
+   * Right now the implementation tries to be fair. If one call returned the block from mailbox {@code i}, then next
+   * call will look for mailbox {@code i+1}, {@code i+2}... in a circular manner.
+   *
+   * In order to unblock a thread blocked here, {@link #onData()} should be called.   *
+   */
+  public E readBlockBlocking() {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
+    }
+    // Standard optimistic execution. First we try to read without acquiring the lock.
+    E block = readDroppingSuccessEos();
+    long timeoutMs = _deadlineMs - System.currentTimeMillis();
+    boolean timeout = false;
+    try {
+      while (block == null) { // we didn't find a mailbox ready to read, so we need to be pessimistic

Review Comment:
   I don't think there is an issue on reading (not pulling, it is already in the mailboxes) another block, but if you find your version cleaner, we can use that.
   
   Applied on 05dbd7975ca27c0d99c4fdbe7ff4551c154d7295



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277737296


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java:
##########
@@ -101,9 +99,8 @@ public void init(PinotConfiguration config, InstanceDataManager instanceDataMana
           new NamedThreadFactory("query_intermediate_worker_on_" + _port + "_port"));
       _queryWorkerLeafExecutorService = Executors.newFixedThreadPool(ResourceManager.DEFAULT_QUERY_WORKER_THREADS,
           new NamedThreadFactory("query_leaf_worker_on_" + _port + "_port"));
-      _scheduler = new OpChainSchedulerService(new RoundRobinScheduler(releaseMs),
-          getQueryWorkerIntermExecutorService());
-      _mailboxService = new MailboxService(_hostname, _port, config, _scheduler::onDataAvailable);
+      _scheduler = new OpChainSchedulerService(getQueryWorkerIntermExecutorService());

Review Comment:
   @walterddr's code used two executors. I think once is enough, but just to be sure, please take a look



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277731284


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -51,16 +51,30 @@ public class ReceivingMailbox {
   // TODO: Revisit if this is the correct way to apply back pressure
   private final BlockingQueue<TransferableBlock> _blocks = new ArrayBlockingQueue<>(DEFAULT_MAX_PENDING_BLOCKS);
   private final AtomicReference<TransferableBlock> _errorBlock = new AtomicReference<>();
+  @Nullable
+  private volatile Reader _reader;
 
   public ReceivingMailbox(String id, Consumer<OpChainId> receiveMailCallback) {
     _id = id;
     _receiveMailCallback = receiveMailCallback;
   }
 
+  public void registeredReader(Reader reader) {

Review Comment:
   This is my main contribution over @walterddr's code. Instead of storing a map in the scheduler that maps op chains to their blocking objects (in his case a blocking list of 1 element) here I'm registering the ReceivingOperator on the mailbox itself. By doing that we decouple the receiving operation from the scheduler.
   
   
   
   The ReceivingMailbox will notify the reader (if any) whenever a message is received. The older `_receiveMailCallback` is maintained because some tests use it, but we may decide to change it in a way that it doesn't need to allocate OpChainIds for each message sent.
   
   You can see BaseMailboxReceiveOperator.onData to understand how this callback is implemented.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281459982


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/OpChainExecutor.java:
##########
@@ -0,0 +1,129 @@
+/**
+ * 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;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class OpChainExecutor implements ExecutorService, AutoCloseable {

Review Comment:
   It is mostly to be able to close it in one line instead of 6. But we can create an ExecutorServiceUtils that does that.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281497007


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {

Review Comment:
   Should be better in d34f90152b5cca9f83ff71225109dc852e502f50



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281474783


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/OpChainExecutor.java:
##########
@@ -0,0 +1,129 @@
+/**
+ * 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;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class OpChainExecutor implements ExecutorService, AutoCloseable {

Review Comment:
   Removed on  306edff063e2c6ff63921feefefef728c23b5727



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1295575927


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java:
##########
@@ -0,0 +1,246 @@
+/**
+ * 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.utils;
+
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BlockingMultiStreamConsumer<E> implements AutoCloseable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlockingMultiStreamConsumer.class);
+  private final Object _id;
+  protected final List<? extends AsyncStream<E>> _mailboxes;
+  protected final ArrayBlockingQueue<Boolean> _newDataReady = new ArrayBlockingQueue<>(1);
+  private final long _deadlineMs;
+  /**
+   * An index that used to calculate where do we are going to start reading.
+   * The invariant is that we are always going to start reading from {@code _lastRead + 1}.
+   * Therefore {@link #_lastRead} must be in the range {@code [-1, mailbox.size() - 1]}
+   */
+  protected int _lastRead;
+  private E _errorBlock = null;
+
+  public BlockingMultiStreamConsumer(Object id, long deadlineMs, List<? extends AsyncStream<E>> asyncProducers) {
+    _id = id;
+    _deadlineMs = deadlineMs;
+    AsyncStream.OnNewData onNewData = this::onData;
+    _mailboxes = asyncProducers;
+    _mailboxes.forEach(blockProducer -> blockProducer.addOnNewDataListener(onNewData));
+    _lastRead = _mailboxes.size() - 1;
+  }
+
+  protected abstract boolean isError(E element);
+
+  protected abstract boolean isEos(E element);
+
+  protected abstract E onTimeout();
+
+  protected abstract E onException(Exception e);
+
+  protected abstract E onEos();
+
+  @Override
+  public void close() {
+    cancelRemainingMailboxes();
+  }
+
+  public void cancel(Throwable t) {
+    cancelRemainingMailboxes();
+  }
+
+  protected void cancelRemainingMailboxes() {
+    for (AsyncStream<E> mailbox : _mailboxes) {
+      mailbox.cancel();
+    }
+  }
+
+  public void onData() {
+    if (_newDataReady.offer(Boolean.TRUE)) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("New data notification delivered on " + _id + ". " + System.identityHashCode(_newDataReady));
+      }
+    } else if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("New data notification ignored on " + _id + ". " + System.identityHashCode(_newDataReady));
+    }
+  }
+
+  /**
+   * Reads the next block for any ready mailbox or blocks until some of them is ready.
+   *
+   * The method implements a sequential read semantic. Meaning that:
+   * <ol>
+   *   <li>EOS is only returned when all mailboxes already emitted EOS or there are no mailboxes</li>
+   *   <li>If an error is read from a mailbox, the error is returned</li>
+   *   <li>If data is read from a mailbox, that data block is returned</li>
+   *   <li>If no mailbox is ready, the calling thread is blocked</li>
+   * </ol>
+   *
+   * Right now the implementation tries to be fair. If one call returned the block from mailbox {@code i}, then next
+   * call will look for mailbox {@code i+1}, {@code i+2}... in a circular manner.
+   *
+   * In order to unblock a thread blocked here, {@link #onData()} should be called.   *
+   */
+  public E readBlockBlocking() {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
+    }
+    // Standard optimistic execution. First we try to read without acquiring the lock.
+    E block = readDroppingSuccessEos();
+    long timeoutMs = _deadlineMs - System.currentTimeMillis();
+    boolean timeout = false;
+    try {
+      while (block == null) { // we didn't find a mailbox ready to read, so we need to be pessimistic

Review Comment:
   I don't think there is an issue on reading (not pulling, it is already in the mailboxes) another block, but if you find your version clearer, we can use that.
   
   Applied on 05dbd7975ca27c0d99c4fdbe7ff4551c154d7295



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #11205: Remove noop

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1293157495


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java:
##########
@@ -118,46 +118,52 @@ public String toExplainString() {
 
   @Override
   protected TransferableBlock getNextBlock() {
-    boolean canContinue = true;
     TransferableBlock transferableBlock;
+    if (LOGGER.isDebugEnabled()) {
+      LOGGER.debug("==[SEND]== Enter getNextBlock from: " + _context.getId());
+    }
     try {
       transferableBlock = _sourceOperator.nextBlock();
-      if (transferableBlock.isNoOpBlock()) {
-        return transferableBlock;
-      } else if (transferableBlock.isEndOfStreamBlock()) {
-        if (transferableBlock.isSuccessfulEndOfStreamBlock()) {
-          // Stats need to be populated here because the block is being sent to the mailbox
-          // and the receiving opChain will not be able to access the stats from the previous opChain
-          TransferableBlock eosBlockWithStats = TransferableBlockUtils.getEndOfStreamTransferableBlock(
-              OperatorUtils.getMetadataFromOperatorStats(_opChainStats.getOperatorStatsMap()));
-          sendTransferableBlock(eosBlockWithStats);
-        } else {
-          sendTransferableBlock(transferableBlock);
-        }
-      } else { // normal blocks
-        // check whether we should continue depending on exchange queue condition.
-        canContinue = sendTransferableBlock(transferableBlock);
+      if (transferableBlock.isSuccessfulEndOfStreamBlock()) {
+        // Stats need to be populated here because the block is being sent to the mailbox
+        // and the receiving opChain will not be able to access the stats from the previous opChain
+        TransferableBlock eosBlockWithStats = TransferableBlockUtils.getEndOfStreamTransferableBlock(
+            OperatorUtils.getMetadataFromOperatorStats(_opChainStats.getOperatorStatsMap()));
+        sendTransferableBlock(eosBlockWithStats, false);
+      } else {
+        sendTransferableBlock(transferableBlock, true);
       }
     } catch (Exception e) {
       transferableBlock = TransferableBlockUtils.getErrorTransferableBlock(e);
       try {
         LOGGER.error("Exception while transferring data on opChain: " + _context.getId(), e);
-        sendTransferableBlock(transferableBlock);
+        sendTransferableBlock(transferableBlock, false);
       } catch (Exception e2) {
         LOGGER.error("Exception while sending error block.", e2);
       }
     }
-    // yield if we cannot continue to put transferable block into the sending queue
-    return canContinue ? transferableBlock : TransferableBlockUtils.getNoOpTransferableBlock();
+    return transferableBlock;
   }
 
-  private boolean sendTransferableBlock(TransferableBlock block)
+  private void sendTransferableBlock(TransferableBlock block, boolean throwIfTimeout)

Review Comment:
   IIUC, if it times out, we want to early terminate the operator, so throwing exception is the fastest option. Can you confirm @walterddr 



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #11205: Remove noop

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1293194215


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java:
##########
@@ -0,0 +1,246 @@
+/**
+ * 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.utils;
+
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BlockingMultiStreamConsumer<E> implements AutoCloseable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlockingMultiStreamConsumer.class);
+  private final Object _id;
+  protected final List<? extends AsyncStream<E>> _mailboxes;
+  protected final ArrayBlockingQueue<Boolean> _newDataReady = new ArrayBlockingQueue<>(1);
+  private final long _deadlineMs;
+  /**
+   * An index that used to calculate where do we are going to start reading.
+   * The invariant is that we are always going to start reading from {@code _lastRead + 1}.
+   * Therefore {@link #_lastRead} must be in the range {@code [-1, mailbox.size() - 1]}
+   */
+  protected int _lastRead;
+  private E _errorBlock = null;
+
+  public BlockingMultiStreamConsumer(Object id, long deadlineMs, List<? extends AsyncStream<E>> asyncProducers) {
+    _id = id;
+    _deadlineMs = deadlineMs;
+    AsyncStream.OnNewData onNewData = this::onData;
+    _mailboxes = asyncProducers;
+    _mailboxes.forEach(blockProducer -> blockProducer.addOnNewDataListener(onNewData));
+    _lastRead = _mailboxes.size() - 1;
+  }
+
+  protected abstract boolean isError(E element);
+
+  protected abstract boolean isEos(E element);
+
+  protected abstract E onTimeout();
+
+  protected abstract E onException(Exception e);
+
+  protected abstract E onEos();
+
+  @Override
+  public void close() {
+    cancelRemainingMailboxes();
+  }
+
+  public void cancel(Throwable t) {
+    cancelRemainingMailboxes();
+  }
+
+  protected void cancelRemainingMailboxes() {
+    for (AsyncStream<E> mailbox : _mailboxes) {
+      mailbox.cancel();
+    }
+  }
+
+  public void onData() {
+    if (_newDataReady.offer(Boolean.TRUE)) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("New data notification delivered on " + _id + ". " + System.identityHashCode(_newDataReady));
+      }
+    } else if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("New data notification ignored on " + _id + ". " + System.identityHashCode(_newDataReady));
+    }
+  }
+
+  /**
+   * Reads the next block for any ready mailbox or blocks until some of them is ready.
+   *
+   * The method implements a sequential read semantic. Meaning that:
+   * <ol>
+   *   <li>EOS is only returned when all mailboxes already emitted EOS or there are no mailboxes</li>
+   *   <li>If an error is read from a mailbox, the error is returned</li>
+   *   <li>If data is read from a mailbox, that data block is returned</li>
+   *   <li>If no mailbox is ready, the calling thread is blocked</li>
+   * </ol>
+   *
+   * Right now the implementation tries to be fair. If one call returned the block from mailbox {@code i}, then next
+   * call will look for mailbox {@code i+1}, {@code i+2}... in a circular manner.
+   *
+   * In order to unblock a thread blocked here, {@link #onData()} should be called.   *
+   */
+  public E readBlockBlocking() {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
+    }
+    // Standard optimistic execution. First we try to read without acquiring the lock.
+    E block = readDroppingSuccessEos();
+    long timeoutMs = _deadlineMs - System.currentTimeMillis();
+    boolean timeout = false;
+    try {
+      while (block == null) { // we didn't find a mailbox ready to read, so we need to be pessimistic

Review Comment:
   I think it works, but with some logic very clean:
   1. The `timeout` is not updated within the `while` loop, which means we might wait the full timeout multiple times (Although in reality we won't wait for 2 full timeout because if the first `poll()` succeeds without having a block, it should return immediately)
   2. After timed out, we still pull another block.
   
   Here I've written down the code that I feel more clear logically:
   ```
     public E readBlockBlocking() {
       if (LOGGER.isTraceEnabled()) {
         LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
       }
       // Standard optimistic execution. First we try to read without acquiring the lock.
       E block = readDroppingSuccessEos();
       if (block != null) {
         return block;
       }
       try {
         while (true) {
           if (LOGGER.isDebugEnabled()) {
             LOGGER.debug("==[RECEIVE]== Blocked on : " + _id + ". " + System.identityHashCode(_newDataReady));
           }
           if (_newDataReady.poll(_deadlineMs - System.currentTimeMillis(), TimeUnit.MILLISECONDS) == null) {
             if (LOGGER.isDebugEnabled()) {
               LOGGER.warn("==[RECEIVE]== Timeout on: " + _id);
             }
             _errorBlock = onTimeout();
             return _errorBlock;
           }
           LOGGER.debug("==[RECEIVE]== More data available. Trying to read again");
           block = readDroppingSuccessEos();
           if (block != null) {
             if (LOGGER.isTraceEnabled()) {
               LOGGER.trace("==[RECEIVE]== Ready to emit on: " + _id);
             }
             return block;
           }
         }
       } catch (InterruptedException e) {
         return onException(e);
       }
     }
   ```
   
   Let me know your opinion



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] codecov-commenter commented on pull request #11205: [Draft] Remove noop

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #11205:
URL: https://github.com/apache/pinot/pull/11205#issuecomment-1655845440

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#11205](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (ea6261d) into [master](https://app.codecov.io/gh/apache/pinot/commit/7231ecfee875ac80ea61b329887ab16ad65fb94f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (7231ecf) will **increase** coverage by `0.00%`.
   > Report is 30 commits behind head on master.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #11205     +/-   ##
   =========================================
     Coverage    0.11%    0.11%             
   =========================================
     Files        2213     2168     -45     
     Lines      118670   116610   -2060     
     Branches    17958    17712    -246     
   =========================================
     Hits          137      137             
   + Misses     118513   116453   -2060     
     Partials       20       20             
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1temurin11 | `?` | |
   | integration1temurin17 | `?` | |
   | integration1temurin20 | `?` | |
   | integration2temurin11 | `?` | |
   | integration2temurin20 | `?` | |
   | unittests1temurin17 | `?` | |
   | unittests1temurin20 | `?` | |
   | unittests2temurin11 | `?` | |
   | unittests2temurin17 | `?` | |
   | unittests2temurin20 | `0.11% <0.00%> (-0.01%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Files Changed](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [...requesthandler/MultiStageBrokerRequestHandler.java](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcmVxdWVzdGhhbmRsZXIvTXVsdGlTdGFnZUJyb2tlclJlcXVlc3RIYW5kbGVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../apache/pinot/common/datablock/DataBlockUtils.java](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZGF0YWJsb2NrL0RhdGFCbG9ja1V0aWxzLmphdmE=) | `0.00% <ø> (ø)` | |
   | [...g/apache/pinot/common/datablock/MetadataBlock.java](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZGF0YWJsb2NrL01ldGFkYXRhQmxvY2suamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...in/java/org/apache/pinot/core/common/Operator.java](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9jb21tb24vT3BlcmF0b3IuamF2YQ==) | `0.00% <ø> (ø)` | |
   | [...apache/pinot/query/mailbox/GrpcSendingMailbox.java](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvbWFpbGJveC9HcnBjU2VuZGluZ01haWxib3guamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...org/apache/pinot/query/mailbox/MailboxService.java](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvbWFpbGJveC9NYWlsYm94U2VydmljZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...g/apache/pinot/query/mailbox/ReceivingMailbox.java](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvbWFpbGJveC9SZWNlaXZpbmdNYWlsYm94LmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...va/org/apache/pinot/query/runtime/QueryRunner.java](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9RdWVyeVJ1bm5lci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [.../pinot/query/runtime/blocks/TransferableBlock.java](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9ibG9ja3MvVHJhbnNmZXJhYmxlQmxvY2suamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...t/query/runtime/blocks/TransferableBlockUtils.java](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcnVudGltZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvcnVudGltZS9ibG9ja3MvVHJhbnNmZXJhYmxlQmxvY2tVdGlscy5qYXZh) | `0.00% <ø> (ø)` | |
   | ... and [18 more](https://app.codecov.io/gh/apache/pinot/pull/11205?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | |
   
   ... and [180 files with indirect coverage changes](https://app.codecov.io/gh/apache/pinot/pull/11205/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   


-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277704105


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/QueryRunnerTest.java:
##########
@@ -211,6 +208,13 @@ public void testSqlWithExceptionMsgChecker(String sql, String exceptionMsg) {
           _reducerScheduler, null, false);
     } catch (RuntimeException rte) {
       Assert.assertTrue(rte.getMessage().contains("Received error query execution result block"));
+      // TODO: The actual message is (usually) something like:

Review Comment:
   I don't know if this is actually what we want to test or not. 
   
   I think there is a race condition and sometimes the message we get is directly the one from the original error block (Illegal Json Path...) and other times we receive a new error block created during the op chain cancellation, which did not contain the message. In order to pass the test I added the original message to that cancellation block, but it may just a hack that makes the test pass but does not implement the logic we actually wanted to test



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277708587


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java:
##########
@@ -74,16 +78,17 @@ public void complete() {
   }
 
   @Override
-  public void cancel(Throwable t) {
+  public void cancel(@Nullable Throwable t) {

Review Comment:
   There were some arguments and parameters that could be null but were not marked as such. I tried to mark the ones I discovered.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277740252


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());

Review Comment:
   Here I'm adding the message in order to fix some tests, but as explained in the affected test, I'm not sure if this is the correct solution



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277782138


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperatorTest.java:
##########
@@ -139,21 +138,11 @@ public void shouldTimeoutOnExtraLongSleep()
       MetadataBlock errorBlock = (MetadataBlock) mailbox.getDataBlock();
       assertTrue(errorBlock.getExceptions().containsKey(QueryException.EXECUTION_TIMEOUT_ERROR_CODE));
     }
-
-    // Longer timeout or default timeout (10s) doesn't result in timeout
-    context =
-        OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, System.currentTimeMillis() + 10_000L,
-            _stageMetadata1);
-    try (MailboxReceiveOperator receiveOp = new MailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, 1)) {
-      Thread.sleep(100L);
-      TransferableBlock mailbox = receiveOp.nextBlock();
-      assertFalse(mailbox.isErrorBlock());
-    }
   }
 
-  @Test
+  @Test(enabled = false)
   public void shouldReceiveSingletonNullMailbox() {
-    when(_mailboxService.getReceivingMailbox(MAILBOX_ID_1)).thenReturn(_mailbox1);
+    when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1);

Review Comment:
   I think this test does not apply now



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1285466350


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {

Review Comment:
   I don't get that. In case the cancel signal is lost, what would happen is that the node that should have receive it would execute the query as it was expected. I assume that the receiving mailbox will be eventually closed but even if it doesn't the reader would timeout. That means that the runnable will throw an exception that will be caught here and therefore cancel the op chain.
   
   By adding a timeout here we would add two competing timeouts (the one reading the mailbox and the one here) which would make the code more undeterministic without giving us more guarantees.
   
   Also, it doesn't seem to be what concerns Jackie. He seems to be concerned about the cost of calling this method several times per query.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1280900873


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -51,16 +52,30 @@ public class ReceivingMailbox {
   // TODO: Revisit if this is the correct way to apply back pressure
   private final BlockingQueue<TransferableBlock> _blocks = new ArrayBlockingQueue<>(DEFAULT_MAX_PENDING_BLOCKS);
   private final AtomicReference<TransferableBlock> _errorBlock = new AtomicReference<>();
+  @Nullable
+  private volatile Reader _reader;

Review Comment:
   discussed offline. let's keep the callback for now b/c 
   1. PB uses it to indicate whether all PB worker finishes (which can be optimized). and 
   2. callback is on a OpChain level and reader is on a mailbox level thus it is not a "superset" of functionality 



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289828568


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/ExecutorServiceUtils.java:
##########
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.executor;
+
+import java.util.List;
+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.spi.env.PinotConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExecutorServiceUtils {
+  private static final Logger LOGGER = LoggerFactory.getLogger(ExecutorServiceUtils.class);
+
+  private ExecutorServiceUtils() {
+  }
+
+  public static ExecutorService createDefault(String baseName) {
+    return Executors.newCachedThreadPool(new NamedThreadFactory(baseName));
+  }
+
+  public static ExecutorService create(PinotConfiguration conf, String confPrefix, String baseName) {
+    //TODO: make this configurable
+    return Executors.newCachedThreadPool(new NamedThreadFactory(baseName));
+  }
+
+  public static void close(ExecutorService executorService) {

Review Comment:
   Done in d09a481a3df7728cb07286918bc6fb7f599f0e90



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang merged pull request #11205: Remove noop

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang merged PR #11205:
URL: https://github.com/apache/pinot/pull/11205


-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1280795098


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -133,4 +152,15 @@ public void cancel() {
   public int getNumPendingBlocks() {
     return _blocks.size();
   }
+
+  private void notifyReader() {
+    Reader reader = _reader;
+    if (reader != null) {

Review Comment:
   The actual reason is that 4 cannot happen because 2 already appended the message in the array



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281457889


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -51,16 +52,30 @@ public class ReceivingMailbox {
   // TODO: Revisit if this is the correct way to apply back pressure
   private final BlockingQueue<TransferableBlock> _blocks = new ArrayBlockingQueue<>(DEFAULT_MAX_PENDING_BLOCKS);
   private final AtomicReference<TransferableBlock> _errorBlock = new AtomicReference<>();
+  @Nullable
+  private volatile Reader _reader;
 
   public ReceivingMailbox(String id, Consumer<OpChainId> receiveMailCallback) {
     _id = id;
     _receiveMailCallback = receiveMailCallback;
   }
 
+  public void registeredReader(Reader reader) {
+    if (_reader != null) {
+      throw new IllegalArgumentException("Only one reader is supported");
+    }
+    LOGGER.debug("==[MAILBOX]== Reader registered for mailbox {}", _id);

Review Comment:
   I can take a look to the new message we have added in this PR. But my impression is that they are not expensive. In this case, for example, it is one array created when a reader is registered (so once per mailbox).



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1283270780


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {

Review Comment:
   yes we rely on upstream cancellation



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289769475


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/ExecutorServiceUtils.java:
##########
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.executor;
+
+import java.util.List;
+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.spi.env.PinotConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExecutorServiceUtils {
+  private static final Logger LOGGER = LoggerFactory.getLogger(ExecutorServiceUtils.class);
+
+  private ExecutorServiceUtils() {
+  }
+
+  public static ExecutorService createDefault(String baseName) {
+    return Executors.newCachedThreadPool(new NamedThreadFactory(baseName));
+  }
+
+  public static ExecutorService create(PinotConfiguration conf, String confPrefix, String baseName) {
+    //TODO: make this configurable
+    return Executors.newCachedThreadPool(new NamedThreadFactory(baseName));
+  }

Review Comment:
   The idea here is that in Java 21 we may want to use a virtual pool here. 
   
   That can be disabled by default (therefore we would need to set a property in order to use loom) or enabled by default (in which case we would use a property to do not use loom). Given that these executors are created in different parts of the code, this util will be used to read the properties in a single place.
   
   One thing I don't like about the current approach is that `create` and `createDefault` are quite generic. Should be something like `createUnbound` because the idea is that we should call this method whenever we want to create a unbound thread pool executor



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289956471


##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java:
##########
@@ -113,14 +113,11 @@ public MultiStageBrokerRequestHandler(PinotConfiguration config, String brokerId
         new WorkerManager(_reducerHostname, _reducerPort, routingManager), _tableCache);
     _queryDispatcher = new QueryDispatcher();
 
-    long releaseMs = config.getProperty(QueryConfig.KEY_OF_SCHEDULER_RELEASE_TIMEOUT_MS,
-        QueryConfig.DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS);
-    _reducerScheduler = new OpChainSchedulerService(new RoundRobinScheduler(releaseMs),
-        Executors.newCachedThreadPool(new NamedThreadFactory("query_broker_reducer_" + _reducerPort + "_port")));
-    _mailboxService = new MailboxService(_reducerHostname, _reducerPort, config, _reducerScheduler::onDataAvailable);
+    _opChainExecutor = ExecutorServiceUtils.create(config, "multiStage", "multiStage_on_" + _reducerPort + "port");

Review Comment:
   Old name recovered  in 93f929f378b9ff74f39bcec6d3f6e2db8827ce79



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1286653961


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java:
##########
@@ -81,9 +84,10 @@ public void cancel(Throwable t) {
         _contentObserver = getContentObserver();
       }
       try {
+        String msg = t != null ? t.getMessage() : "Unknown";

Review Comment:
   (minor) Is this needed?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/ExecutorServiceUtils.java:
##########
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.executor;
+
+import java.util.List;
+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.spi.env.PinotConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExecutorServiceUtils {
+  private static final Logger LOGGER = LoggerFactory.getLogger(ExecutorServiceUtils.class);
+
+  private ExecutorServiceUtils() {
+  }
+
+  public static ExecutorService createDefault(String baseName) {
+    return Executors.newCachedThreadPool(new NamedThreadFactory(baseName));
+  }
+
+  public static ExecutorService create(PinotConfiguration conf, String confPrefix, String baseName) {
+    //TODO: make this configurable
+    return Executors.newCachedThreadPool(new NamedThreadFactory(baseName));
+  }
+
+  public static void close(ExecutorService executorService) {

Review Comment:
   (minor) Let's pass in the timeout



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java:
##########
@@ -173,30 +165,32 @@ public void cancel(long requestId) {
 
   @VisibleForTesting
   public ExecutorService getQueryWorkerLeafExecutorService() {
-    return _queryWorkerLeafExecutorService;
+    return _opChainExecutor;
   }
 
   @VisibleForTesting
   public ExecutorService getQueryWorkerIntermExecutorService() {
-    return _queryWorkerIntermExecutorService;
+    return _opChainExecutor;
   }

Review Comment:
   (minor) Can we clean this up since there is only one executor



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java:
##########
@@ -118,46 +118,52 @@ public String toExplainString() {
 
   @Override
   protected TransferableBlock getNextBlock() {
-    boolean canContinue = true;
     TransferableBlock transferableBlock;
+    if (LOGGER.isDebugEnabled()) {
+      LOGGER.debug("==[SEND]== Enter getNextBlock from: " + _context.getId());
+    }
     try {
       transferableBlock = _sourceOperator.nextBlock();
-      if (transferableBlock.isNoOpBlock()) {
-        return transferableBlock;
-      } else if (transferableBlock.isEndOfStreamBlock()) {
-        if (transferableBlock.isSuccessfulEndOfStreamBlock()) {
-          // Stats need to be populated here because the block is being sent to the mailbox
-          // and the receiving opChain will not be able to access the stats from the previous opChain
-          TransferableBlock eosBlockWithStats = TransferableBlockUtils.getEndOfStreamTransferableBlock(
-              OperatorUtils.getMetadataFromOperatorStats(_opChainStats.getOperatorStatsMap()));
-          sendTransferableBlock(eosBlockWithStats);
-        } else {
-          sendTransferableBlock(transferableBlock);
-        }
-      } else { // normal blocks
-        // check whether we should continue depending on exchange queue condition.
-        canContinue = sendTransferableBlock(transferableBlock);
+      if (transferableBlock.isSuccessfulEndOfStreamBlock()) {
+        // Stats need to be populated here because the block is being sent to the mailbox
+        // and the receiving opChain will not be able to access the stats from the previous opChain
+        TransferableBlock eosBlockWithStats = TransferableBlockUtils.getEndOfStreamTransferableBlock(
+            OperatorUtils.getMetadataFromOperatorStats(_opChainStats.getOperatorStatsMap()));
+        sendTransferableBlock(eosBlockWithStats, false);
+      } else {
+        sendTransferableBlock(transferableBlock, true);
       }
     } catch (Exception e) {
       transferableBlock = TransferableBlockUtils.getErrorTransferableBlock(e);
       try {
         LOGGER.error("Exception while transferring data on opChain: " + _context.getId(), e);
-        sendTransferableBlock(transferableBlock);
+        sendTransferableBlock(transferableBlock, false);
       } catch (Exception e2) {
         LOGGER.error("Exception while sending error block.", e2);
       }
     }
-    // yield if we cannot continue to put transferable block into the sending queue
-    return canContinue ? transferableBlock : TransferableBlockUtils.getNoOpTransferableBlock();
+    return transferableBlock;
   }
 
-  private boolean sendTransferableBlock(TransferableBlock block)
+  private void sendTransferableBlock(TransferableBlock block, boolean throwIfTimeout)

Review Comment:
   Should we always throw on timeout?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultiStageOperator.java:
##########
@@ -44,6 +45,7 @@ public MultiStageOperator(OpChainExecutionContext context) {
     _opChainStats = _context.getStats();
   }
 
+  @Nonnull

Review Comment:
   (minor) Since multi-stage operator never returns `null` block (only very few single-stage operator can return `null`), suggest not changing the `Operator` interface right now because that will introduce quite some warnings in single-stage engine



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/TransformOperator.java:
##########
@@ -86,13 +86,14 @@ protected TransferableBlock getNextBlock() {
     try {
       TransferableBlock block = _upstreamOperator.nextBlock();
       return transform(block);
-    } catch (Exception e) {
+    } catch (RuntimeException e) {
       return TransferableBlockUtils.getErrorTransferableBlock(e);
     }
   }
 
-  private TransferableBlock transform(TransferableBlock block)
-      throws Exception {
+  private TransferableBlock transform(TransferableBlock block) {
+    // TODO: Other operators keep the first erroneous block, while this keep the last.
+    //  We should decide what is what we want to do and be consistent with that.

Review Comment:
   I guess it doesn't matter because the execution will terminate after the first error block is returned?



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java:
##########
@@ -113,14 +113,11 @@ public MultiStageBrokerRequestHandler(PinotConfiguration config, String brokerId
         new WorkerManager(_reducerHostname, _reducerPort, routingManager), _tableCache);
     _queryDispatcher = new QueryDispatcher();
 
-    long releaseMs = config.getProperty(QueryConfig.KEY_OF_SCHEDULER_RELEASE_TIMEOUT_MS,
-        QueryConfig.DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS);
-    _reducerScheduler = new OpChainSchedulerService(new RoundRobinScheduler(releaseMs),
-        Executors.newCachedThreadPool(new NamedThreadFactory("query_broker_reducer_" + _reducerPort + "_port")));
-    _mailboxService = new MailboxService(_reducerHostname, _reducerPort, config, _reducerScheduler::onDataAvailable);
+    _opChainExecutor = ExecutorServiceUtils.create(config, "multiStage", "multiStage_on_" + _reducerPort + "port");

Review Comment:
   (minor) Shall we keep the same thread name, or is it intentional to change it?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java:
##########
@@ -188,4 +193,10 @@ private boolean isType(MetadataBlock.MetadataBlockType type) {
     MetadataBlock metadata = (MetadataBlock) _dataBlock;
     return metadata.getType() == type;
   }
+
+  @Override
+  public String toString() {
+    String blockType = isErrorBlock() ? "error" : isSuccessfulEndOfStreamBlock() ? "eos" : "data";
+    return "TransferableBlock{blockType=" + blockType + ", _numRows=" + _numRows + ", _container=" + _container + '}';

Review Comment:
   We should not include `_container` here. It could be huge



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/TransformOperator.java:
##########
@@ -86,13 +86,14 @@ protected TransferableBlock getNextBlock() {
     try {
       TransferableBlock block = _upstreamOperator.nextBlock();
       return transform(block);
-    } catch (Exception e) {
+    } catch (RuntimeException e) {

Review Comment:
   Should we catch it or just throw it out?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/ExecutorServiceUtils.java:
##########
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.executor;
+
+import java.util.List;
+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.spi.env.PinotConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExecutorServiceUtils {
+  private static final Logger LOGGER = LoggerFactory.getLogger(ExecutorServiceUtils.class);
+
+  private ExecutorServiceUtils() {
+  }
+
+  public static ExecutorService createDefault(String baseName) {
+    return Executors.newCachedThreadPool(new NamedThreadFactory(baseName));
+  }
+
+  public static ExecutorService create(PinotConfiguration conf, String confPrefix, String baseName) {
+    //TODO: make this configurable
+    return Executors.newCachedThreadPool(new NamedThreadFactory(baseName));
+  }

Review Comment:
   (minor) I feel this is not general enough to be an util class for 2 reasons:
   - It is always creating a cached thread pool
   - We are not using any config
   
   Suggest directly creating the executor instead of using this method



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java:
##########
@@ -167,8 +167,13 @@ public boolean isSuccessfulEndOfStreamBlock() {
   /**
    * @return whether this block represents a NOOP block
    */
+  @Deprecated(forRemoval = true)
   public boolean isNoOpBlock() {
-    return isType(MetadataBlock.MetadataBlockType.NOOP);
+    return false;

Review Comment:
   (minor) Remember to clean this up



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +32,73 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
 
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
-  }
-
-  @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
-        @Override
-        public void runJob() {
-          boolean isFinished = false;
-          boolean returnedErrorBlock = false;
-          Throwable thrown = null;
-          try {
-            LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
-            operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
-            TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
-              result = operatorChain.getRoot().nextBlock();
-            }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
-            } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
-            }
-          } catch (Exception e) {
-            LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
-            thrown = e;
-          } finally {
-            if (returnedErrorBlock || thrown != null) {
-              cancelOpChain(operatorChain, thrown);
-            } else if (isFinished) {
-              closeOpChain(operatorChain);
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
+      @Override
+      public void runJob() {
+        boolean isFinished = false;
+        TransferableBlock returnedErrorBlock = null;
+        Throwable thrown = null;
+        try {
+          LOGGER.trace("({}): Executing", operatorChain);
+          operatorChain.getStats().executing();
+          TransferableBlock result = operatorChain.getRoot().nextBlock();
+          while (!result.isEndOfStreamBlock()) {
+            result = operatorChain.getRoot().nextBlock();
+          }
+          isFinished = true;
+          if (result.isErrorBlock()) {
+            returnedErrorBlock = result;
+            LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                result.getDataBlock().getExceptions());
+          } else {
+            LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
+          }
+        } catch (Exception e) {
+          LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
+          thrown = e;
+        } finally {
+          _submittedOpChainMap.remove(operatorChain.getId());
+          if (returnedErrorBlock != null || thrown != null) {
+            if (thrown == null) {
+              thrown = new RuntimeException("Error block " + returnedErrorBlock.getDataBlock().getExceptions());
             }
+            operatorChain.cancel(thrown);
+          } else if (isFinished) {
+            operatorChain.close();
           }
         }
-      });
-    }
-  }
-
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
-  }
-
-  private void closeOpChain(OpChain opChain) {
-    try {
-      opChain.close();
-    } finally {
-      _scheduler.deregister(opChain);
-    }
+      }
+    });
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  private void cancelOpChain(OpChain opChain, Throwable e) {
-    try {
-      opChain.cancel(e);
-    } finally {
-      _scheduler.deregister(opChain);
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. for leaf stage this cannot be a dangling opchain b/c they will eventually be cleared up
+    // via query timeout.
+    List<Map.Entry<OpChainId, Future<?>>> entries =
+        _submittedOpChainMap.entrySet().stream().filter(entry -> entry.getKey().getRequestId() == requestId)
+            .collect(Collectors.toList());
+    for (Map.Entry<OpChainId, Future<?>> entry : entries) {
+      OpChainId key = entry.getKey();
+      Future<?> future = entry.getValue();
+      if (future != null) {
+        future.cancel(true);
+      }
+      _submittedOpChainMap.remove(key);

Review Comment:
   (minor) Can reduce several map access
   ```suggestion
       Iterator<Map.Entry<OpChainId, Future<?>>> iterator = _submittedOpChainMap.entrySet().iterator();
       while (iterator.hasNext()) {
         Map.Entry<OpChainId, Future<?>> entry = iterator.next();
         if (entry.getKey().getRequestId() == requestId) {
           entry.getValue().cancel(true);
           iterator.remove();
         }
       }
   ```
   
   Not introduced in this PR, but can there be race condition of a query being cancelled first, then one OpChain is registered?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java:
##########
@@ -0,0 +1,246 @@
+/**
+ * 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.utils;
+
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BlockingMultiStreamConsumer<E> implements AutoCloseable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlockingMultiStreamConsumer.class);
+  private final Object _id;
+  protected final List<? extends AsyncStream<E>> _mailboxes;
+  protected final ArrayBlockingQueue<Boolean> _newDataReady = new ArrayBlockingQueue<>(1);
+  private final long _deadlineMs;
+  /**
+   * An index that used to calculate where do we are going to start reading.
+   * The invariant is that we are always going to start reading from {@code _lastRead + 1}.
+   * Therefore {@link #_lastRead} must be in the range {@code [-1, mailbox.size() - 1]}
+   */
+  protected int _lastRead;
+  private E _errorBlock = null;
+
+  public BlockingMultiStreamConsumer(Object id, long deadlineMs, List<? extends AsyncStream<E>> asyncProducers) {
+    _id = id;
+    _deadlineMs = deadlineMs;
+    AsyncStream.OnNewData onNewData = this::onData;
+    _mailboxes = asyncProducers;
+    _mailboxes.forEach(blockProducer -> blockProducer.addOnNewDataListener(onNewData));
+    _lastRead = _mailboxes.size() - 1;
+  }
+
+  protected abstract boolean isError(E element);
+
+  protected abstract boolean isEos(E element);
+
+  protected abstract E onTimeout();
+
+  protected abstract E onException(Exception e);
+
+  protected abstract E onEos();
+
+  @Override
+  public void close() {
+    cancelRemainingMailboxes();
+  }
+
+  public void cancel(Throwable t) {
+    cancelRemainingMailboxes();
+  }
+
+  protected void cancelRemainingMailboxes() {
+    for (AsyncStream<E> mailbox : _mailboxes) {
+      mailbox.cancel();
+    }
+  }
+
+  public void onData() {
+    if (_newDataReady.offer(Boolean.TRUE)) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("New data notification delivered on " + _id + ". " + System.identityHashCode(_newDataReady));
+      }
+    } else if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("New data notification ignored on " + _id + ". " + System.identityHashCode(_newDataReady));
+    }
+  }
+
+  /**
+   * Reads the next block for any ready mailbox or blocks until some of them is ready.
+   *
+   * The method implements a sequential read semantic. Meaning that:
+   * <ol>
+   *   <li>EOS is only returned when all mailboxes already emitted EOS or there are no mailboxes</li>
+   *   <li>If an error is read from a mailbox, the error is returned</li>
+   *   <li>If data is read from a mailbox, that data block is returned</li>
+   *   <li>If no mailbox is ready, the calling thread is blocked</li>
+   * </ol>
+   *
+   * Right now the implementation tries to be fair. If one call returned the block from mailbox {@code i}, then next
+   * call will look for mailbox {@code i+1}, {@code i+2}... in a circular manner.
+   *
+   * In order to unblock a thread blocked here, {@link #onData()} should be called.   *
+   */
+  public E readBlockBlocking() {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
+    }
+    // Standard optimistic execution. First we try to read without acquiring the lock.
+    E block = readDroppingSuccessEos();
+    long timeoutMs = _deadlineMs - System.currentTimeMillis();
+    boolean timeout = false;
+    try {
+      while (block == null) { // we didn't find a mailbox ready to read, so we need to be pessimistic
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.debug("==[RECEIVE]== Blocked on : " + _id + ". " + System.identityHashCode(_newDataReady));
+        }
+        timeout = _newDataReady.poll(timeoutMs, TimeUnit.MILLISECONDS) == null;
+        LOGGER.debug("==[RECEIVE]== More data available. Trying to read again");
+        block = readDroppingSuccessEos();
+      }
+
+      if (timeout) {
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.warn("==[RECEIVE]== Timeout on: " + _id);
+        }
+        _errorBlock = onTimeout();
+        return _errorBlock;
+      } else if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("==[RECEIVE]== Ready to emit on: " + _id);
+      }
+    } catch (InterruptedException ex) {
+      return onException(ex);
+    }
+    return block;
+  }
+
+  /**
+   * This is a utility method that reads tries to read from the different mailboxes in a circular manner.
+   *
+   * The method is a bit more complex than expected because ir order to simplify {@link #readBlockBlocking} we added
+   * some extra logic here. For example, this method checks for timeouts, add some logs, releases mailboxes that emitted
+   * EOS and in case an error block is found, stores it.
+   *
+   * @return the new block to consume or null if none is found. EOS is only emitted when all mailboxes already emitted
+   * EOS.
+   */
+  @Nullable
+  private E readDroppingSuccessEos() {
+    if (System.currentTimeMillis() > _deadlineMs) {
+      _errorBlock = onTimeout();
+      return _errorBlock;
+    }
+
+    E block = readBlockOrNull();
+    while (block != null && isEos(block) && !_mailboxes.isEmpty()) {

Review Comment:
   (minor) Can `_mailboxes` be empty if `block != null`?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java:
##########
@@ -61,6 +61,9 @@ public GrpcSendingMailbox(String id, ChannelManager channelManager, String hostn
   @Override
   public void send(TransferableBlock block)
       throws IOException {
+    if (LOGGER.isDebugEnabled()) {
+      LOGGER.debug("==[GRPC SEND]== sending data to: " + _id);

Review Comment:
   (minor) Same for other logs
   ```suggestion
         LOGGER.debug("==[GRPC SEND]== sending data to: {}", _id);
   ```



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java:
##########
@@ -0,0 +1,246 @@
+/**
+ * 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.utils;
+
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BlockingMultiStreamConsumer<E> implements AutoCloseable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlockingMultiStreamConsumer.class);
+  private final Object _id;
+  protected final List<? extends AsyncStream<E>> _mailboxes;
+  protected final ArrayBlockingQueue<Boolean> _newDataReady = new ArrayBlockingQueue<>(1);
+  private final long _deadlineMs;
+  /**
+   * An index that used to calculate where do we are going to start reading.
+   * The invariant is that we are always going to start reading from {@code _lastRead + 1}.
+   * Therefore {@link #_lastRead} must be in the range {@code [-1, mailbox.size() - 1]}
+   */
+  protected int _lastRead;
+  private E _errorBlock = null;
+
+  public BlockingMultiStreamConsumer(Object id, long deadlineMs, List<? extends AsyncStream<E>> asyncProducers) {
+    _id = id;
+    _deadlineMs = deadlineMs;
+    AsyncStream.OnNewData onNewData = this::onData;
+    _mailboxes = asyncProducers;
+    _mailboxes.forEach(blockProducer -> blockProducer.addOnNewDataListener(onNewData));
+    _lastRead = _mailboxes.size() - 1;
+  }
+
+  protected abstract boolean isError(E element);
+
+  protected abstract boolean isEos(E element);
+
+  protected abstract E onTimeout();
+
+  protected abstract E onException(Exception e);
+
+  protected abstract E onEos();
+
+  @Override
+  public void close() {
+    cancelRemainingMailboxes();
+  }
+
+  public void cancel(Throwable t) {
+    cancelRemainingMailboxes();
+  }
+
+  protected void cancelRemainingMailboxes() {
+    for (AsyncStream<E> mailbox : _mailboxes) {
+      mailbox.cancel();
+    }
+  }
+
+  public void onData() {
+    if (_newDataReady.offer(Boolean.TRUE)) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("New data notification delivered on " + _id + ". " + System.identityHashCode(_newDataReady));
+      }
+    } else if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("New data notification ignored on " + _id + ". " + System.identityHashCode(_newDataReady));
+    }
+  }
+
+  /**
+   * Reads the next block for any ready mailbox or blocks until some of them is ready.
+   *
+   * The method implements a sequential read semantic. Meaning that:
+   * <ol>
+   *   <li>EOS is only returned when all mailboxes already emitted EOS or there are no mailboxes</li>
+   *   <li>If an error is read from a mailbox, the error is returned</li>
+   *   <li>If data is read from a mailbox, that data block is returned</li>
+   *   <li>If no mailbox is ready, the calling thread is blocked</li>
+   * </ol>
+   *
+   * Right now the implementation tries to be fair. If one call returned the block from mailbox {@code i}, then next
+   * call will look for mailbox {@code i+1}, {@code i+2}... in a circular manner.
+   *
+   * In order to unblock a thread blocked here, {@link #onData()} should be called.   *
+   */
+  public E readBlockBlocking() {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
+    }
+    // Standard optimistic execution. First we try to read without acquiring the lock.
+    E block = readDroppingSuccessEos();
+    long timeoutMs = _deadlineMs - System.currentTimeMillis();
+    boolean timeout = false;
+    try {
+      while (block == null) { // we didn't find a mailbox ready to read, so we need to be pessimistic

Review Comment:
   I somehow see why you want to have this while check. The previous call of this might get a block from line 111 without consuming the `_newDataReady`, and next call might consume `_newDataReady` without having a new block. I feel it works if we break on `timeout`, but let's think more race conditions, and add a test for this



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SetOperator.java:
##########
@@ -112,15 +112,14 @@ protected void constructRightBlockSet() {
 
   protected TransferableBlock constructResultBlockSet(TransferableBlock leftBlock) {
     List<Object[]> rows = new ArrayList<>();
+    // TODO: Other operators keep the first erroneous block, while this keep the last.
+    //  We should decide what is what we want to do and be consistent with that.

Review Comment:
   I don't follow this `TODO`. Do you mean we should collect the first error block from right block set if exists?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java:
##########
@@ -0,0 +1,246 @@
+/**
+ * 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.utils;
+
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BlockingMultiStreamConsumer<E> implements AutoCloseable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlockingMultiStreamConsumer.class);
+  private final Object _id;
+  protected final List<? extends AsyncStream<E>> _mailboxes;
+  protected final ArrayBlockingQueue<Boolean> _newDataReady = new ArrayBlockingQueue<>(1);
+  private final long _deadlineMs;
+  /**
+   * An index that used to calculate where do we are going to start reading.
+   * The invariant is that we are always going to start reading from {@code _lastRead + 1}.
+   * Therefore {@link #_lastRead} must be in the range {@code [-1, mailbox.size() - 1]}
+   */
+  protected int _lastRead;
+  private E _errorBlock = null;
+
+  public BlockingMultiStreamConsumer(Object id, long deadlineMs, List<? extends AsyncStream<E>> asyncProducers) {
+    _id = id;
+    _deadlineMs = deadlineMs;
+    AsyncStream.OnNewData onNewData = this::onData;
+    _mailboxes = asyncProducers;
+    _mailboxes.forEach(blockProducer -> blockProducer.addOnNewDataListener(onNewData));
+    _lastRead = _mailboxes.size() - 1;
+  }
+
+  protected abstract boolean isError(E element);
+
+  protected abstract boolean isEos(E element);
+
+  protected abstract E onTimeout();
+
+  protected abstract E onException(Exception e);
+
+  protected abstract E onEos();
+
+  @Override
+  public void close() {
+    cancelRemainingMailboxes();
+  }
+
+  public void cancel(Throwable t) {
+    cancelRemainingMailboxes();
+  }
+
+  protected void cancelRemainingMailboxes() {
+    for (AsyncStream<E> mailbox : _mailboxes) {
+      mailbox.cancel();
+    }
+  }
+
+  public void onData() {
+    if (_newDataReady.offer(Boolean.TRUE)) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("New data notification delivered on " + _id + ". " + System.identityHashCode(_newDataReady));
+      }
+    } else if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("New data notification ignored on " + _id + ". " + System.identityHashCode(_newDataReady));
+    }
+  }
+
+  /**
+   * Reads the next block for any ready mailbox or blocks until some of them is ready.
+   *
+   * The method implements a sequential read semantic. Meaning that:
+   * <ol>
+   *   <li>EOS is only returned when all mailboxes already emitted EOS or there are no mailboxes</li>
+   *   <li>If an error is read from a mailbox, the error is returned</li>
+   *   <li>If data is read from a mailbox, that data block is returned</li>
+   *   <li>If no mailbox is ready, the calling thread is blocked</li>
+   * </ol>
+   *
+   * Right now the implementation tries to be fair. If one call returned the block from mailbox {@code i}, then next
+   * call will look for mailbox {@code i+1}, {@code i+2}... in a circular manner.
+   *
+   * In order to unblock a thread blocked here, {@link #onData()} should be called.   *
+   */
+  public E readBlockBlocking() {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
+    }
+    // Standard optimistic execution. First we try to read without acquiring the lock.
+    E block = readDroppingSuccessEos();
+    long timeoutMs = _deadlineMs - System.currentTimeMillis();
+    boolean timeout = false;
+    try {
+      while (block == null) { // we didn't find a mailbox ready to read, so we need to be pessimistic

Review Comment:
   I don't follow this part. Once we reach the timeout we should immediately return timeout error. Also, why do we have a `while` loop here? We should wait on `_newDataReady` once?



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1295577054


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java:
##########
@@ -0,0 +1,246 @@
+/**
+ * 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.utils;
+
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BlockingMultiStreamConsumer<E> implements AutoCloseable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlockingMultiStreamConsumer.class);
+  private final Object _id;
+  protected final List<? extends AsyncStream<E>> _mailboxes;
+  protected final ArrayBlockingQueue<Boolean> _newDataReady = new ArrayBlockingQueue<>(1);
+  private final long _deadlineMs;
+  /**
+   * An index that used to calculate where do we are going to start reading.
+   * The invariant is that we are always going to start reading from {@code _lastRead + 1}.
+   * Therefore {@link #_lastRead} must be in the range {@code [-1, mailbox.size() - 1]}
+   */
+  protected int _lastRead;
+  private E _errorBlock = null;
+
+  public BlockingMultiStreamConsumer(Object id, long deadlineMs, List<? extends AsyncStream<E>> asyncProducers) {
+    _id = id;
+    _deadlineMs = deadlineMs;
+    AsyncStream.OnNewData onNewData = this::onData;
+    _mailboxes = asyncProducers;
+    _mailboxes.forEach(blockProducer -> blockProducer.addOnNewDataListener(onNewData));
+    _lastRead = _mailboxes.size() - 1;
+  }
+
+  protected abstract boolean isError(E element);
+
+  protected abstract boolean isEos(E element);
+
+  protected abstract E onTimeout();
+
+  protected abstract E onException(Exception e);
+
+  protected abstract E onEos();
+
+  @Override
+  public void close() {
+    cancelRemainingMailboxes();
+  }
+
+  public void cancel(Throwable t) {
+    cancelRemainingMailboxes();
+  }
+
+  protected void cancelRemainingMailboxes() {
+    for (AsyncStream<E> mailbox : _mailboxes) {
+      mailbox.cancel();
+    }
+  }
+
+  public void onData() {
+    if (_newDataReady.offer(Boolean.TRUE)) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("New data notification delivered on " + _id + ". " + System.identityHashCode(_newDataReady));
+      }
+    } else if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("New data notification ignored on " + _id + ". " + System.identityHashCode(_newDataReady));
+    }
+  }
+
+  /**
+   * Reads the next block for any ready mailbox or blocks until some of them is ready.
+   *
+   * The method implements a sequential read semantic. Meaning that:
+   * <ol>
+   *   <li>EOS is only returned when all mailboxes already emitted EOS or there are no mailboxes</li>
+   *   <li>If an error is read from a mailbox, the error is returned</li>
+   *   <li>If data is read from a mailbox, that data block is returned</li>
+   *   <li>If no mailbox is ready, the calling thread is blocked</li>
+   * </ol>
+   *
+   * Right now the implementation tries to be fair. If one call returned the block from mailbox {@code i}, then next
+   * call will look for mailbox {@code i+1}, {@code i+2}... in a circular manner.
+   *
+   * In order to unblock a thread blocked here, {@link #onData()} should be called.   *
+   */
+  public E readBlockBlocking() {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
+    }
+    // Standard optimistic execution. First we try to read without acquiring the lock.
+    E block = readDroppingSuccessEos();
+    long timeoutMs = _deadlineMs - System.currentTimeMillis();
+    boolean timeout = false;
+    try {
+      while (block == null) { // we didn't find a mailbox ready to read, so we need to be pessimistic
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.debug("==[RECEIVE]== Blocked on : " + _id + ". " + System.identityHashCode(_newDataReady));
+        }
+        timeout = _newDataReady.poll(timeoutMs, TimeUnit.MILLISECONDS) == null;
+        LOGGER.debug("==[RECEIVE]== More data available. Trying to read again");
+        block = readDroppingSuccessEos();
+      }
+
+      if (timeout) {
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.warn("==[RECEIVE]== Timeout on: " + _id);
+        }
+        _errorBlock = onTimeout();
+        return _errorBlock;
+      } else if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("==[RECEIVE]== Ready to emit on: " + _id);
+      }
+    } catch (InterruptedException ex) {
+      return onException(ex);
+    }
+    return block;
+  }
+
+  /**
+   * This is a utility method that reads tries to read from the different mailboxes in a circular manner.
+   *
+   * The method is a bit more complex than expected because ir order to simplify {@link #readBlockBlocking} we added
+   * some extra logic here. For example, this method checks for timeouts, add some logs, releases mailboxes that emitted
+   * EOS and in case an error block is found, stores it.
+   *
+   * @return the new block to consume or null if none is found. EOS is only emitted when all mailboxes already emitted
+   * EOS.
+   */
+  @Nullable
+  private E readDroppingSuccessEos() {
+    if (System.currentTimeMillis() > _deadlineMs) {
+      _errorBlock = onTimeout();
+      return _errorBlock;
+    }
+
+    E block = readBlockOrNull();
+    while (block != null && isEos(block) && !_mailboxes.isEmpty()) {

Review Comment:
   I've moved the condition to an assertion in https://github.com/apache/pinot/commit/05dbd7975ca27c0d99c4fdbe7ff4551c154d7295



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on PR #11205:
URL: https://github.com/apache/pinot/pull/11205#issuecomment-1655545212

   cc @Jackie-Jiang @walterddr @xiangfu0 


-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277744774


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BaseMailboxReceiveOperator.java:
##########
@@ -90,9 +106,108 @@ public void cancel(Throwable t) {
   }
 
   protected void cancelRemainingMailboxes() {
-    ReceivingMailbox mailbox;
-    while ((mailbox = _mailboxes.poll()) != null) {
+    for (ReceivingMailbox mailbox : _mailboxes) {
       mailbox.cancel();
     }
   }
+
+  public void onData() {
+    _lock.lock();
+    try {
+      _notEmpty.signal();

Review Comment:
   This is how we unblock the reader opchain thread



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277745637


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BaseMailboxReceiveOperator.java:
##########
@@ -90,9 +106,108 @@ public void cancel(Throwable t) {
   }
 
   protected void cancelRemainingMailboxes() {
-    ReceivingMailbox mailbox;
-    while ((mailbox = _mailboxes.poll()) != null) {
+    for (ReceivingMailbox mailbox : _mailboxes) {
       mailbox.cancel();
     }
   }
+
+  public void onData() {
+    _lock.lock();
+    try {
+      _notEmpty.signal();
+    } finally {
+      _lock.unlock();
+    }
+  }
+
+  protected TransferableBlock readBlockBlocking() {

Review Comment:
   I've move this code here because both ReceivingMailboxOperator and SortedReceivingMailboxOperator use the same code.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277785384


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/SortedMailboxReceiveOperatorTest.java:
##########
@@ -163,23 +163,11 @@ public void shouldTimeoutOnExtraLongSleep()
       MetadataBlock errorBlock = (MetadataBlock) mailbox.getDataBlock();
       assertTrue(errorBlock.getExceptions().containsKey(QueryException.EXECUTION_TIMEOUT_ERROR_CODE));
     }
-
-    // Longer timeout or default timeout (10s) doesn't result in timeout
-    context =
-        OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, System.currentTimeMillis() + 10_000L,
-            _stageMetadata1);
-    try (SortedMailboxReceiveOperator receiveOp = new SortedMailboxReceiveOperator(context,
-        RelDistribution.Type.SINGLETON, DATA_SCHEMA, COLLATION_KEYS, COLLATION_DIRECTIONS, COLLATION_NULL_DIRECTIONS,
-        false, 1)) {
-      Thread.sleep(100L);
-      TransferableBlock mailbox = receiveOp.nextBlock();
-      assertFalse(mailbox.isErrorBlock());
-    }
   }
 
-  @Test
+  @Test(enabled = false)
   public void shouldReceiveSingletonNullMailbox() {
-    when(_mailboxService.getReceivingMailbox(MAILBOX_ID_1)).thenReturn(_mailbox1);
+    when(_mailboxService.getReceivingMailbox(eq(MAILBOX_ID_1))).thenReturn(_mailbox1);

Review Comment:
   Another test that I'm nto sure if we need to keep or not



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277707830


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxService.java:
##########
@@ -144,17 +150,30 @@ public SendingMailbox getSendingMailbox(String hostname, int port, String mailbo
     }
   }
 
-  /**
-   * Returns the receiving mailbox for the given mailbox id.
-   */
-  public ReceivingMailbox getReceivingMailbox(String mailboxId) {
+  public ReceivingMailbox getReceivingMailbox(String mailboxId, @Nullable Consumer<OpChainId> extraCallback) {
     try {
-      return _receivingMailboxCache.get(mailboxId, () -> new ReceivingMailbox(mailboxId, _unblockOpChainCallback));
+      Consumer<OpChainId> callback;
+      if (extraCallback == null) {
+        callback = _unblockOpChainCallback;
+      } else {
+        callback = opChainId -> {
+          extraCallback.accept(opChainId);

Review Comment:
   This is my main contribution over @walterddr's code. Instead of storing a map in the scheduler that maps op chains to their blocking objets (in his case a blocking list of 1 element) here I'm sending a message to the opchain itself. By doing that we decouple the receiving operation from the scheduler.
   
   You can see `BaseMailboxReceiveOperator.onData` to understand what it does.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1280085055


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java:
##########
@@ -62,10 +65,10 @@ public OpChainExecutionContext(MailboxService mailboxService, long requestId, in
     _traceEnabled = traceEnabled;
   }
 
-  public OpChainExecutionContext(PhysicalPlanContext physicalPlanContext) {
+  public OpChainExecutionContext(PhysicalPlanContext physicalPlanContext, Executor executor) {
     this(physicalPlanContext.getMailboxService(), physicalPlanContext.getRequestId(), physicalPlanContext.getStageId(),
         physicalPlanContext.getServer(), physicalPlanContext.getDeadlineMs(), physicalPlanContext.getStageMetadata(),
-        physicalPlanContext.getPipelineBreakerResult(), physicalPlanContext.isTraceEnabled());
+        physicalPlanContext.getPipelineBreakerResult(), physicalPlanContext.isTraceEnabled(), executor);

Review Comment:
   nit: it was wondering if we should separate the plan context from the machine context. 
   based on what i see: `mailboxService` / `executor` is machine context and the rest of physicalPlanContext should still remains. 
   
   This should be a good start but we can do better in next iteration



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/BaseMailboxReceiveOperator.java:
##########
@@ -61,13 +63,18 @@ public BaseMailboxReceiveOperator(OpChainExecutionContext context, RelDistributi
         context.getStageMetadata().getWorkerMetadataList().get(workerId).getMailBoxInfosMap().get(senderStageId);
     if (senderMailBoxMetadatas != null && !senderMailBoxMetadatas.getMailBoxIdList().isEmpty()) {
       _mailboxIds = MailboxIdUtils.toMailboxIds(requestId, senderMailBoxMetadatas);
-      _mailboxes = _mailboxIds.stream()
-          .map(mailboxId -> _mailboxService.getReceivingMailbox(mailboxId))
-          .collect(Collectors.toCollection(ArrayDeque::new));
     } else {
       _mailboxIds = Collections.emptyList();
-      _mailboxes = new ArrayDeque<>();
     }
+    List<ReadMailboxAsyncStream> asyncStreams = _mailboxIds.stream()

Review Comment:
   note to self. skip reviewing async stream for now



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SetOperator.java:
##########
@@ -112,15 +112,14 @@ protected void constructRightBlockSet() {
 
   protected TransferableBlock constructResultBlockSet(TransferableBlock leftBlock) {
     List<Object[]> rows = new ArrayList<>();
+    // TODO: Other operators keep the first erroneous block, while this keep the last.
+    //  We should decide what is what we want to do and be consistent with that.

Review Comment:
   CC @xiangfu0 



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java:
##########
@@ -167,8 +167,13 @@ public boolean isSuccessfulEndOfStreamBlock() {
   /**
    * @return whether this block represents a NOOP block
    */
+  @Deprecated(forRemoval = true)
   public boolean isNoOpBlock() {
-    return isType(MetadataBlock.MetadataBlockType.NOOP);
+    return false;

Review Comment:
   yes let's clean up the tests so we don't need to keep this 



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java:
##########
@@ -188,4 +193,10 @@ private boolean isType(MetadataBlock.MetadataBlockType type) {
     MetadataBlock metadata = (MetadataBlock) _dataBlock;
     return metadata.getType() == type;
   }
+
+  @Override
+  public String toString() {
+    String blockType = isErrorBlock() ? "error" : isSuccessfulEndOfStreamBlock() ? "eos" : "data";
+    return "TransferableBlock{blockType=" + blockType + ", _numRows=" + _numRows + ", _container=" + _container + '}';

Review Comment:
   :+1:



##########
pinot-core/src/main/java/org/apache/pinot/core/common/Operator.java:
##########
@@ -39,6 +39,7 @@ public interface Operator<T extends Block> {
    * @throws EarlyTerminationException if the operator is early-terminated (interrupted) before processing the next
    *         block of data. Operator can early terminated when the query times out, or is already satisfied.
    */
+  @Nullable

Review Comment:
   what's this annotation about? if this only affects v2 we should annotate on the multistage base operator?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java:
##########
@@ -101,9 +99,8 @@ public void init(PinotConfiguration config, InstanceDataManager instanceDataMana
           new NamedThreadFactory("query_intermediate_worker_on_" + _port + "_port"));
       _queryWorkerLeafExecutorService = Executors.newFixedThreadPool(ResourceManager.DEFAULT_QUERY_WORKER_THREADS,
           new NamedThreadFactory("query_leaf_worker_on_" + _port + "_port"));
-      _scheduler = new OpChainSchedulerService(new RoundRobinScheduler(releaseMs),
-          getQueryWorkerIntermExecutorService());
-      _mailboxService = new MailboxService(_hostname, _port, config, _scheduler::onDataAvailable);
+      _scheduler = new OpChainSchedulerService(getQueryWorkerIntermExecutorService());

Review Comment:
   1 is enough for now. later we want to deal with mixed v1/v2 workload we might want to reuse the same executor as the v1 leaf so that it is fair. but for now this is good



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1280337032


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -51,16 +51,30 @@ public class ReceivingMailbox {
   // TODO: Revisit if this is the correct way to apply back pressure
   private final BlockingQueue<TransferableBlock> _blocks = new ArrayBlockingQueue<>(DEFAULT_MAX_PENDING_BLOCKS);
   private final AtomicReference<TransferableBlock> _errorBlock = new AtomicReference<>();
+  @Nullable
+  private volatile Reader _reader;
 
   public ReceivingMailbox(String id, Consumer<OpChainId> receiveMailCallback) {
     _id = id;
     _receiveMailCallback = receiveMailCallback;
   }
 
+  public void registeredReader(Reader reader) {

Review Comment:
   I've applied your technique of using blocking queues instead of locks. I'm not 100% sure if that is needed, but I'm ok to use that for now.
   
   The race condition wasn't actually here but in BlockigToAsyncStream. There, I notified the reader just before finishing the completable future, so it could happen that the reader started to look for blocks before the completable future was finished.
   
   The solution is super simple: The main completable future should fetch from the blocking source and return its value without notifying. Then, a second runnable that notifies the reader should be executed once the main one finished. By doing that there is a happen before between the reader and the main completable future. Test seem to be consistently working on my computer.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1283270780


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {

Review Comment:
   yes we rely on upstream cancellation. if any exception occurred during submission or reduce we will invoke cancel see #10401 



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281476283


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);

Review Comment:
   Applied on 9031c6b5e09cd546c9b6a1657bf72548af129780



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #11205: Remove noop

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1293178220


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java:
##########
@@ -0,0 +1,246 @@
+/**
+ * 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.utils;
+
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BlockingMultiStreamConsumer<E> implements AutoCloseable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlockingMultiStreamConsumer.class);
+  private final Object _id;
+  protected final List<? extends AsyncStream<E>> _mailboxes;
+  protected final ArrayBlockingQueue<Boolean> _newDataReady = new ArrayBlockingQueue<>(1);
+  private final long _deadlineMs;
+  /**
+   * An index that used to calculate where do we are going to start reading.
+   * The invariant is that we are always going to start reading from {@code _lastRead + 1}.
+   * Therefore {@link #_lastRead} must be in the range {@code [-1, mailbox.size() - 1]}
+   */
+  protected int _lastRead;
+  private E _errorBlock = null;
+
+  public BlockingMultiStreamConsumer(Object id, long deadlineMs, List<? extends AsyncStream<E>> asyncProducers) {
+    _id = id;
+    _deadlineMs = deadlineMs;
+    AsyncStream.OnNewData onNewData = this::onData;
+    _mailboxes = asyncProducers;
+    _mailboxes.forEach(blockProducer -> blockProducer.addOnNewDataListener(onNewData));
+    _lastRead = _mailboxes.size() - 1;
+  }
+
+  protected abstract boolean isError(E element);
+
+  protected abstract boolean isEos(E element);
+
+  protected abstract E onTimeout();
+
+  protected abstract E onException(Exception e);
+
+  protected abstract E onEos();
+
+  @Override
+  public void close() {
+    cancelRemainingMailboxes();
+  }
+
+  public void cancel(Throwable t) {
+    cancelRemainingMailboxes();
+  }
+
+  protected void cancelRemainingMailboxes() {
+    for (AsyncStream<E> mailbox : _mailboxes) {
+      mailbox.cancel();
+    }
+  }
+
+  public void onData() {
+    if (_newDataReady.offer(Boolean.TRUE)) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("New data notification delivered on " + _id + ". " + System.identityHashCode(_newDataReady));
+      }
+    } else if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("New data notification ignored on " + _id + ". " + System.identityHashCode(_newDataReady));
+    }
+  }
+
+  /**
+   * Reads the next block for any ready mailbox or blocks until some of them is ready.
+   *
+   * The method implements a sequential read semantic. Meaning that:
+   * <ol>
+   *   <li>EOS is only returned when all mailboxes already emitted EOS or there are no mailboxes</li>
+   *   <li>If an error is read from a mailbox, the error is returned</li>
+   *   <li>If data is read from a mailbox, that data block is returned</li>
+   *   <li>If no mailbox is ready, the calling thread is blocked</li>
+   * </ol>
+   *
+   * Right now the implementation tries to be fair. If one call returned the block from mailbox {@code i}, then next
+   * call will look for mailbox {@code i+1}, {@code i+2}... in a circular manner.
+   *
+   * In order to unblock a thread blocked here, {@link #onData()} should be called.   *
+   */
+  public E readBlockBlocking() {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
+    }
+    // Standard optimistic execution. First we try to read without acquiring the lock.
+    E block = readDroppingSuccessEos();
+    long timeoutMs = _deadlineMs - System.currentTimeMillis();
+    boolean timeout = false;
+    try {
+      while (block == null) { // we didn't find a mailbox ready to read, so we need to be pessimistic
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.debug("==[RECEIVE]== Blocked on : " + _id + ". " + System.identityHashCode(_newDataReady));
+        }
+        timeout = _newDataReady.poll(timeoutMs, TimeUnit.MILLISECONDS) == null;
+        LOGGER.debug("==[RECEIVE]== More data available. Trying to read again");
+        block = readDroppingSuccessEos();
+      }
+
+      if (timeout) {
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.warn("==[RECEIVE]== Timeout on: " + _id);
+        }
+        _errorBlock = onTimeout();
+        return _errorBlock;
+      } else if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("==[RECEIVE]== Ready to emit on: " + _id);
+      }
+    } catch (InterruptedException ex) {
+      return onException(ex);
+    }
+    return block;
+  }
+
+  /**
+   * This is a utility method that reads tries to read from the different mailboxes in a circular manner.
+   *
+   * The method is a bit more complex than expected because ir order to simplify {@link #readBlockBlocking} we added
+   * some extra logic here. For example, this method checks for timeouts, add some logs, releases mailboxes that emitted
+   * EOS and in case an error block is found, stores it.
+   *
+   * @return the new block to consume or null if none is found. EOS is only emitted when all mailboxes already emitted
+   * EOS.
+   */
+  @Nullable
+  private E readDroppingSuccessEos() {
+    if (System.currentTimeMillis() > _deadlineMs) {
+      _errorBlock = onTimeout();
+      return _errorBlock;
+    }
+
+    E block = readBlockOrNull();
+    while (block != null && isEos(block) && !_mailboxes.isEmpty()) {

Review Comment:
   Why? We remove mailbox within the loop



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289833588


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java:
##########
@@ -188,4 +193,10 @@ private boolean isType(MetadataBlock.MetadataBlockType type) {
     MetadataBlock metadata = (MetadataBlock) _dataBlock;
     return metadata.getType() == type;
   }
+
+  @Override
+  public String toString() {
+    String blockType = isErrorBlock() ? "error" : isSuccessfulEndOfStreamBlock() ? "eos" : "data";
+    return "TransferableBlock{blockType=" + blockType + ", _numRows=" + _numRows + ", _container=" + _container + '}';

Review Comment:
   Removed in 55d9513640904c5651be88b06bc7a99db5a57c62



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289886813


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java:
##########
@@ -61,6 +61,9 @@ public GrpcSendingMailbox(String id, ChannelManager channelManager, String hostn
   @Override
   public void send(TransferableBlock block)
       throws IOException {
+    if (LOGGER.isDebugEnabled()) {
+      LOGGER.debug("==[GRPC SEND]== sending data to: " + _id);

Review Comment:
   There are several ways to do the logging. Usually the `{}` is the best in terms of readability vs performance. But in terms of performance, what I'm doing here is better. Specifically, order in performance (from higher to lower) is:
   1. add if ward and use + in the log. We don't use this option that much because it is the worse in terms of readibility
   2. use `{}`. This requires to call a `LOGGER.debug(String, Object...)` method, so it creates an array. In case `debug` is disabled, the array is not used, but it is very probable that the JIT will detect that and do not allocate the array. In that case the performance of this option should be as good as 1, but it is not guaranteed that the JIT will be smart enough. In case debug is enabled, the array will always be created and therefore performance will be better in 1.
   4. use concatenation everywhere. This is the worse if debug is disabled because we need to call toString in all objects and we need to do the concatenation... just to discover that we don't actually need the String. JIT may get rid of this, but seems very unlikely given how large the code may be and the implications toString may have.
   
   We can also add the if and use `{}` inside. It would probably be faster than 2, but not so much and legibility won't be great.
   
   In this case @walterddr was worried about the performance implications of the extra logs he added, so I tried to use the solution that give us more performance in order to reduce that impact to the minimum



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289962057


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperatorTest.java:
##########
@@ -139,21 +138,11 @@ public void shouldTimeoutOnExtraLongSleep()
       MetadataBlock errorBlock = (MetadataBlock) mailbox.getDataBlock();
       assertTrue(errorBlock.getExceptions().containsKey(QueryException.EXECUTION_TIMEOUT_ERROR_CODE));
     }
-
-    // Longer timeout or default timeout (10s) doesn't result in timeout
-    context =
-        OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, System.currentTimeMillis() + 10_000L,
-            _stageMetadata1);
-    try (MailboxReceiveOperator receiveOp = new MailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, 1)) {
-      Thread.sleep(100L);
-      TransferableBlock mailbox = receiveOp.nextBlock();

Review Comment:
   @Jackie-Jiang I guess we should just remove this tests?



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on PR #11205:
URL: https://github.com/apache/pinot/pull/11205#issuecomment-1655998110

   operator tests are almost certainly required to be entirely re-written due to the new behavior without no-op. i would say let's focus on getting the resource/query runner tests passing first 


-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277779616


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/TransformOperator.java:
##########
@@ -86,13 +86,14 @@ protected TransferableBlock getNextBlock() {
     try {
       TransferableBlock block = _upstreamOperator.nextBlock();
       return transform(block);
-    } catch (Exception e) {
+    } catch (RuntimeException e) {
       return TransferableBlockUtils.getErrorTransferableBlock(e);
     }
   }
 
-  private TransferableBlock transform(TransferableBlock block)
-      throws Exception {
+  private TransferableBlock transform(TransferableBlock block) {
+    // TODO: Other operators keep the first erroneous block, while this keep the last.
+    //  We should decide what is what we want to do and be consistent with that.

Review Comment:
   Discussion opened to do not forget about this case



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1280753470


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -51,16 +52,30 @@ public class ReceivingMailbox {
   // TODO: Revisit if this is the correct way to apply back pressure
   private final BlockingQueue<TransferableBlock> _blocks = new ArrayBlockingQueue<>(DEFAULT_MAX_PENDING_BLOCKS);
   private final AtomicReference<TransferableBlock> _errorBlock = new AtomicReference<>();
+  @Nullable
+  private volatile Reader _reader;

Review Comment:
   if we have the reader API. do we still need the callback registration? (we can remove if this is fully covered by the reader callback



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1283274341


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. for leaf stage this cannot be a dangling opchain b/c they will eventually be cleared up
+    // via query timeout.
+    List<OpChainId> opChainIdsToCancel = _submittedOpChainMap.keySet()

Review Comment:
   yes we need to clean up the map i think i commented in https://github.com/walterddr/pinot/pull/58#discussion_r1189345501



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1280818049


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -51,16 +52,30 @@ public class ReceivingMailbox {
   // TODO: Revisit if this is the correct way to apply back pressure
   private final BlockingQueue<TransferableBlock> _blocks = new ArrayBlockingQueue<>(DEFAULT_MAX_PENDING_BLOCKS);
   private final AtomicReference<TransferableBlock> _errorBlock = new AtomicReference<>();
+  @Nullable
+  private volatile Reader _reader;
 
   public ReceivingMailbox(String id, Consumer<OpChainId> receiveMailCallback) {
     _id = id;
     _receiveMailCallback = receiveMailCallback;
   }
 
+  public void registeredReader(Reader reader) {
+    if (_reader != null) {
+      throw new IllegalArgumentException("Only one reader is supported");
+    }
+    LOGGER.debug("==[MAILBOX]== Reader registered for mailbox {}", _id);

Review Comment:
   should we clean up these debug msgs? otherwise the string eval still happens right?



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on PR #11205:
URL: https://github.com/apache/pinot/pull/11205#issuecomment-1683963385

   I've decided to follow the approach suggested by @Jackie-Jiang on MailboxSendOperator. Now when always fail if the buffer is full. But that breaks the following tests, which verifies that EOS is sent even when the buffer size is null:
   
   ```
   Error:    MailboxSendOperatorTest.shouldSendDataBlock:159 expected EOS block to propagate next even if capacity is now 0. expected [TransferableBlock{blockType=eos, _numRows=0}] but found [TransferableBlock{blockType=error, _numRows=0}]
   ```
   
   What do we want to do in this case?


-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1295560610


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java:
##########
@@ -0,0 +1,246 @@
+/**
+ * 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.utils;
+
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BlockingMultiStreamConsumer<E> implements AutoCloseable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlockingMultiStreamConsumer.class);
+  private final Object _id;
+  protected final List<? extends AsyncStream<E>> _mailboxes;
+  protected final ArrayBlockingQueue<Boolean> _newDataReady = new ArrayBlockingQueue<>(1);
+  private final long _deadlineMs;
+  /**
+   * An index that used to calculate where do we are going to start reading.
+   * The invariant is that we are always going to start reading from {@code _lastRead + 1}.
+   * Therefore {@link #_lastRead} must be in the range {@code [-1, mailbox.size() - 1]}
+   */
+  protected int _lastRead;
+  private E _errorBlock = null;
+
+  public BlockingMultiStreamConsumer(Object id, long deadlineMs, List<? extends AsyncStream<E>> asyncProducers) {
+    _id = id;
+    _deadlineMs = deadlineMs;
+    AsyncStream.OnNewData onNewData = this::onData;
+    _mailboxes = asyncProducers;
+    _mailboxes.forEach(blockProducer -> blockProducer.addOnNewDataListener(onNewData));
+    _lastRead = _mailboxes.size() - 1;
+  }
+
+  protected abstract boolean isError(E element);
+
+  protected abstract boolean isEos(E element);
+
+  protected abstract E onTimeout();
+
+  protected abstract E onException(Exception e);
+
+  protected abstract E onEos();
+
+  @Override
+  public void close() {
+    cancelRemainingMailboxes();
+  }
+
+  public void cancel(Throwable t) {
+    cancelRemainingMailboxes();
+  }
+
+  protected void cancelRemainingMailboxes() {
+    for (AsyncStream<E> mailbox : _mailboxes) {
+      mailbox.cancel();
+    }
+  }
+
+  public void onData() {
+    if (_newDataReady.offer(Boolean.TRUE)) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("New data notification delivered on " + _id + ". " + System.identityHashCode(_newDataReady));
+      }
+    } else if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("New data notification ignored on " + _id + ". " + System.identityHashCode(_newDataReady));
+    }
+  }
+
+  /**
+   * Reads the next block for any ready mailbox or blocks until some of them is ready.
+   *
+   * The method implements a sequential read semantic. Meaning that:
+   * <ol>
+   *   <li>EOS is only returned when all mailboxes already emitted EOS or there are no mailboxes</li>
+   *   <li>If an error is read from a mailbox, the error is returned</li>
+   *   <li>If data is read from a mailbox, that data block is returned</li>
+   *   <li>If no mailbox is ready, the calling thread is blocked</li>
+   * </ol>
+   *
+   * Right now the implementation tries to be fair. If one call returned the block from mailbox {@code i}, then next
+   * call will look for mailbox {@code i+1}, {@code i+2}... in a circular manner.
+   *
+   * In order to unblock a thread blocked here, {@link #onData()} should be called.   *
+   */
+  public E readBlockBlocking() {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
+    }
+    // Standard optimistic execution. First we try to read without acquiring the lock.
+    E block = readDroppingSuccessEos();
+    long timeoutMs = _deadlineMs - System.currentTimeMillis();
+    boolean timeout = false;
+    try {
+      while (block == null) { // we didn't find a mailbox ready to read, so we need to be pessimistic
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.debug("==[RECEIVE]== Blocked on : " + _id + ". " + System.identityHashCode(_newDataReady));
+        }
+        timeout = _newDataReady.poll(timeoutMs, TimeUnit.MILLISECONDS) == null;
+        LOGGER.debug("==[RECEIVE]== More data available. Trying to read again");
+        block = readDroppingSuccessEos();
+      }
+
+      if (timeout) {
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.warn("==[RECEIVE]== Timeout on: " + _id);
+        }
+        _errorBlock = onTimeout();
+        return _errorBlock;
+      } else if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("==[RECEIVE]== Ready to emit on: " + _id);
+      }
+    } catch (InterruptedException ex) {
+      return onException(ex);
+    }
+    return block;
+  }
+
+  /**
+   * This is a utility method that reads tries to read from the different mailboxes in a circular manner.
+   *
+   * The method is a bit more complex than expected because ir order to simplify {@link #readBlockBlocking} we added
+   * some extra logic here. For example, this method checks for timeouts, add some logs, releases mailboxes that emitted
+   * EOS and in case an error block is found, stores it.
+   *
+   * @return the new block to consume or null if none is found. EOS is only emitted when all mailboxes already emitted
+   * EOS.
+   */
+  @Nullable
+  private E readDroppingSuccessEos() {
+    if (System.currentTimeMillis() > _deadlineMs) {
+      _errorBlock = onTimeout();
+      return _errorBlock;
+    }
+
+    E block = readBlockOrNull();
+    while (block != null && isEos(block) && !_mailboxes.isEmpty()) {

Review Comment:
   You are right. We only return EOS if there is an EOS in some mailbox, which implies there has to be at least one mailbox



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on PR #11205:
URL: https://github.com/apache/pinot/pull/11205#issuecomment-1655552099

   Important: Current commit does not support pipeline breaker. We are going to work on that during the next days. We don't expect to merge this PR before pipeline breaker is adapted to this new model.
   
   Most tests in pinot-query-runtime are working. I wasn't able to fix all tests in QueryRunnerTest. Some of them seem to relay on pipeline breaker, but there may be other bugs. For example, `OpChain._opChainFinishCallback` was added after made his fork, so I have to add changes in order to make these tests pass. I guess there may be more cases like that.
   
   Also, I've changed the way receive mailbox operator blocks when there all mailboxes are empty. Instead of waiting on a structure maintained by the opchain scheduler, now there is a direct lock between mailboxes and the operator. I decided to store that callback in MailboxService, which seems less intrusive than the scheduler, but we can debate about better options.


-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277737643


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java:
##########
@@ -140,14 +144,14 @@ public void processQuery(DistributedStagePlan distributedStagePlan, Map<String,
     long deadlineMs = System.currentTimeMillis() + timeoutMs;
 
     // run pre-stage execution for all pipeline breakers
-    PipelineBreakerResult pipelineBreakerResult = PipelineBreakerExecutor.executePipelineBreakers(_scheduler,
-        _mailboxService, distributedStagePlan, deadlineMs, requestId, isTraceEnabled);
+//    PipelineBreakerResult pipelineBreakerResult = PipelineBreakerExecutor.executePipelineBreakers(_scheduler,

Review Comment:
   This is a TODO



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277744002


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java:
##########
@@ -159,12 +155,16 @@ public String toExplainString() {
   @Override
   protected TransferableBlock getNextBlock() {
     try {
-      if (!_readyToConstruct && !consumeInputBlocks()) {
-        return TransferableBlockUtils.getNoOpTransferableBlock();
+      TransferableBlock finalBlock;
+      if (_isGroupByAggregation) {
+        finalBlock = consumeGroupBy();

Review Comment:
   This is a lateral change, not completely necessary, but helps readibility, removes a condition inside the loom and makes it easier the JIT compilation



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281500827


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);

Review Comment:
   Done in 2e4bde



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277738789


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java:
##########
@@ -167,8 +167,13 @@ public boolean isSuccessfulEndOfStreamBlock() {
   /**
    * @return whether this block represents a NOOP block
    */
+  @Deprecated(forRemoval = true)
   public boolean isNoOpBlock() {
-    return isType(MetadataBlock.MetadataBlockType.NOOP);
+    return false;

Review Comment:
   This code won't compile in Java 8 (`Deprecated.forRemoval` was added in Java 8), but we should remove this method before merging this.
   
   The reason to keep it is because there are a couple of test that still use it.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1279601149


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/SchedulerService.java:
##########
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.executor;
+
+import org.apache.pinot.query.runtime.operator.OpChain;
+
+
+public interface SchedulerService {

Review Comment:
    OpChainScheduler interface can be deleted after this 



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -51,16 +51,30 @@ public class ReceivingMailbox {
   // TODO: Revisit if this is the correct way to apply back pressure
   private final BlockingQueue<TransferableBlock> _blocks = new ArrayBlockingQueue<>(DEFAULT_MAX_PENDING_BLOCKS);
   private final AtomicReference<TransferableBlock> _errorBlock = new AtomicReference<>();
+  @Nullable
+  private volatile Reader _reader;
 
   public ReceivingMailbox(String id, Consumer<OpChainId> receiveMailCallback) {
     _id = id;
     _receiveMailCallback = receiveMailCallback;
   }
 
+  public void registeredReader(Reader reader) {

Review Comment:
   can we decouple this from the no-op removal? it seems like there's race conditions that hasn't been resolved in tests 



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1283154051


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {

Review Comment:
   This code came from @walterddr's PR, so TBH I don't have the context to answer the question.
   
   Anyway, is this a blocking issue? The plan was to have this merged before the end of the week. Can we do that with this cancellation mechanism or should we try to find another way?



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1291096823


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java:
##########
@@ -203,84 +143,72 @@ public void shouldCallCancelOnOperatorsThatReturnErrorBlock()
       return null;
     }).when(_operatorA).cancel(Mockito.any());
 
-    schedulerService.startAsync().awaitRunning();
     schedulerService.register(opChain);
 
     Assert.assertTrue(latch.await(10, TimeUnit.SECONDS), "expected await to be called in less than 10 seconds");
-    schedulerService.stopAsync().awaitTerminated();
   }
 
-  @Test
-  public void shouldCallCancelOnOpChainsWhenItIsCancelledByDispatch()
-      throws InterruptedException {
-    initExecutor(1);
-    OpChain opChain = getChain(_operatorA);
-    Mockito.when(_scheduler.next(Mockito.anyLong(), Mockito.any())).thenAnswer((Answer<OpChain>) invocation -> {
-      Thread.sleep(100);
-      return opChain;
-    });
-    OpChainSchedulerService schedulerService = new OpChainSchedulerService(_scheduler, _executor);
-
-    Mockito.when(_operatorA.nextBlock()).thenReturn(TransferableBlockUtils.getNoOpTransferableBlock());
-
-    CountDownLatch cancelLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      cancelLatch.countDown();
-      return null;
-    }).when(_operatorA).cancel(Mockito.any());
-    CountDownLatch deregisterLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      deregisterLatch.countDown();
-      return null;
-    }).when(_scheduler).deregister(Mockito.same(opChain));
-    CountDownLatch awaitLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      awaitLatch.countDown();
-      return null;
-    }).when(_scheduler).yield(Mockito.any());
-
-    schedulerService.startAsync().awaitRunning();
-    schedulerService.register(opChain);
-
-    Assert.assertTrue(awaitLatch.await(10, TimeUnit.SECONDS), "expected await to be called in less than 10 seconds");
-
-    // now cancel the request.
-    schedulerService.cancel(123);
-
-    Assert.assertTrue(cancelLatch.await(10, TimeUnit.SECONDS), "expected OpChain to be cancelled");
-    Assert.assertTrue(deregisterLatch.await(10, TimeUnit.SECONDS), "expected OpChain to be deregistered");
-    Mockito.verify(_operatorA, Mockito.times(1)).cancel(Mockito.any());
-    Mockito.verify(_scheduler, Mockito.times(1)).deregister(Mockito.any());
-    schedulerService.stopAsync().awaitTerminated();
-  }
+//  @Test
+//  public void shouldCallCancelOnOpChainsWhenItIsCancelledByDispatch()

Review Comment:
   Test modified to test what it say (that op chains can be cancelled) without using noop



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289862143


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java:
##########
@@ -0,0 +1,246 @@
+/**
+ * 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.utils;
+
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BlockingMultiStreamConsumer<E> implements AutoCloseable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlockingMultiStreamConsumer.class);
+  private final Object _id;
+  protected final List<? extends AsyncStream<E>> _mailboxes;
+  protected final ArrayBlockingQueue<Boolean> _newDataReady = new ArrayBlockingQueue<>(1);
+  private final long _deadlineMs;
+  /**
+   * An index that used to calculate where do we are going to start reading.
+   * The invariant is that we are always going to start reading from {@code _lastRead + 1}.
+   * Therefore {@link #_lastRead} must be in the range {@code [-1, mailbox.size() - 1]}
+   */
+  protected int _lastRead;
+  private E _errorBlock = null;
+
+  public BlockingMultiStreamConsumer(Object id, long deadlineMs, List<? extends AsyncStream<E>> asyncProducers) {
+    _id = id;
+    _deadlineMs = deadlineMs;
+    AsyncStream.OnNewData onNewData = this::onData;
+    _mailboxes = asyncProducers;
+    _mailboxes.forEach(blockProducer -> blockProducer.addOnNewDataListener(onNewData));
+    _lastRead = _mailboxes.size() - 1;
+  }
+
+  protected abstract boolean isError(E element);
+
+  protected abstract boolean isEos(E element);
+
+  protected abstract E onTimeout();
+
+  protected abstract E onException(Exception e);
+
+  protected abstract E onEos();
+
+  @Override
+  public void close() {
+    cancelRemainingMailboxes();
+  }
+
+  public void cancel(Throwable t) {
+    cancelRemainingMailboxes();
+  }
+
+  protected void cancelRemainingMailboxes() {
+    for (AsyncStream<E> mailbox : _mailboxes) {
+      mailbox.cancel();
+    }
+  }
+
+  public void onData() {
+    if (_newDataReady.offer(Boolean.TRUE)) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("New data notification delivered on " + _id + ". " + System.identityHashCode(_newDataReady));
+      }
+    } else if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("New data notification ignored on " + _id + ". " + System.identityHashCode(_newDataReady));
+    }
+  }
+
+  /**
+   * Reads the next block for any ready mailbox or blocks until some of them is ready.
+   *
+   * The method implements a sequential read semantic. Meaning that:
+   * <ol>
+   *   <li>EOS is only returned when all mailboxes already emitted EOS or there are no mailboxes</li>
+   *   <li>If an error is read from a mailbox, the error is returned</li>
+   *   <li>If data is read from a mailbox, that data block is returned</li>
+   *   <li>If no mailbox is ready, the calling thread is blocked</li>
+   * </ol>
+   *
+   * Right now the implementation tries to be fair. If one call returned the block from mailbox {@code i}, then next
+   * call will look for mailbox {@code i+1}, {@code i+2}... in a circular manner.
+   *
+   * In order to unblock a thread blocked here, {@link #onData()} should be called.   *
+   */
+  public E readBlockBlocking() {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
+    }
+    // Standard optimistic execution. First we try to read without acquiring the lock.
+    E block = readDroppingSuccessEos();
+    long timeoutMs = _deadlineMs - System.currentTimeMillis();
+    boolean timeout = false;
+    try {
+      while (block == null) { // we didn't find a mailbox ready to read, so we need to be pessimistic
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.debug("==[RECEIVE]== Blocked on : " + _id + ". " + System.identityHashCode(_newDataReady));
+        }
+        timeout = _newDataReady.poll(timeoutMs, TimeUnit.MILLISECONDS) == null;
+        LOGGER.debug("==[RECEIVE]== More data available. Trying to read again");
+        block = readDroppingSuccessEos();
+      }
+
+      if (timeout) {
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.warn("==[RECEIVE]== Timeout on: " + _id);
+        }
+        _errorBlock = onTimeout();
+        return _errorBlock;
+      } else if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("==[RECEIVE]== Ready to emit on: " + _id);
+      }
+    } catch (InterruptedException ex) {
+      return onException(ex);
+    }
+    return block;
+  }
+
+  /**
+   * This is a utility method that reads tries to read from the different mailboxes in a circular manner.
+   *
+   * The method is a bit more complex than expected because ir order to simplify {@link #readBlockBlocking} we added
+   * some extra logic here. For example, this method checks for timeouts, add some logs, releases mailboxes that emitted
+   * EOS and in case an error block is found, stores it.
+   *
+   * @return the new block to consume or null if none is found. EOS is only emitted when all mailboxes already emitted
+   * EOS.
+   */
+  @Nullable
+  private E readDroppingSuccessEos() {
+    if (System.currentTimeMillis() > _deadlineMs) {
+      _errorBlock = onTimeout();
+      return _errorBlock;
+    }
+
+    E block = readBlockOrNull();
+    while (block != null && isEos(block) && !_mailboxes.isEmpty()) {

Review Comment:
   Yep, when we read the last EOS from the last mailbox.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1291097043


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java:
##########
@@ -203,84 +143,72 @@ public void shouldCallCancelOnOperatorsThatReturnErrorBlock()
       return null;
     }).when(_operatorA).cancel(Mockito.any());
 
-    schedulerService.startAsync().awaitRunning();
     schedulerService.register(opChain);
 
     Assert.assertTrue(latch.await(10, TimeUnit.SECONDS), "expected await to be called in less than 10 seconds");
-    schedulerService.stopAsync().awaitTerminated();
   }
 
-  @Test
-  public void shouldCallCancelOnOpChainsWhenItIsCancelledByDispatch()
-      throws InterruptedException {
-    initExecutor(1);
-    OpChain opChain = getChain(_operatorA);
-    Mockito.when(_scheduler.next(Mockito.anyLong(), Mockito.any())).thenAnswer((Answer<OpChain>) invocation -> {
-      Thread.sleep(100);
-      return opChain;
-    });
-    OpChainSchedulerService schedulerService = new OpChainSchedulerService(_scheduler, _executor);
-
-    Mockito.when(_operatorA.nextBlock()).thenReturn(TransferableBlockUtils.getNoOpTransferableBlock());
-
-    CountDownLatch cancelLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      cancelLatch.countDown();
-      return null;
-    }).when(_operatorA).cancel(Mockito.any());
-    CountDownLatch deregisterLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      deregisterLatch.countDown();
-      return null;
-    }).when(_scheduler).deregister(Mockito.same(opChain));
-    CountDownLatch awaitLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      awaitLatch.countDown();
-      return null;
-    }).when(_scheduler).yield(Mockito.any());
-
-    schedulerService.startAsync().awaitRunning();
-    schedulerService.register(opChain);
-
-    Assert.assertTrue(awaitLatch.await(10, TimeUnit.SECONDS), "expected await to be called in less than 10 seconds");
-
-    // now cancel the request.
-    schedulerService.cancel(123);
-
-    Assert.assertTrue(cancelLatch.await(10, TimeUnit.SECONDS), "expected OpChain to be cancelled");
-    Assert.assertTrue(deregisterLatch.await(10, TimeUnit.SECONDS), "expected OpChain to be deregistered");
-    Mockito.verify(_operatorA, Mockito.times(1)).cancel(Mockito.any());
-    Mockito.verify(_scheduler, Mockito.times(1)).deregister(Mockito.any());
-    schedulerService.stopAsync().awaitTerminated();
-  }
+//  @Test
+//  public void shouldCallCancelOnOpChainsWhenItIsCancelledByDispatch()

Review Comment:
   314c332384161987de632425f2e2e6763097c40b



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289833588


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java:
##########
@@ -188,4 +193,10 @@ private boolean isType(MetadataBlock.MetadataBlockType type) {
     MetadataBlock metadata = (MetadataBlock) _dataBlock;
     return metadata.getType() == type;
   }
+
+  @Override
+  public String toString() {
+    String blockType = isErrorBlock() ? "error" : isSuccessfulEndOfStreamBlock() ? "eos" : "data";
+    return "TransferableBlock{blockType=" + blockType + ", _numRows=" + _numRows + ", _container=" + _container + '}';

Review Comment:
   Removed in 55d9513640904c5651be88b06bc7a99db5a57c62 



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281493383


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. for leaf stage this cannot be a dangling opchain b/c they will eventually be cleared up
+    // via query timeout.
+    List<OpChainId> opChainIdsToCancel = _submittedOpChainMap.keySet()
+        .stream().filter(opChainId -> opChainId.getRequestId() == requestId).collect(Collectors.toList());
+    for (OpChainId opChainId : opChainIdsToCancel) {
+      Future<?> future = _submittedOpChainMap.get(opChainId);
+      if (future != null) {
+        future.cancel(true);
+      }
+    }
   }
 
   private void closeOpChain(OpChain opChain) {
-    try {
       opChain.close();
-    } finally {
-      _scheduler.deregister(opChain);
-    }
   }
 
-  private void cancelOpChain(OpChain opChain, Throwable e) {
-    try {
-      opChain.cancel(e);
-    } finally {
-      _scheduler.deregister(opChain);
-    }
+  private void cancelOpChain(OpChain opChain, @Nullable Throwable t) {

Review Comment:
   Fixed in fcf8bc49d36f005a7170933a52a5bfcbb28ff933



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281500827


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);

Review Comment:
   Done in 2e4bde2031b0b7ccf694bb4d8927734b4579681e



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1280081624


##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java:
##########
@@ -115,13 +116,14 @@ public MultiStageBrokerRequestHandler(PinotConfiguration config, String brokerId
 
     long releaseMs = config.getProperty(QueryConfig.KEY_OF_SCHEDULER_RELEASE_TIMEOUT_MS,
         QueryConfig.DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS);
-    _reducerScheduler = new OpChainSchedulerService(new RoundRobinScheduler(releaseMs),
+    _reducerScheduler = new OpChainSchedulerService(
         Executors.newCachedThreadPool(new NamedThreadFactory("query_broker_reducer_" + _reducerPort + "_port")));
-    _mailboxService = new MailboxService(_reducerHostname, _reducerPort, config, _reducerScheduler::onDataAvailable);
+    _mailboxService = new MailboxService(_reducerHostname, _reducerPort, config);
 
     // TODO: move this to a startUp() function.
-    _reducerScheduler.startAsync();
     _mailboxService.start();
+    //TODO: make this configurable
+    _opChainExecutor = new OpChainExecutor(new NamedThreadFactory("op_chain_worker_on_" + _reducerPort + "_port"));

Review Comment:
   shoudl also set this opChainExecutor to the _reducerScheduler in line 120



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281493383


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. for leaf stage this cannot be a dangling opchain b/c they will eventually be cleared up
+    // via query timeout.
+    List<OpChainId> opChainIdsToCancel = _submittedOpChainMap.keySet()
+        .stream().filter(opChainId -> opChainId.getRequestId() == requestId).collect(Collectors.toList());
+    for (OpChainId opChainId : opChainIdsToCancel) {
+      Future<?> future = _submittedOpChainMap.get(opChainId);
+      if (future != null) {
+        future.cancel(true);
+      }
+    }
   }
 
   private void closeOpChain(OpChain opChain) {
-    try {
       opChain.close();
-    } finally {
-      _scheduler.deregister(opChain);
-    }
   }
 
-  private void cancelOpChain(OpChain opChain, Throwable e) {
-    try {
-      opChain.cancel(e);
-    } finally {
-      _scheduler.deregister(opChain);
-    }
+  private void cancelOpChain(OpChain opChain, @Nullable Throwable t) {

Review Comment:
   Fixed in fcf8bc4



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. for leaf stage this cannot be a dangling opchain b/c they will eventually be cleared up
+    // via query timeout.
+    List<OpChainId> opChainIdsToCancel = _submittedOpChainMap.keySet()

Review Comment:
   > We also need to remove the entry because there is no guarantee that the future already starts executing
   
   In fact the entry was never removed, even if the future was executed, right?



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);

Review Comment:
   Applied on 9031c6



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/OpChainExecutor.java:
##########
@@ -0,0 +1,129 @@
+/**
+ * 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;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class OpChainExecutor implements ExecutorService, AutoCloseable {

Review Comment:
   Removed on  306edff



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {

Review Comment:
   Should be better in d34f90



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. for leaf stage this cannot be a dangling opchain b/c they will eventually be cleared up
+    // via query timeout.
+    List<OpChainId> opChainIdsToCancel = _submittedOpChainMap.keySet()

Review Comment:
   I've changed the code, but it is not clear to me the requirements of this method.
   
   It is also not clear to me whether we can use `entrySet().iterator()` because we are going to be modifying the map to remove entries from the iterator while other threads will be adding elements. By the specification, the behavior there is undefined. Specifically, javadoc says we can remove elements from the entrySet, but doesn't specify what happens when other threads modify the map while iterating.
   
   Given that this method is not going to be executed that often and the list shouldn't be that large, I don't think it is worth to enter in unspecified territory in order to do not create a short living array list.
   



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java:
##########
@@ -115,12 +115,12 @@ public MultiStageBrokerRequestHandler(PinotConfiguration config, String brokerId
 
     long releaseMs = config.getProperty(QueryConfig.KEY_OF_SCHEDULER_RELEASE_TIMEOUT_MS,
         QueryConfig.DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS);
-    _reducerScheduler = new OpChainSchedulerService(new RoundRobinScheduler(releaseMs),
-        Executors.newCachedThreadPool(new NamedThreadFactory("query_broker_reducer_" + _reducerPort + "_port")));
-    _mailboxService = new MailboxService(_reducerHostname, _reducerPort, config, _reducerScheduler::onDataAvailable);
+    //TODO: make this configurable
+    _opChainExecutor = new OpChainExecutor(new NamedThreadFactory("op_chain_worker_on_" + _reducerPort + "_port"));

Review Comment:
   Fixed on  306edff



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289961773


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java:
##########
@@ -203,84 +143,72 @@ public void shouldCallCancelOnOperatorsThatReturnErrorBlock()
       return null;
     }).when(_operatorA).cancel(Mockito.any());
 
-    schedulerService.startAsync().awaitRunning();
     schedulerService.register(opChain);
 
     Assert.assertTrue(latch.await(10, TimeUnit.SECONDS), "expected await to be called in less than 10 seconds");
-    schedulerService.stopAsync().awaitTerminated();
   }
 
-  @Test
-  public void shouldCallCancelOnOpChainsWhenItIsCancelledByDispatch()
-      throws InterruptedException {
-    initExecutor(1);
-    OpChain opChain = getChain(_operatorA);
-    Mockito.when(_scheduler.next(Mockito.anyLong(), Mockito.any())).thenAnswer((Answer<OpChain>) invocation -> {
-      Thread.sleep(100);
-      return opChain;
-    });
-    OpChainSchedulerService schedulerService = new OpChainSchedulerService(_scheduler, _executor);
-
-    Mockito.when(_operatorA.nextBlock()).thenReturn(TransferableBlockUtils.getNoOpTransferableBlock());
-
-    CountDownLatch cancelLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      cancelLatch.countDown();
-      return null;
-    }).when(_operatorA).cancel(Mockito.any());
-    CountDownLatch deregisterLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      deregisterLatch.countDown();
-      return null;
-    }).when(_scheduler).deregister(Mockito.same(opChain));
-    CountDownLatch awaitLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      awaitLatch.countDown();
-      return null;
-    }).when(_scheduler).yield(Mockito.any());
-
-    schedulerService.startAsync().awaitRunning();
-    schedulerService.register(opChain);
-
-    Assert.assertTrue(awaitLatch.await(10, TimeUnit.SECONDS), "expected await to be called in less than 10 seconds");
-
-    // now cancel the request.
-    schedulerService.cancel(123);
-
-    Assert.assertTrue(cancelLatch.await(10, TimeUnit.SECONDS), "expected OpChain to be cancelled");
-    Assert.assertTrue(deregisterLatch.await(10, TimeUnit.SECONDS), "expected OpChain to be deregistered");
-    Mockito.verify(_operatorA, Mockito.times(1)).cancel(Mockito.any());
-    Mockito.verify(_scheduler, Mockito.times(1)).deregister(Mockito.any());
-    schedulerService.stopAsync().awaitTerminated();
-  }
+//  @Test
+//  public void shouldCallCancelOnOpChainsWhenItIsCancelledByDispatch()

Review Comment:
   What do you think we should do there?



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1291103612


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java:
##########
@@ -173,30 +165,32 @@ public void cancel(long requestId) {
 
   @VisibleForTesting
   public ExecutorService getQueryWorkerLeafExecutorService() {
-    return _queryWorkerLeafExecutorService;
+    return _opChainExecutor;
   }
 
   @VisibleForTesting
   public ExecutorService getQueryWorkerIntermExecutorService() {
-    return _queryWorkerIntermExecutorService;
+    return _opChainExecutor;
   }

Review Comment:
   Done



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289865538


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/TransformOperator.java:
##########
@@ -86,13 +86,14 @@ protected TransferableBlock getNextBlock() {
     try {
       TransferableBlock block = _upstreamOperator.nextBlock();
       return transform(block);
-    } catch (Exception e) {
+    } catch (RuntimeException e) {

Review Comment:
   I don't actually know what we want or should do here. I'm keeping the old behavior but being more specific.
   
   Previous code was catching `Exception` while no checked exception was thrown here. So I assumed that even older code was throwing a checked exception. Now that only runtime exceptions are thrown I think it is better to explicitly only catch `RuntimeException`s.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289819281


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/ExecutorServiceUtils.java:
##########
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.executor;
+
+import java.util.List;
+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.spi.env.PinotConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ExecutorServiceUtils {
+  private static final Logger LOGGER = LoggerFactory.getLogger(ExecutorServiceUtils.class);
+
+  private ExecutorServiceUtils() {
+  }
+
+  public static ExecutorService createDefault(String baseName) {
+    return Executors.newCachedThreadPool(new NamedThreadFactory(baseName));
+  }
+
+  public static ExecutorService create(PinotConfiguration conf, String confPrefix, String baseName) {
+    //TODO: make this configurable
+    return Executors.newCachedThreadPool(new NamedThreadFactory(baseName));
+  }

Review Comment:
   Here you have an example on how to make executors customizable: https://github.com/gortiz/pinot/pull/6
   
   We can merge it here if you want, although it may be better to do that in another PR to keep this one simpler



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277768278


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java:
##########
@@ -46,41 +42,6 @@ public String toExplainString() {
 
   @Override
   protected TransferableBlock getNextBlock() {
-    if (_errorBlock != null) {
-      return _errorBlock;
-    }
-    if (System.currentTimeMillis() > _context.getDeadlineMs()) {
-      _errorBlock = TransferableBlockUtils.getErrorTransferableBlock(QueryException.EXECUTION_TIMEOUT_ERROR);
-      return _errorBlock;
-    }
-
-    // Poll from every mailbox in round-robin fashion:
-    // - Return the first content block
-    // - If no content block found but there are mailboxes not finished, return no-op block
-    // - If all content blocks are already returned, return end-of-stream block
-    int numMailboxes = _mailboxes.size();
-    for (int i = 0; i < numMailboxes; i++) {
-      ReceivingMailbox mailbox = _mailboxes.remove();
-      TransferableBlock block = mailbox.poll();
-
-      // Release the mailbox when the block is end-of-stream
-      if (block != null && block.isSuccessfulEndOfStreamBlock()) {
-        _mailboxService.releaseReceivingMailbox(mailbox);
-        _opChainStats.getOperatorStatsMap().putAll(block.getResultMetadata());
-        continue;
-      }
-
-      // Add the mailbox back to the queue if the block is not end-of-stream
-      _mailboxes.add(mailbox);
-      if (block != null) {
-        if (block.isErrorBlock()) {
-          _errorBlock = block;
-        }
-        return block;
-      }
-    }
-
-    return _mailboxes.isEmpty() ? TransferableBlockUtils.getEndOfStreamTransferableBlock()
-        : TransferableBlockUtils.getNoOpTransferableBlock();
+    return readBlockBlocking();

Review Comment:
   Now the logic is the same implemented in the parent method. I decided to do not implement `getNextBlock` directly in `BaseReceiveOperator` just in case we add some extra receive operator later. Also, I think the sorted is clearer calling `readBlockBlocking` instead of `super.getNextBlock()`



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277778428


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SetOperator.java:
##########
@@ -112,15 +112,14 @@ protected void constructRightBlockSet() {
 
   protected TransferableBlock constructResultBlockSet(TransferableBlock leftBlock) {
     List<Object[]> rows = new ArrayList<>();
+    // TODO: Other operators keep the first erroneous block, while this keep the last.
+    //  We should decide what is what we want to do and be consistent with that.

Review Comment:
   I'm not 100% sure about this, but we should decide what to do in these cases



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277780780


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java:
##########
@@ -203,84 +143,72 @@ public void shouldCallCancelOnOperatorsThatReturnErrorBlock()
       return null;
     }).when(_operatorA).cancel(Mockito.any());
 
-    schedulerService.startAsync().awaitRunning();
     schedulerService.register(opChain);
 
     Assert.assertTrue(latch.await(10, TimeUnit.SECONDS), "expected await to be called in less than 10 seconds");
-    schedulerService.stopAsync().awaitTerminated();
   }
 
-  @Test
-  public void shouldCallCancelOnOpChainsWhenItIsCancelledByDispatch()
-      throws InterruptedException {
-    initExecutor(1);
-    OpChain opChain = getChain(_operatorA);
-    Mockito.when(_scheduler.next(Mockito.anyLong(), Mockito.any())).thenAnswer((Answer<OpChain>) invocation -> {
-      Thread.sleep(100);
-      return opChain;
-    });
-    OpChainSchedulerService schedulerService = new OpChainSchedulerService(_scheduler, _executor);
-
-    Mockito.when(_operatorA.nextBlock()).thenReturn(TransferableBlockUtils.getNoOpTransferableBlock());
-
-    CountDownLatch cancelLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      cancelLatch.countDown();
-      return null;
-    }).when(_operatorA).cancel(Mockito.any());
-    CountDownLatch deregisterLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      deregisterLatch.countDown();
-      return null;
-    }).when(_scheduler).deregister(Mockito.same(opChain));
-    CountDownLatch awaitLatch = new CountDownLatch(1);
-    Mockito.doAnswer(inv -> {
-      awaitLatch.countDown();
-      return null;
-    }).when(_scheduler).yield(Mockito.any());
-
-    schedulerService.startAsync().awaitRunning();
-    schedulerService.register(opChain);
-
-    Assert.assertTrue(awaitLatch.await(10, TimeUnit.SECONDS), "expected await to be called in less than 10 seconds");
-
-    // now cancel the request.
-    schedulerService.cancel(123);
-
-    Assert.assertTrue(cancelLatch.await(10, TimeUnit.SECONDS), "expected OpChain to be cancelled");
-    Assert.assertTrue(deregisterLatch.await(10, TimeUnit.SECONDS), "expected OpChain to be deregistered");
-    Mockito.verify(_operatorA, Mockito.times(1)).cancel(Mockito.any());
-    Mockito.verify(_scheduler, Mockito.times(1)).deregister(Mockito.any());
-    schedulerService.stopAsync().awaitTerminated();
-  }
+//  @Test
+//  public void shouldCallCancelOnOpChainsWhenItIsCancelledByDispatch()

Review Comment:
   I'm not sure if this tets is still needed or makes sense.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277784575


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxSendOperatorTest.java:
##########
@@ -160,20 +151,23 @@ public void shouldSendDataBlock()
     // When:
     block = mailboxSendOperator.nextBlock();
     // Then:
-    assertTrue(block.isNoOpBlock(), "expected No-op block to propagate next b/c remaining capacity is 0.");
+    assertTrue(block.isErrorBlock(), "expected error block to propagate next b/c remaining capacity is 0.");
 
     // When:
     block = mailboxSendOperator.nextBlock();
     // Then:
     assertSame(block, eosBlock, "expected EOS block to propagate next even if capacity is now 0.");
     ArgumentCaptor<TransferableBlock> captor = ArgumentCaptor.forClass(TransferableBlock.class);
-    verify(_exchange, times(3)).offerBlock(captor.capture(), anyLong());
+    verify(_exchange, Mockito.times(4)).offerBlock(captor.capture(), anyLong());

Review Comment:
   I've removed several of these mockito checks. The test should not check how many times an internal method is called. Tests should check the external contract, not the internal one.
   
   I had tons of test failling for that reason (because even we return the same, in the new implementation we do an extra call to read the EOS). In this case I decided to keep it because the test is using mockito to read the actual results of the operator. I don't think that is a good idea, but it would take some time to implement the same logic using _better practices_.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277781492


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperatorTest.java:
##########
@@ -139,21 +138,11 @@ public void shouldTimeoutOnExtraLongSleep()
       MetadataBlock errorBlock = (MetadataBlock) mailbox.getDataBlock();
       assertTrue(errorBlock.getExceptions().containsKey(QueryException.EXECUTION_TIMEOUT_ERROR_CODE));
     }
-
-    // Longer timeout or default timeout (10s) doesn't result in timeout
-    context =
-        OperatorTestUtil.getOpChainContext(_mailboxService, RECEIVER_ADDRESS, System.currentTimeMillis() + 10_000L,
-            _stageMetadata1);
-    try (MailboxReceiveOperator receiveOp = new MailboxReceiveOperator(context, RelDistribution.Type.SINGLETON, 1)) {
-      Thread.sleep(100L);
-      TransferableBlock mailbox = receiveOp.nextBlock();

Review Comment:
   Now we fail in this case as well



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277785056


##########
pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OpChainTest.java:
##########
@@ -112,7 +112,8 @@ public void setUp() {
       when(_exchange.getRemainingCapacity()).thenReturn(1);
       when(_mailbox2.poll()).then(x -> {
         if (_blockList.isEmpty()) {
-          return TransferableBlockUtils.getNoOpTransferableBlock();
+          //return TransferableBlockUtils.getNoOpTransferableBlock();
+          return TransferableBlockUtils.getEndOfStreamTransferableBlock();

Review Comment:
   I guess this is the correct result right now. I need to remove the comment above



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on pull request #11205: Remove noop

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on PR #11205:
URL: https://github.com/apache/pinot/pull/11205#issuecomment-1684644041

   Applied a commit to fix `MailboxSendOperatorTest`. Since we no longer need to return no-op block, the old testing logic no longer apply


-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #11205: Remove noop

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1293154431


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +32,73 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
 
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
-  }
-
-  @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
-        @Override
-        public void runJob() {
-          boolean isFinished = false;
-          boolean returnedErrorBlock = false;
-          Throwable thrown = null;
-          try {
-            LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
-            operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
-            TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
-              result = operatorChain.getRoot().nextBlock();
-            }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
-            } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
-            }
-          } catch (Exception e) {
-            LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
-            thrown = e;
-          } finally {
-            if (returnedErrorBlock || thrown != null) {
-              cancelOpChain(operatorChain, thrown);
-            } else if (isFinished) {
-              closeOpChain(operatorChain);
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
+      @Override
+      public void runJob() {
+        boolean isFinished = false;
+        TransferableBlock returnedErrorBlock = null;
+        Throwable thrown = null;
+        try {
+          LOGGER.trace("({}): Executing", operatorChain);
+          operatorChain.getStats().executing();
+          TransferableBlock result = operatorChain.getRoot().nextBlock();
+          while (!result.isEndOfStreamBlock()) {
+            result = operatorChain.getRoot().nextBlock();
+          }
+          isFinished = true;
+          if (result.isErrorBlock()) {
+            returnedErrorBlock = result;
+            LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                result.getDataBlock().getExceptions());
+          } else {
+            LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
+          }
+        } catch (Exception e) {
+          LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
+          thrown = e;
+        } finally {
+          _submittedOpChainMap.remove(operatorChain.getId());
+          if (returnedErrorBlock != null || thrown != null) {
+            if (thrown == null) {
+              thrown = new RuntimeException("Error block " + returnedErrorBlock.getDataBlock().getExceptions());
             }
+            operatorChain.cancel(thrown);
+          } else if (isFinished) {
+            operatorChain.close();
           }
         }
-      });
-    }
-  }
-
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
-  }
-
-  private void closeOpChain(OpChain opChain) {
-    try {
-      opChain.close();
-    } finally {
-      _scheduler.deregister(opChain);
-    }
+      }
+    });
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  private void cancelOpChain(OpChain opChain, Throwable e) {
-    try {
-      opChain.cancel(e);
-    } finally {
-      _scheduler.deregister(opChain);
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. for leaf stage this cannot be a dangling opchain b/c they will eventually be cleared up
+    // via query timeout.
+    List<Map.Entry<OpChainId, Future<?>>> entries =
+        _submittedOpChainMap.entrySet().stream().filter(entry -> entry.getKey().getRequestId() == requestId)
+            .collect(Collectors.toList());
+    for (Map.Entry<OpChainId, Future<?>> entry : entries) {
+      OpChainId key = entry.getKey();
+      Future<?> future = entry.getValue();
+      if (future != null) {
+        future.cancel(true);
+      }
+      _submittedOpChainMap.remove(key);

Review Comment:
   I actually have the opposite opinion here. With `iterator` I'm pretty sure I can get the intended behavior assuming `ConcurrentHashMap` is a very well tested class. If I first cache all the entries into a separate list, and then iterate over them without `ConcurrentHashMap` knowing this list's existence, I cannot guarantee the map won't modify the entry. So IMO it is actually safer to use the `iterator`



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289877958


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java:
##########
@@ -118,46 +118,52 @@ public String toExplainString() {
 
   @Override
   protected TransferableBlock getNextBlock() {
-    boolean canContinue = true;
     TransferableBlock transferableBlock;
+    if (LOGGER.isDebugEnabled()) {
+      LOGGER.debug("==[SEND]== Enter getNextBlock from: " + _context.getId());
+    }
     try {
       transferableBlock = _sourceOperator.nextBlock();
-      if (transferableBlock.isNoOpBlock()) {
-        return transferableBlock;
-      } else if (transferableBlock.isEndOfStreamBlock()) {
-        if (transferableBlock.isSuccessfulEndOfStreamBlock()) {
-          // Stats need to be populated here because the block is being sent to the mailbox
-          // and the receiving opChain will not be able to access the stats from the previous opChain
-          TransferableBlock eosBlockWithStats = TransferableBlockUtils.getEndOfStreamTransferableBlock(
-              OperatorUtils.getMetadataFromOperatorStats(_opChainStats.getOperatorStatsMap()));
-          sendTransferableBlock(eosBlockWithStats);
-        } else {
-          sendTransferableBlock(transferableBlock);
-        }
-      } else { // normal blocks
-        // check whether we should continue depending on exchange queue condition.
-        canContinue = sendTransferableBlock(transferableBlock);
+      if (transferableBlock.isSuccessfulEndOfStreamBlock()) {
+        // Stats need to be populated here because the block is being sent to the mailbox
+        // and the receiving opChain will not be able to access the stats from the previous opChain
+        TransferableBlock eosBlockWithStats = TransferableBlockUtils.getEndOfStreamTransferableBlock(
+            OperatorUtils.getMetadataFromOperatorStats(_opChainStats.getOperatorStatsMap()));
+        sendTransferableBlock(eosBlockWithStats, false);
+      } else {
+        sendTransferableBlock(transferableBlock, true);
       }
     } catch (Exception e) {
       transferableBlock = TransferableBlockUtils.getErrorTransferableBlock(e);
       try {
         LOGGER.error("Exception while transferring data on opChain: " + _context.getId(), e);
-        sendTransferableBlock(transferableBlock);
+        sendTransferableBlock(transferableBlock, false);
       } catch (Exception e2) {
         LOGGER.error("Exception while sending error block.", e2);
       }
     }
-    // yield if we cannot continue to put transferable block into the sending queue
-    return canContinue ? transferableBlock : TransferableBlockUtils.getNoOpTransferableBlock();
+    return transferableBlock;
   }
 
-  private boolean sendTransferableBlock(TransferableBlock block)
+  private void sendTransferableBlock(TransferableBlock block, boolean throwIfTimeout)

Review Comment:
   The idea here is that sometimes we don't want to do that. Specially when we found an exception in the happy path. There we send the exception downstream as an error block. In case that error blocks times out... what do we want to do?
   
   If we decide to throw a timeout there we would hide the original exception. IIRC @walterddr's original code is the one that introduce this _I don't want to throw timeout_ parameter and I though it was a good idea. But we can discuss about that.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289954302


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +32,73 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
 
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
-  }
-
-  @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
-        @Override
-        public void runJob() {
-          boolean isFinished = false;
-          boolean returnedErrorBlock = false;
-          Throwable thrown = null;
-          try {
-            LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
-            operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
-            TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
-              result = operatorChain.getRoot().nextBlock();
-            }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
-            } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
-            }
-          } catch (Exception e) {
-            LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
-            thrown = e;
-          } finally {
-            if (returnedErrorBlock || thrown != null) {
-              cancelOpChain(operatorChain, thrown);
-            } else if (isFinished) {
-              closeOpChain(operatorChain);
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
+      @Override
+      public void runJob() {
+        boolean isFinished = false;
+        TransferableBlock returnedErrorBlock = null;
+        Throwable thrown = null;
+        try {
+          LOGGER.trace("({}): Executing", operatorChain);
+          operatorChain.getStats().executing();
+          TransferableBlock result = operatorChain.getRoot().nextBlock();
+          while (!result.isEndOfStreamBlock()) {
+            result = operatorChain.getRoot().nextBlock();
+          }
+          isFinished = true;
+          if (result.isErrorBlock()) {
+            returnedErrorBlock = result;
+            LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                result.getDataBlock().getExceptions());
+          } else {
+            LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
+          }
+        } catch (Exception e) {
+          LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
+          thrown = e;
+        } finally {
+          _submittedOpChainMap.remove(operatorChain.getId());
+          if (returnedErrorBlock != null || thrown != null) {
+            if (thrown == null) {
+              thrown = new RuntimeException("Error block " + returnedErrorBlock.getDataBlock().getExceptions());
             }
+            operatorChain.cancel(thrown);
+          } else if (isFinished) {
+            operatorChain.close();
           }
         }
-      });
-    }
-  }
-
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
-  }
-
-  private void closeOpChain(OpChain opChain) {
-    try {
-      opChain.close();
-    } finally {
-      _scheduler.deregister(opChain);
-    }
+      }
+    });
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  private void cancelOpChain(OpChain opChain, Throwable e) {
-    try {
-      opChain.cancel(e);
-    } finally {
-      _scheduler.deregister(opChain);
+  @Override
+  public void cancel(long requestId) {
+    // simple cancellation. for leaf stage this cannot be a dangling opchain b/c they will eventually be cleared up
+    // via query timeout.
+    List<Map.Entry<OpChainId, Future<?>>> entries =
+        _submittedOpChainMap.entrySet().stream().filter(entry -> entry.getKey().getRequestId() == requestId)
+            .collect(Collectors.toList());
+    for (Map.Entry<OpChainId, Future<?>> entry : entries) {
+      OpChainId key = entry.getKey();
+      Future<?> future = entry.getValue();
+      if (future != null) {
+        future.cancel(true);
+      }
+      _submittedOpChainMap.remove(key);

Review Comment:
   I didn't want to use that approach because Map.entrySet says:
   
   ```If the map is modified while an iteration over the set is in progress (except through the iterator's own remove operation, or through the setValue operation on a map entry returned by the iterator) the results of the iteration are undefined```
   
   On the other hand, ConcurrentHashMap.entrySet says:
   ```
   The view's iterators and spliterators are [weakly consistent](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/package-summary.html#Weakly).
   ```
   Which is defined in https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/package-summary.html#Weakly as:
   ```
   Most concurrent Collection implementations (including most Queues) also differ from the usual java.util conventions in that their [Iterators](https://docs.oracle.com/javase/8/docs/api/java/util/Iterator.html) and [Spliterators](https://docs.oracle.com/javase/8/docs/api/java/util/Spliterator.html) provide weakly consistent rather than fast-fail traversal:
   
   - they may proceed concurrently with other operations
   - they will never throw [ConcurrentModificationException](https://docs.oracle.com/javase/8/docs/api/java/util/ConcurrentModificationException.html)
   - they are guaranteed to traverse elements as they existed upon construction exactly once, and may (but are not guaranteed to) reflect any modifications subsequent to construction.
   ```
   
   So your code should be safe, but seems difficult to reason about that. We also don't know the blocking implications, although they doesn't seem to be that important (we assume they are correct, but we don't care that much if we block other accesses to the map for some extra microseconds).
   
   That is why I though the current code is easier to reason about, but I'm open to use yours if you prefer that.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1285457564


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -51,16 +51,30 @@ public class ReceivingMailbox {
   // TODO: Revisit if this is the correct way to apply back pressure
   private final BlockingQueue<TransferableBlock> _blocks = new ArrayBlockingQueue<>(DEFAULT_MAX_PENDING_BLOCKS);
   private final AtomicReference<TransferableBlock> _errorBlock = new AtomicReference<>();
+  @Nullable
+  private volatile Reader _reader;
 
   public ReceivingMailbox(String id, Consumer<OpChainId> receiveMailCallback) {
     _id = id;
     _receiveMailCallback = receiveMailCallback;
   }
 
+  public void registeredReader(Reader reader) {

Review Comment:
   > At that moment the reader is not registered but there are contents within the mailbox
   
   Is that something that can only happen in `InMemorySendingMailbox`? I though it could also happen in the grpc one. Anyway, this supports this pattern:
   1. If reader is created before the writer, no problem. It will be blocked in the queue and registered as listener.
   2. In the other case, the writer will add the block to the queue and will notify nobody. It will continue producing messages until the stream is closed or the buffer is full (same conditions we had before).
   3. Once the reader is registered, it won't see the previous notifications (because sender didn't send them) but it will first read the queue.
   4. Once the queue is completely consumed, instead of blocking on the queue, we block on a operator defined structure (which is also a queue, but that is not relevant). That structure is shared for the operator, which may have several read mailboxes, and unblock whenever one them has data. In fact this part is very similar to what we had before. But instead of storing that structure in a map in the dispatcher /scheduler (which sounds strange now that there is nothing to schedule), it is stored in the operator.
   
   > Our use pattern is not standard, so not sure if we should put a general framework.
   
   That is not correct. First, it is quite standard. In fact reactive-streams implements the same feature (in a even more generic fashion, letting you chose between cold and hot streams). I would even suggest to use reactive patterns in future to make the code simpler from our side.
   
   But also that is not correct inside our own project. Right now, in master, both `MailboxReceiveOperator` and `SortedMailboxReceiveOperator` share this code (in the algorithmic form used in master, with noops and delegating on the dispatcher/scheduler to block). Also, as shown in the previous commits, pipeline breaker could be implemented using the same pattern (but using blocking streams instead of unblocking). I've removed that because you had some concerns, but we use that pattern in at least 2 places. Given how complex is to reason about concurrency, I think it is quite better to use a single, generic structure that can be tested in isolation and then used that in different places.
   
   If you are really concerned about the lack of reusability, then we can rollback the changes introduced in  c936badf3f8f158568d51793ba59fe657acf7a13, when I've added this class in order to reuse the code between receiving mailbox and pipeline breaker and go back to the version we had in  650e4a88c032c1e9e0345cf82c577474fdd357e8, where mostly the same code was in `BaseMailboxReceiveOperator` and therefore only `MailboxReceiveOperator` and `SortedMailboxReceiveOperator`.
   
   Alternatively you may be asking to go even further back and recover the algorithm where the blocking code was in the dispatcher/scheduler. We can discuss about that, but I think that is an incorrect pattern that introduces dependencies between the scheduler and the operators. We can go back there if you think there is a correctness issue with this proposed new structure, but I don't think reusability is a strong reason recover that code.
   
   



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1286919699


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java:
##########
@@ -81,9 +84,10 @@ public void cancel(Throwable t) {
         _contentObserver = getContentObserver();
       }
       try {
+        String msg = t != null ? t.getMessage() : "Unknown";

Review Comment:
   Id doesn't hurt. I think it is a small cost with the huge benefit of not having to see NPEs in the logs while trying to find what was the actual reason to cancel the op chain :laughing: 



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289857744


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java:
##########
@@ -0,0 +1,246 @@
+/**
+ * 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.utils;
+
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public abstract class BlockingMultiStreamConsumer<E> implements AutoCloseable {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BlockingMultiStreamConsumer.class);
+  private final Object _id;
+  protected final List<? extends AsyncStream<E>> _mailboxes;
+  protected final ArrayBlockingQueue<Boolean> _newDataReady = new ArrayBlockingQueue<>(1);
+  private final long _deadlineMs;
+  /**
+   * An index that used to calculate where do we are going to start reading.
+   * The invariant is that we are always going to start reading from {@code _lastRead + 1}.
+   * Therefore {@link #_lastRead} must be in the range {@code [-1, mailbox.size() - 1]}
+   */
+  protected int _lastRead;
+  private E _errorBlock = null;
+
+  public BlockingMultiStreamConsumer(Object id, long deadlineMs, List<? extends AsyncStream<E>> asyncProducers) {
+    _id = id;
+    _deadlineMs = deadlineMs;
+    AsyncStream.OnNewData onNewData = this::onData;
+    _mailboxes = asyncProducers;
+    _mailboxes.forEach(blockProducer -> blockProducer.addOnNewDataListener(onNewData));
+    _lastRead = _mailboxes.size() - 1;
+  }
+
+  protected abstract boolean isError(E element);
+
+  protected abstract boolean isEos(E element);
+
+  protected abstract E onTimeout();
+
+  protected abstract E onException(Exception e);
+
+  protected abstract E onEos();
+
+  @Override
+  public void close() {
+    cancelRemainingMailboxes();
+  }
+
+  public void cancel(Throwable t) {
+    cancelRemainingMailboxes();
+  }
+
+  protected void cancelRemainingMailboxes() {
+    for (AsyncStream<E> mailbox : _mailboxes) {
+      mailbox.cancel();
+    }
+  }
+
+  public void onData() {
+    if (_newDataReady.offer(Boolean.TRUE)) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("New data notification delivered on " + _id + ". " + System.identityHashCode(_newDataReady));
+      }
+    } else if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("New data notification ignored on " + _id + ". " + System.identityHashCode(_newDataReady));
+    }
+  }
+
+  /**
+   * Reads the next block for any ready mailbox or blocks until some of them is ready.
+   *
+   * The method implements a sequential read semantic. Meaning that:
+   * <ol>
+   *   <li>EOS is only returned when all mailboxes already emitted EOS or there are no mailboxes</li>
+   *   <li>If an error is read from a mailbox, the error is returned</li>
+   *   <li>If data is read from a mailbox, that data block is returned</li>
+   *   <li>If no mailbox is ready, the calling thread is blocked</li>
+   * </ol>
+   *
+   * Right now the implementation tries to be fair. If one call returned the block from mailbox {@code i}, then next
+   * call will look for mailbox {@code i+1}, {@code i+2}... in a circular manner.
+   *
+   * In order to unblock a thread blocked here, {@link #onData()} should be called.   *
+   */
+  public E readBlockBlocking() {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("==[RECEIVE]== Enter getNextBlock from: " + _id + " mailboxSize: " + _mailboxes.size());
+    }
+    // Standard optimistic execution. First we try to read without acquiring the lock.
+    E block = readDroppingSuccessEos();
+    long timeoutMs = _deadlineMs - System.currentTimeMillis();
+    boolean timeout = false;
+    try {
+      while (block == null) { // we didn't find a mailbox ready to read, so we need to be pessimistic

Review Comment:
   > Once we reach the timeout we should immediately return timeout error.
   
   We do, although the code may not be 100% clear. We can timeout either in `timeout = _newDataReady.poll(timeoutMs, TimeUnit.MILLISECONDS) == null;` a couple of lines above or in `readDroppingSuccessEos`, in which case it will return a not null block and while will finish.
   
   We can modify the while predicate to be `while (!timeout && block == null)` if you found it easier to read.
   
   > Also, why do we have a while loop here? We should wait on _newDataReady once?
   > I somehow see why you want to have this while check
   
   Also, we protect ourself in case we the sender adds a null block. When this code was also used in pipeline breaker it was an important detail.
   
   But in general when dealing with concurrency is better to repeat the condition after the synchronization. As you say, it is difficult to reason about concurrency. In this case `_newDataReady` does not guarantee that there are data on the queue. It is just used to unblock the reader thread if waiting for more data. You can see in ArrayBlockingQueue implementation that the `notEmpty` condition there is used in the same way. In general, you should always check the precondition in a while when waiting on a signal.
   
   I would like to add a jcstress test here, but they are not easy to be include in a maven pipeline



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1289961342


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/SetOperator.java:
##########
@@ -112,15 +112,14 @@ protected void constructRightBlockSet() {
 
   protected TransferableBlock constructResultBlockSet(TransferableBlock leftBlock) {
     List<Object[]> rows = new ArrayList<>();
+    // TODO: Other operators keep the first erroneous block, while this keep the last.
+    //  We should decide what is what we want to do and be consistent with that.

Review Comment:
   I think once we read an error from an operator, we don't read that operator again. But in case we do, what should we return? The first error or a new one? Right now some operators keep the first error emitted and always return the same error afterwards, but others create a new error each time they are being called.
   
   My TODO is to try to homogenize that to either always return the first error or always create a new one. Alternative, we can define that we cannot call `getNextBlock()` (or read from mailboxes) once the first error is emitted. In that case we should throw an exception.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277786351


##########
pinot-query-runtime/src/test/resources/log4j2.xml:
##########
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    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.
+
+-->
+<Configuration>
+  <Appenders>
+    <Console name="console" target="SYSTEM_OUT">
+      <PatternLayout pattern="%d{HH:mm:ss.SSS} %p [%c{1}] [%t] %m%n"/>
+    </Console>
+  </Appenders>
+  <Loggers>
+    <Logger name="org.apache.pinot" level="warn" additivity="false">
+      <AppenderRef ref="console"/>
+    </Logger>
+    <Logger name="org.apache.pinot.query" level="trace" additivity="false">

Review Comment:
   We should change that trace level before merging



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277707830


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/MailboxService.java:
##########
@@ -144,17 +150,30 @@ public SendingMailbox getSendingMailbox(String hostname, int port, String mailbo
     }
   }
 
-  /**
-   * Returns the receiving mailbox for the given mailbox id.
-   */
-  public ReceivingMailbox getReceivingMailbox(String mailboxId) {
+  public ReceivingMailbox getReceivingMailbox(String mailboxId, @Nullable Consumer<OpChainId> extraCallback) {
     try {
-      return _receivingMailboxCache.get(mailboxId, () -> new ReceivingMailbox(mailboxId, _unblockOpChainCallback));
+      Consumer<OpChainId> callback;
+      if (extraCallback == null) {
+        callback = _unblockOpChainCallback;
+      } else {
+        callback = opChainId -> {
+          extraCallback.accept(opChainId);

Review Comment:
   This is my main contribution over @walterddr's code. Instead of storing a map in the scheduler that maps op chains to their blocking objets (in his case a blocking list of 1 element) here I'm sending a message to the opchain itself. By doing that we decouple the receiving operation from the scheduler.
   
   You can see `BaseMailboxReceiveOperator.onData` to understand what it does.



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277736498


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -133,4 +152,15 @@ public void cancel() {
   public int getNumPendingBlocks() {
     return _blocks.size();
   }
+
+  private void notifyReader() {
+    Reader reader = _reader;
+    if (reader != null) {

Review Comment:
   To be fair, there is a race condition here. Being paranoid, the following sequence is legal and problematic:
   1. The mailbox is created by the sender side
   2. A message is sent. But just after `_reader` is read...
   3. The reader is registered
   4. The reader polls the mailbox and found nothing
   5. The reader blocks on the `notEmpty` condition
   6. The mailbox executes this line, which is false because when read, _reader was null
   7. The message is not sent
   8. No more messages are sent by the sender
   9. The reader is infinitely waiting for the condition
   
   I'll try to fix that next week by adding a lock with an optimistic read, so performance should not be affected



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1277739458


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/TransferableBlock.java:
##########
@@ -188,4 +193,10 @@ private boolean isType(MetadataBlock.MetadataBlockType type) {
     MetadataBlock metadata = (MetadataBlock) _dataBlock;
     return metadata.getType() == type;
   }
+
+  @Override
+  public String toString() {
+    String blockType = isErrorBlock() ? "error" : isSuccessfulEndOfStreamBlock() ? "eos" : "data";
+    return "TransferableBlock{blockType=" + blockType + ", _numRows=" + _numRows + ", _container=" + _container + '}';

Review Comment:
   We may want to improve the `toString`, but it is very useful in logs and right now it doesn't print customer information



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1280340245


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java:
##########
@@ -133,4 +152,15 @@ public void cancel() {
   public int getNumPendingBlocks() {
     return _blocks.size();
   }
+
+  private void notifyReader() {
+    Reader reader = _reader;
+    if (reader != null) {

Review Comment:
   This race condition has been fixed by adding the following invariant:
   - In order to call poll, `_reader` must be not null.
   
   I've added a `Precondition` in poll that verifies that (as long as there is a single reader thread, which is something we assume).



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on PR #11205:
URL: https://github.com/apache/pinot/pull/11205#issuecomment-1658237230

   I've just added support for pipeline breaker


-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] gortiz commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1281606825


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -31,151 +33,84 @@
 import org.slf4j.LoggerFactory;
 
 
-/**
- * This class provides the implementation for scheduling multistage queries on a single node based
- * on the {@link OpChainScheduler} logic that is passed in. Multistage queries support partial execution
- * and will return a NOOP metadata block as a "yield" signal, indicating that the next operator
- * chain ({@link OpChainScheduler#next} will be requested.
- */
-@SuppressWarnings("UnstableApiUsage")
-public class OpChainSchedulerService extends AbstractExecutionThreadService {
+public class OpChainSchedulerService implements SchedulerService {
   private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
-  /**
-   * Default time scheduler is allowed to wait for a runnable OpChain to be available.
-   */
-  private static final long DEFAULT_SCHEDULER_NEXT_WAIT_MS = 100;
-  /**
-   * Default cancel signal retention, this should be set to several times larger than
-   * {@link org.apache.pinot.query.service.QueryConfig#DEFAULT_SCHEDULER_RELEASE_TIMEOUT_MS}.
-   */
-  private static final long SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS = 60_000L;
-
-  private final OpChainScheduler _scheduler;
-  private final ExecutorService _workerPool;
-  private final Cache<Long, Long> _cancelledRequests = CacheBuilder.newBuilder()
-      .expireAfterWrite(SCHEDULER_CANCELLATION_SIGNAL_RETENTION_MS, TimeUnit.MILLISECONDS).build();
 
-  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
-    _scheduler = scheduler;
-    _workerPool = workerPool;
-  }
+  private final ExecutorService _executorService;
+  private final ConcurrentHashMap<OpChainId, Future<?>> _submittedOpChainMap;
 
-  @Override
-  protected void triggerShutdown() {
-    // TODO: Figure out shutdown lifecycle with graceful shutdown in mind.
-    LOGGER.info("Triggered shutdown on OpChainScheduler...");
+  public OpChainSchedulerService(ExecutorService executorService) {
+    _executorService = executorService;
+    _submittedOpChainMap = new ConcurrentHashMap<>();
   }
 
   @Override
-  protected void run()
-      throws Exception {
-    while (isRunning()) {
-      OpChain operatorChain = _scheduler.next(DEFAULT_SCHEDULER_NEXT_WAIT_MS, TimeUnit.MILLISECONDS);
-      if (operatorChain == null) {
-        continue;
-      }
-      LOGGER.trace("({}): Scheduling", operatorChain);
-      _workerPool.submit(new TraceRunnable() {
+  public void register(OpChain operatorChain) {
+    Future<?> scheduledFuture = _executorService.submit(new TraceRunnable() {
         @Override
         public void runJob() {
           boolean isFinished = false;
-          boolean returnedErrorBlock = false;
+          TransferableBlock returnedErrorBlock = null;
           Throwable thrown = null;
           try {
             LOGGER.trace("({}): Executing", operatorChain);
-            // throw if the operatorChain is cancelled.
-            if (_cancelledRequests.asMap().containsKey(operatorChain.getId().getRequestId())) {
-              throw new InterruptedException("Query was cancelled!");
-            }
             operatorChain.getStats().executing();
-            // so long as there's work to be done, keep getting the next block
-            // when the operator chain returns a NOOP block, then yield the execution
-            // of this to another worker
             TransferableBlock result = operatorChain.getRoot().nextBlock();
-            while (!result.isNoOpBlock() && !result.isEndOfStreamBlock()) {
+            while (!result.isEndOfStreamBlock()) {
               result = operatorChain.getRoot().nextBlock();
             }
-
-            if (result.isNoOpBlock()) {
-              // TODO: There should be a waiting-for-data state in OpChainStats.
-              operatorChain.getStats().queued();
-              _scheduler.yield(operatorChain);
+            isFinished = true;
+            if (result.isErrorBlock()) {
+              returnedErrorBlock = result;
+              LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
+                  result.getDataBlock().getExceptions());
             } else {
-              isFinished = true;
-              if (result.isErrorBlock()) {
-                returnedErrorBlock = true;
-                LOGGER.error("({}): Completed erroneously {} {}", operatorChain, operatorChain.getStats(),
-                    result.getDataBlock().getExceptions());
-              } else {
-                LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
-              }
+              LOGGER.debug("({}): Completed {}", operatorChain, operatorChain.getStats());
             }
           } catch (Exception e) {
             LOGGER.error("({}): Failed to execute operator chain! {}", operatorChain, operatorChain.getStats(), e);
             thrown = e;
           } finally {
-            if (returnedErrorBlock || thrown != null) {
+            if (returnedErrorBlock != null || thrown != null) {
+              if (thrown == null) {
+                String blockMsg;
+                if (returnedErrorBlock.getDataBlock() instanceof MetadataBlock) {
+                  MetadataBlock metadataBlock = (MetadataBlock) returnedErrorBlock.getDataBlock();
+                  blockMsg = String.join(", ", metadataBlock.getExceptions().values());
+                } else {
+                  blockMsg = "Unknown";
+                }
+                thrown = new RuntimeException("Error block " + blockMsg);
+              }
               cancelOpChain(operatorChain, thrown);
             } else if (isFinished) {
               closeOpChain(operatorChain);
             }
           }
         }
       });
-    }
+    _submittedOpChainMap.put(operatorChain.getId(), scheduledFuture);
   }
 
-  /**
-   * Register a new operator chain with the scheduler.
-   *
-   * @param operatorChain the chain to register
-   */
-  public final void register(OpChain operatorChain) {
-    operatorChain.getStats().queued();
-    _scheduler.register(operatorChain);
-    LOGGER.debug("({}): Scheduler is now handling operator chain listening to mailboxes {}. "
-            + "There are a total of {} chains awaiting execution.", operatorChain,
-        operatorChain.getReceivingMailboxIds(),
-        _scheduler.size());
-  }
-
-  /**
-   * Async cancel a request. Request will not be fully cancelled until the next time opChain is being polled.
-   *
-   * @param requestId requestId to be cancelled.
-   */
-  public final void cancel(long requestId) {
-    _cancelledRequests.put(requestId, requestId);
-  }
-
-  /**
-   * This method should be called whenever data is available for an {@link OpChain} to consume.
-   * Implementations of this method should be idempotent, it may be called in the scenario that no data is available.
-   *
-   * @param opChainId the identifier of the operator chain
-   */
-  public final void onDataAvailable(OpChainId opChainId) {
-    _scheduler.onDataAvailable(opChainId);
-  }
-
-  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine
-  public ExecutorService getWorkerPool() {
-    return _workerPool;
+  @Override
+  public void cancel(long requestId) {

Review Comment:
   I may not have the whole picture but this way to cancel a request doesn't seem so expensive to me TBH. I would not expect to receive thousands of cancellation request per second, but a few per day. The cost of cancelling them is also not that high. Is there something I'm not getting?



-- 
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-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] walterddr commented on a diff in pull request #11205: [Draft] Remove noop

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #11205:
URL: https://github.com/apache/pinot/pull/11205#discussion_r1280085055


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java:
##########
@@ -62,10 +65,10 @@ public OpChainExecutionContext(MailboxService mailboxService, long requestId, in
     _traceEnabled = traceEnabled;
   }
 
-  public OpChainExecutionContext(PhysicalPlanContext physicalPlanContext) {
+  public OpChainExecutionContext(PhysicalPlanContext physicalPlanContext, Executor executor) {
     this(physicalPlanContext.getMailboxService(), physicalPlanContext.getRequestId(), physicalPlanContext.getStageId(),
         physicalPlanContext.getServer(), physicalPlanContext.getDeadlineMs(), physicalPlanContext.getStageMetadata(),
-        physicalPlanContext.getPipelineBreakerResult(), physicalPlanContext.isTraceEnabled());
+        physicalPlanContext.getPipelineBreakerResult(), physicalPlanContext.isTraceEnabled(), executor);

Review Comment:
   nit: i was wondering if we should separate the plan context from the machine context. 
   based on what i see: `mailboxService` / `executor` is machine context and the rest of physicalPlanContext should still remain. 
   
   This should be a good start but we can clean this up further in the next iteration



-- 
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-unsubscribe@pinot.apache.org

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


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