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/15 19:16:23 UTC

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

61yao commented on code in PR #9753:
URL: https://github.com/apache/pinot/pull/9753#discussion_r1016131586


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

Review Comment:
   Add a comment saying this is guarded by monitor below?



##########
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();
+  private 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:
   Is this logging expensive? I feel there would be a lot of logs if we log the pause every time. Can we have some class such as OpChainStats to hold the data and we decide later where to report them? 



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OpChain.java:
##########
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.operator;
+
+import com.google.common.base.Suppliers;
+import java.util.function.Supplier;
+import org.apache.pinot.common.request.context.ThreadTimer;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+
+
+/**
+ * An {@code OpChain} represents a chain of operators that are separated
+ * by send/receive stages.
+ */
+public class OpChain {

Review Comment:
   Ideally this OpChain should capture the timeout info rather than we rely on MailboxSendOperator timeout. It is clear when we should timeout rather than relying on if the root operator times out correctly, this will work.  



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OpChain.java:
##########
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.runtime.operator;
+
+import com.google.common.base.Suppliers;
+import java.util.function.Supplier;
+import org.apache.pinot.common.request.context.ThreadTimer;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.query.runtime.blocks.TransferableBlock;
+
+
+/**
+ * An {@code OpChain} represents a chain of operators that are separated
+ * by send/receive stages.
+ */
+public class OpChain {

Review Comment:
   I feel we should also have OpChain ID somewhere. maybe in future PRs :)



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