ankitsultana commented on code in PR #10289:
URL: https://github.com/apache/pinot/pull/10289#discussion_r1111432772


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainScheduler.java:
##########
@@ -18,52 +18,67 @@
  */
 package org.apache.pinot.query.runtime.executor;
 
+import java.util.concurrent.TimeUnit;
+import javax.annotation.concurrent.ThreadSafe;
 import org.apache.pinot.query.mailbox.MailboxIdentifier;
 import org.apache.pinot.query.runtime.operator.OpChain;
 
 
 /**
- * An interface that defines different scheduling strategies to work with the
- * {@link OpChainSchedulerService}. All methods are thread safe and can be 
guaranteed
- * to never be called concurrently - therefore all implementations may use data
- * structures that are not concurrent.
+ * An interface that defines different scheduling strategies to work with the 
{@link OpChainSchedulerService}.
  */
+@ThreadSafe
 public interface OpChainScheduler {
-
   /**
+   * Registers a new OpChain with the scheduler.
    * @param operatorChain the operator chain to register
-   * @param isNew         whether or not this is the first time the operator 
is scheduled
    */
-  void register(OpChain operatorChain, boolean isNew);
+  void register(OpChain operatorChain);
 
   /**
-   * This method is called whenever {@code mailbox} has new data available to 
consume,
-   * this can be useful for advanced scheduling algorithms
-   *
-   * @param mailbox the mailbox ID
+   * When the OpChain is finished, error or otherwise, deregister is called 
for it so the scheduler can do any required
+   * cleanup. After an OpChain is de-registered, the scheduler service will 
never call any other method for it.
+   * However, the {@link #onDataAvailable} callback may be called even after 
an OpChain is de-registered, and the
+   * scheduler should handle that scenario.
+   * @param operatorChain an operator chain that is finished (error or 
otherwise).
    */
-  void onDataAvailable(MailboxIdentifier mailbox);
+  void deregister(OpChain operatorChain);
 
   /**
-   * This method is called when scheduler is terminating. It should clean up 
all of the resources if there are any.
-   * register() and onDataAvailable() shouldn't be called anymore after 
shutDown is called.
+   * Used by {@link OpChainSchedulerService} to indicate that a given OpChain 
can be suspended until it receives some
+   * data. Note that this method is only used by the scheduler service to 
"indicate" that an OpChain can be suspended.
+   * The decision on whether to actually suspend or not can be taken by the 
scheduler.
    */
-  void shutDown();
+  void yield(OpChain opChain);
 
   /**
-   * @return whether or not there is any work for the scheduler to do
+   * A callback called whenever data is received for the given mailbox. This 
can be used by the scheduler
+   * implementations to re-scheduled suspended OpChains. This method may be 
called for an OpChain that has not yet
+   * been scheduled, or an OpChain that has already been de-registered.
+   * @param mailbox the mailbox ID
    */
-  boolean hasNext();
+  void onDataAvailable(MailboxIdentifier mailbox);
 
   /**
-   * @return the next operator chain to process
-   * @throws java.util.NoSuchElementException if {@link #hasNext()} returns 
false
-   *         prior to this call
+   * Returns an OpChain that is ready to be run by {@link 
OpChainSchedulerService}, waiting for the given time if
+   * there are no such OpChains ready yet. Will return null if there's no 
ready OpChains even after the specified time.
+   *
+   * @param time non-negative value that determines the time the scheduler 
will wait for new OpChains to be ready.
+   * @param timeUnit TimeUnit for the await time.
+   * @return a non-null OpChain that's ready to be run, or null if there's no 
OpChain ready even after waiting for the
+   *         given time.
+   * @throws InterruptedException if the wait for a ready OpChain was 
interrupted.
    */
-  OpChain next();
+  OpChain next(long time, TimeUnit timeUnit)

Review Comment:
   This will return null if no OpChain is ready to run right now. There are 
many cases where this may be true: all op-chains are already running, no 
op-chain is registered, etc.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to