You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/11/07 23:16:55 UTC

[GitHub] [pinot] agavra commented on a diff in pull request #9753: [multistage] implement naive round robin operator chain scheduling

agavra commented on code in PR #9753:
URL: https://github.com/apache/pinot/pull/9753#discussion_r1015990164


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -0,0 +1,147 @@
+/**
+ * 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 com.google.common.util.concurrent.AbstractExecutionThreadService;
+import com.google.common.util.concurrent.Monitor;
+import java.util.concurrent.ExecutorService;
+import org.apache.pinot.common.request.context.ThreadTimer;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.query.mailbox.MailboxIdentifier;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.operator.OpChain;
+import org.slf4j.Logger;
+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.
+ *
+ * <p>Note that a yielded operator chain will be re-registered with the underlying scheduler.
+ */
+@SuppressWarnings("UnstableApiUsage")
+public class OpChainSchedulerService extends AbstractExecutionThreadService {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
+
+  private final OpChainScheduler _scheduler;
+  private final ExecutorService _workerPool;
+
+  // anything that is guarded by this monitor should be non-blocking
+  private final Monitor _monitor = new Monitor();
+  protected final Monitor.Guard _hasNextOrClosing = new Monitor.Guard(_monitor) {
+    @Override
+    public boolean isSatisfied() {
+      return _scheduler.hasNext() || !isRunning();
+    }
+  };
+
+  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
+    _scheduler = scheduler;
+    _workerPool = workerPool;
+  }
+
+  @Override
+  protected void triggerShutdown() {
+    // this wil just notify all waiters that the scheduler is shutting down
+    _monitor.enter();
+    _monitor.leave();
+  }
+
+  @Override
+  protected void run()
+      throws Exception {
+    while (isRunning()) {
+      _monitor.enterWhen(_hasNextOrClosing);
+      try {
+        if (!isRunning()) {
+          return;
+        }
+
+        OpChain operatorChain = _scheduler.next();
+        _workerPool.submit(new TraceRunnable() {
+          @Override
+          public void runJob() {
+            try {
+              ThreadTimer timer = operatorChain.getAndStartTimer();
+
+              // 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()) {
+                LOGGER.debug("Got block with {} rows.", result.getNumRows());
+                result = operatorChain.getRoot().nextBlock();
+              }
+
+              if (!result.isEndOfStreamBlock()) {
+                // not complete, needs to re-register for scheduling
+                register(operatorChain);
+              } else {
+                LOGGER.info("Execution time: " + timer.getThreadTimeNs());

Review Comment:
   for more complex scheduling algorithms, we will add a callback here to `complete` or `unregister` an operator chain. that requires a unique way to identify operator chains which adds a bit more code so I avoided it for this PR



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java:
##########
@@ -0,0 +1,147 @@
+/**
+ * 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 com.google.common.util.concurrent.AbstractExecutionThreadService;
+import com.google.common.util.concurrent.Monitor;
+import java.util.concurrent.ExecutorService;
+import org.apache.pinot.common.request.context.ThreadTimer;
+import org.apache.pinot.core.util.trace.TraceRunnable;
+import org.apache.pinot.query.mailbox.MailboxIdentifier;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+import org.apache.pinot.query.runtime.operator.OpChain;
+import org.slf4j.Logger;
+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.
+ *
+ * <p>Note that a yielded operator chain will be re-registered with the underlying scheduler.
+ */
+@SuppressWarnings("UnstableApiUsage")
+public class OpChainSchedulerService extends AbstractExecutionThreadService {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(OpChainSchedulerService.class);
+
+  private final OpChainScheduler _scheduler;
+  private final ExecutorService _workerPool;
+
+  // anything that is guarded by this monitor should be non-blocking
+  private final Monitor _monitor = new Monitor();
+  protected final Monitor.Guard _hasNextOrClosing = new Monitor.Guard(_monitor) {
+    @Override
+    public boolean isSatisfied() {
+      return _scheduler.hasNext() || !isRunning();
+    }
+  };
+
+  public OpChainSchedulerService(OpChainScheduler scheduler, ExecutorService workerPool) {
+    _scheduler = scheduler;
+    _workerPool = workerPool;
+  }
+
+  @Override
+  protected void triggerShutdown() {
+    // this wil just notify all waiters that the scheduler is shutting down
+    _monitor.enter();
+    _monitor.leave();
+  }
+
+  @Override
+  protected void run()
+      throws Exception {
+    while (isRunning()) {
+      _monitor.enterWhen(_hasNextOrClosing);
+      try {
+        if (!isRunning()) {
+          return;
+        }
+
+        OpChain operatorChain = _scheduler.next();
+        _workerPool.submit(new TraceRunnable() {
+          @Override
+          public void runJob() {
+            try {
+              ThreadTimer timer = operatorChain.getAndStartTimer();
+
+              // 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()) {
+                LOGGER.debug("Got block with {} rows.", result.getNumRows());
+                result = operatorChain.getRoot().nextBlock();
+              }
+
+              if (!result.isEndOfStreamBlock()) {
+                // not complete, needs to re-register for scheduling
+                register(operatorChain);
+              } else {
+                LOGGER.info("Execution time: " + timer.getThreadTimeNs());
+              }
+            } catch (Exception e) {
+              LOGGER.error("Failed to execute query!", e);
+            }
+          }
+        });
+      } finally {
+        _monitor.leave();
+      }
+    }
+  }
+
+  /**
+   * Register a new operator chain with the scheduler.
+   *
+   * @param operatorChain the chain to register
+   */
+  public final void register(OpChain operatorChain) {
+    _monitor.enter();
+    try {
+      _scheduler.register(operatorChain);
+    } finally {
+      _monitor.leave();
+    }
+  }
+
+  /**
+   * This method should be called whenever data is available in a given mailbox.
+   * Implementations of this method should be idempotent, it may be called in the
+   * scenario that no mail is available.
+   *
+   * @param mailbox the identifier of the mailbox that now has data
+   */
+  public final void onDataAvailable(MailboxIdentifier mailbox) {
+    _monitor.enter();
+    try {
+      _scheduler.onDataAvailable(mailbox);
+    } finally {
+      _monitor.leave();
+    }
+  }
+
+  // TODO: remove this method after we pipe down the proper executor pool to the v1 engine

Review Comment:
   this is a bit unfortunate, but it's how the existing code works and refactoring it would be out of scope for this PR. While it's not particularly efficient, it also isn't dangerous - V1 queries are non-blocking, so using the same worker pool for executing V1 queries (that are issued as part of V2) and V2 intermediate queries does not threaten liveness.



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