Copilot commented on code in PR #16903: URL: https://github.com/apache/pinot/pull/16903#discussion_r2402251567
########## pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java: ########## @@ -315,4 +280,377 @@ public List<DataBuffer> getSerializedStats() { return _serializedStats; } } + + /// The state of the queue. + /// + /// ``` + /// +-------------------+ offerEos +-------------------+ + /// | FULL_OPEN | -----------> | UPSTREAM_FINISHED| + /// +-------------------+ +-------------------+ + /// | | + /// | earlyTerminate | poll -- when all pending data is read + /// v v + /// +-------------------+ offerEos +-------------------+ + /// | WAITING_EOS | -----------> | FULL_CLOSED | + /// +-------------------+ +-------------------+ + /// ``` + private enum State { + /// The queue is open for both read and write. + /// + /// - [#poll()] returns the pending blocks in the queue, or null if the queue is empty. + /// - [#offer] accepts both data and EOS blocks. + /// + /// Transitions to [State#UPSTREAM_FINISHED] when an EOS block is offered or to [State#WAITING_EOS] when + /// [#earlyTerminate()] is called. + FULL_OPEN, + /// The downstream is not interested in reading more data but is waiting for an EOS block to get the stats. + /// + /// - [#poll()] returns null. + /// - [#offer] rejects all data blocks. + /// + /// Transitions to [State#FULL_CLOSED] when an EOS block is offered. + WAITING_EOS, + /// The upstream has indicated that no more data will be sent. + /// + /// - [#poll()] returns the pending blocks in the queue and then the EOS block. + /// - [#offer] rejects all blocks. + /// + /// Transitions to [State#FULL_CLOSED] when the EOS block is read by [#poll()]. + UPSTREAM_FINISHED, + /// The queue is closed for both read and write. + /// + /// - [#poll()] always returns the EOS block, which is always not null. + /// - [#offer] rejects all blocks. + /// + /// No transitions out of this state. + FULL_CLOSED + } + + /// This is a special bounded blocking queue implementation similar to ArrayBlockingQueue, but: + /// - Only accepts a single reader (aka downstream). + /// - Only accepts a multiple concurrent writers (aka upstream) + /// - Can be [closed for write][#closeForWrite(MseBlock.Eos, List)]. + /// - Can be [#earlyTerminate()]d. + /// + /// Read the [State] enum to understand the different states and their transitions. + /// + /// All methods of this class are thread-safe and may block, although only [#offer] should block for a long time. + @ThreadSafe + private static class CancellableBlockingQueue { + private final String _id; + @Nullable + private volatile Reader _reader; + /// This is set when the queue is in [State#FULL_CLOSED] or [State#UPSTREAM_FINISHED]. + @Nullable + @GuardedBy("_lock") + private MseBlockWithStats _eos; + /// The current state of the queue. + /// + /// All changes to this field must be done by calling [#changeState(State, String)] in order to log the state + /// transitions. + @GuardedBy("_lock") + private State _state = State.FULL_OPEN; + /// The items in the queue. + /// + /// This is a circular array where [#_putIndex] is the index to add the next item and [#_takeIndex] is the index to + /// take the next item from. Only data blocks are stored in this array, the EOS block is stored in [#_eos]. + /// + /// Like in normal blocking queues, elements are added when upstream threads call [#offer] and removed when the + /// downstream thread calls [#poll]. Unlike normal blocking queues, elements will be [removed][#drainDataBlocks()] + /// when transitioning to [State#WAITING_EOS] or [State#FULL_CLOSED]. + @GuardedBy("_lock") + private final MseBlock.Data[] _dataBlocks; + @GuardedBy("_lock") + private int _takeIndex; + @GuardedBy("_lock") + private int _putIndex; + @GuardedBy("_lock") + private int _count; + /// Threads waiting to add more data to the queue. + /// + /// This is used to prevent the following situation: + /// 1. The queue is full. + /// 2. Thread A tries to add data. Thread A will be blocked waiting for space in the queue. + /// 3. Thread B adds an EOS block, which will transition the queue to [State#UPSTREAM_FINISHED]. + /// 4. Thread C reads data from the queue in a loop, the scheduler doesn't give time to Thread A. + /// 5. Thread C consumes all data from the queue and then reads the EOS block. + /// 6. Finally Thread A is unblocked and adds data to the queue, even though the queue is already closed for write + /// + /// As a result the block from A will be lost. Instead, we use this counter to return null in [#poll] when the + /// queue is empty but there are still threads trying to add data to the queue. + @GuardedBy("_lock") + private int _pendingData; + private final ReentrantLock _lock = new ReentrantLock(); + private final Condition _notFull = _lock.newCondition(); + + public CancellableBlockingQueue(String id, int capacity) { + _id = id; + _dataBlocks = new MseBlock.Data[capacity]; + } + + /// Notifies the downstream that there is data to read. + private void notifyReader() { + Reader reader = _reader; + if (reader != null) { + LOGGER.debug("Notifying reader"); + reader.blockReadyToRead(); + } else { + LOGGER.debug("No reader to notify"); + } + } + + /// Offers a successful or erroneous EOS block into the queue, returning the status of the operation. + /// + /// This method never blocks for long, as it doesn't need to wait for space in the queue. + public ReceivingMailboxStatus offerEos(MseBlock.Eos block, List<DataBuffer> stats) { + ReentrantLock lock = _lock; + lock.lock(); + try { + switch (_state) { + case FULL_CLOSED: + case UPSTREAM_FINISHED: + // The queue is closed for write. Always reject the block. + LOGGER.debug("Mailbox: {} is already closed for write, ignoring the late {} block", _id, block); + return ReceivingMailboxStatus.ALREADY_TERMINATED; + case WAITING_EOS: + // We got the EOS block we expected. Close the queue for both read and write. + changeState(State.FULL_CLOSED, "received EOS block"); + _eos = new MseBlockWithStats(block, stats); + notifyReader(); + return ReceivingMailboxStatus.LAST_BLOCK; + case FULL_OPEN: + changeState(State.UPSTREAM_FINISHED, "received EOS block"); + _eos = new MseBlockWithStats(block, stats); + notifyReader(); + if (block.isError()) { + drainDataBlocks(); + _notFull.signal(); + } + return ReceivingMailboxStatus.LAST_BLOCK; + default: + throw new IllegalStateException("Unexpected state: " + _state); + } + } finally { + lock.unlock(); + } + } + + /// Offers a data block into the queue within the timeout specified, returning the status of the operation. + public ReceivingMailboxStatus offerData(MseBlock.Data block, long timeout, TimeUnit timeUnit) + throws InterruptedException, TimeoutException { + ReentrantLock lock = _lock; + lock.lockInterruptibly(); + try { + while (true) { + switch (_state) { + case FULL_CLOSED: + case UPSTREAM_FINISHED: + // The queue is closed for write. Always reject the block. + LOGGER.debug("Mailbox: {} is already closed for write, ignoring the late data block", _id); + return ReceivingMailboxStatus.ALREADY_TERMINATED; + case WAITING_EOS: + // The downstream is not interested in reading more data. + LOGGER.debug("Mailbox: {} is not interesting in late data block", _id); + return ReceivingMailboxStatus.WAITING_EOS; + case FULL_OPEN: + if (offerDataToBuffer(block, timeout, timeUnit)) { + notifyReader(); + return ReceivingMailboxStatus.SUCCESS; + } + // otherwise transitioned to FULL_CLOSED or WAITING_EOS while waiting for space in the queue + // and we need to re-evaluate the state + break; + default: + throw new IllegalStateException("Unexpected state: " + _state); + } + } + } finally { + lock.unlock(); + } + } + + /// Offers a data block into the queue within the timeout specified, returning true if the block was added + /// successfully. + /// + /// This method can only be called while the queue is in the FULL_OPEN state and the lock is held. + /// + /// This method can time out, in which case we automatically transition to the [State#FULL_CLOSED] state. + /// But instead of returning false, we throw a [TimeoutException]. This is because the caller may want to + /// distinguish between a timeout and other reasons for not being able to add the block to the queue in order to + /// report different error messages. + /// + /// @return true if the block was added successfully, false if the state changed while waiting. + /// @throws InterruptedException if the thread is interrupted while waiting for space in the queue. + /// @throws TimeoutException if the timeout specified elapsed before space was available in the queue. + @GuardedBy("_lock") + private synchronized boolean offerDataToBuffer(MseBlock.Data block, long timeout, TimeUnit timeUnit) Review Comment: The method is marked as @GuardedBy(\"_lock\") but also has 'synchronized' modifier. Since the calling method already acquires the lock, the synchronized modifier is redundant and should be removed. ```suggestion private boolean offerDataToBuffer(MseBlock.Data block, long timeout, TimeUnit timeUnit) ``` ########## pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/ResourceBasedQueriesTest.java: ########## @@ -60,6 +60,7 @@ import org.apache.pinot.spi.utils.CommonConstants.MultiStageQueryRunner; import org.apache.pinot.spi.utils.JsonUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.assertj.core.api.Assertions; Review Comment: [nitpick] Consider using a static import for Assertions.assertThat() since it's only used once in the file. This would be more consistent with the existing static imports like Assert.assertTrue(). ```suggestion import static org.assertj.core.api.Assertions.assertThat; ``` ########## pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/ReceivingMailbox.java: ########## @@ -200,63 +178,34 @@ private synchronized void updateWaitCpuTime() { _lastArriveTime = now; } - /** - * Sets an error block into the mailbox. No more blocks are accepted after calling this method. - */ + /// Sets an error block into the mailbox. No more blocks are accepted after calling this method. public void setErrorBlock(ErrorMseBlock errorBlock, List<DataBuffer> serializedStats) { - if (_errorBlock.compareAndSet(null, new MseBlockWithStats(errorBlock, serializedStats))) { - _blocks.clear(); - notifyReader(); - } + _blocks.offerEos(errorBlock, serializedStats); } - /** - * Returns the first block from the mailbox, or {@code null} if there is no block received yet. Error block is - * returned if exists. - */ + /// Returns the first block from the mailbox, or {@code null} if there is no block received yet. @Nullable public MseBlockWithStats poll() { - Preconditions.checkState(_reader != null, "A reader must be registered"); - MseBlockWithStats errorBlock = _errorBlock.get(); - return errorBlock != null ? errorBlock : _blocks.poll(); + return _blocks.poll(); } - /** - * Early terminate the mailbox, called when upstream doesn't expect any more data block. - */ + /// Early terminate the mailbox, called when upstream doesn't expect any more *data* block. public void earlyTerminate() { - _isEarlyTerminated = true; + _blocks.earlyTerminate(); } - /** - * Cancels the mailbox. No more blocks are accepted after calling this method. Should only be called by the receive - * operator to clean up the remaining blocks. - */ + /// Cancels the mailbox. No more blocks are accepted after calling this method and [#poll] will always return + /// an error block. public void cancel() { LOGGER.debug("Cancelling mailbox: {}", _id); - if (_errorBlock.get() == null) { - MseBlockWithStats errorBlock = new MseBlockWithStats( - ErrorMseBlock.fromError(QueryErrorCode.EXECUTION_TIMEOUT, "Cancelled by receiver"), - Collections.emptyList()); - if (_errorBlock.compareAndSet(null, errorBlock)) { - _cancelledErrorBlock = errorBlock; - _blocks.clear(); - } - } + _blocks.offerEos(ErrorMseBlock.fromException(null), List.of()); } + /// Returns the number of pending **data** blocks in the mailbox. + /// + /// will always return the same EOS block. Review Comment: This comment fragment appears to be incomplete or misplaced. It doesn't relate to the getNumPendingBlocks() method below it and should be completed or removed. ```suggestion ``` ########## pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/utils/BlockingMultiStreamConsumer.java: ########## @@ -393,7 +395,7 @@ protected ReceivingMailbox.MseBlockWithStats onTimeout() { return onException(terminateException.getErrorCode(), terminateException.getMessage()); } // TODO: Add the sender stage id to the error message - String errMsg = "Timed out on stage " + _stageId + " waiting for data sent by a child stage"; + String errMsg = "Timed out on stage " + _stageId + " waiting for data from child stage " + _senderStageId; Review Comment: Use String.format() or StringBuilder for string concatenation involving multiple variables to improve readability and performance. ```suggestion String errMsg = String.format("Timed out on stage %d waiting for data from child stage %d", _stageId, _senderStageId); ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org