Interface TaskMailbox

  • All Known Implementing Classes:
    TaskMailboxImpl

    @Internal
    public interface TaskMailbox
    A task mailbox provides read and write access to a mailbox and has a lifecycle of open -> (quiesced) -> closed. Mails have a priority that can be used to retrieve only relevant letters.

    Threading model

    The mailbox is bound to a mailbox thread passed during creation. Most operations may only occur through that thread. Write operations (put(Mail), putFirst(Mail)) can be executed by any thread. All other methods can only be invoked by the mailbox thread, which is passed upon construction. To verify that the current thread is allowed to take any mail, use isMailboxThread(), but all methods will perform the check themselves and fail accordingly if called from another thread.

    Life cycle

    In the open state, the mailbox supports put and take operations. In the quiesced state, the mailbox supports only take operations.

    Batch

    A batch is a local view on the mailbox that does not contain simultaneously added mails similar to iterators of copy-on-write collections.

    A batch serves two purposes: it reduces synchronization if more than one mail is processable at the time of the creation of a batch. Furthermore, it allows to divide the work of a mailbox in smaller logical chunks, such that the task threads cannot be blocked by a mail that enqueues itself and thus provides input starvation.

    A batch is created with createBatch() and consumed with tryTakeFromBatch(). Note that there is no blocking takeFromBatch as batches can only be created and consumed from the mailbox thread.

    Also note that a batch can only be created in the MailboxProcessor.runMailboxLoop(). A batch must not be extended in any of the consuming methods as we may run into task input starvation again. Assume a case where the mailbox loop handles a timeout event that produces a record. That record is subsequently handled downstream, where it may lead to a MailboxExecutor.yield() triggering another consumption method. If we extend the batch in any way during that processing, we effectively lose the bounded processing guarantee of mails inside the mailbox loop.

    • Field Summary

      Fields 
      Modifier and Type Field Description
      static int MAX_PRIORITY
      The maximal priority for mails.
      static int MIN_PRIORITY
      The minimal priority for mails.
    • Method Summary

      All Methods Instance Methods Abstract Methods 
      Modifier and Type Method Description
      List<Mail> close()
      Close the mailbox.
      boolean createBatch()
      Creates a batch of mails that can be taken with tryTakeFromBatch().
      List<Mail> drain()
      Drains the mailbox and returns all mails that were still enqueued.
      TaskMailbox.State getState()
      Returns the current state of the mailbox as defined by the lifecycle enum TaskMailbox.State.
      boolean hasMail()
      Returns true if the mailbox contains mail.
      boolean isMailboxThread()
      Check if the current thread is the mailbox thread.
      void put​(Mail mail)
      Enqueues the given mail to the mailbox and blocks until there is capacity for a successful put.
      void putFirst​(Mail mail)
      Adds the given action to the head of the mailbox.
      void quiesce()
      Quiesce the mailbox.
      void runExclusively​(Runnable runnable)
      Runs the given code exclusively on this mailbox.
      int size()
      Returns the current number of mails in this mailbox.
      Mail take​(int priority)
      This method returns the oldest mail from the mailbox (head of queue) or blocks until a mail is available.
      Optional<Mail> tryTake​(int priority)
      Returns an optional with either the oldest mail from the mailbox (head of queue) if the mailbox is not empty or an empty optional otherwise.
      Optional<Mail> tryTakeFromBatch()
      Returns an optional with either the oldest mail from the batch (head of queue) if the batch is not empty or an empty optional otherwise.
    • Field Detail

      • MIN_PRIORITY

        static final int MIN_PRIORITY
        The minimal priority for mails. The priority is used when no operator is associated with the mail.
        See Also:
        Constant Field Values
      • MAX_PRIORITY

        static final int MAX_PRIORITY
        The maximal priority for mails. This priority indicates that the message should be performed before any mail associated with an operator.
        See Also:
        Constant Field Values
    • Method Detail

      • isMailboxThread

        boolean isMailboxThread()
        Check if the current thread is the mailbox thread.

        Read operations will fail if they are called from another thread.

        Returns:
        only true if called from the mailbox thread.
      • hasMail

        boolean hasMail()
        Returns true if the mailbox contains mail.

        Must be called from the mailbox thread (isMailboxThread().

      • tryTake

        Optional<Mail> tryTake​(int priority)
        Returns an optional with either the oldest mail from the mailbox (head of queue) if the mailbox is not empty or an empty optional otherwise.

        Must be called from the mailbox thread (isMailboxThread().

        Returns:
        an optional with either the oldest mail from the mailbox (head of queue) if the mailbox is not empty or an empty optional otherwise.
        Throws:
        IllegalStateException - if mailbox is already closed.
      • createBatch

        boolean createBatch()
        Creates a batch of mails that can be taken with tryTakeFromBatch(). The batch does not affect tryTake(int) and take(int); that is, they return the same mails even if no batch had been created.

        The default batch is empty. Thus, this method must be invoked once before tryTakeFromBatch().

        If a batch is not completely consumed by tryTakeFromBatch(), its elements are carried over to the new batch.

        Must be called from the mailbox thread (isMailboxThread().

        Returns:
        true if there is at least one element in the batch; that is, if there is any mail at all at the time of the invocation.
      • tryTakeFromBatch

        Optional<Mail> tryTakeFromBatch()
        Returns an optional with either the oldest mail from the batch (head of queue) if the batch is not empty or an empty optional otherwise.

        Must be called from the mailbox thread (isMailboxThread().

        Note that there is no blocking takeFromBatch as batches can only be created and consumed from the mailbox thread.

        Returns:
        an optional with either the oldest mail from the batch (head of queue) if the batch is not empty or an empty optional otherwise.
        Throws:
        TaskMailbox.MailboxClosedException - if mailbox is already closed.
      • put

        void put​(Mail mail)
        Enqueues the given mail to the mailbox and blocks until there is capacity for a successful put.

        Mails can be added from any thread.

        Parameters:
        mail - the mail to enqueue.
        Throws:
        TaskMailbox.MailboxClosedException - if the mailbox is quiesced or closed.
      • putFirst

        void putFirst​(Mail mail)
        Adds the given action to the head of the mailbox.

        Mails can be added from any thread.

        Parameters:
        mail - the mail to enqueue.
        Throws:
        TaskMailbox.MailboxClosedException - if the mailbox is quiesced or closed.
      • drain

        List<Mail> drain()
        Drains the mailbox and returns all mails that were still enqueued.
        Returns:
        list with all mails that where enqueued in the mailbox.
      • quiesce

        void quiesce()
        Quiesce the mailbox. In this state, the mailbox supports only take operations and all pending and future put operations will throw TaskMailbox.MailboxClosedException.
      • close

        @Nonnull
        List<Mail> close()
        Close the mailbox. In this state, all pending and future put operations and all pending and future take operations will throw TaskMailbox.MailboxClosedException. Returns all mails that were still enqueued.
        Returns:
        list with all mails that where enqueued in the mailbox at the time of closing.
      • getState

        @Nonnull
        TaskMailbox.State getState()
        Returns the current state of the mailbox as defined by the lifecycle enum TaskMailbox.State.
        Returns:
        the current state of the mailbox.
      • size

        int size()
        Returns the current number of mails in this mailbox. (This includes mails in the batch not processed yet.)
        Returns:
        number of mails in the mailbox.
      • runExclusively

        void runExclusively​(Runnable runnable)
        Runs the given code exclusively on this mailbox. No synchronized operations can be run concurrently to the given runnable (e.g., put(Mail) or modifying lifecycle methods).

        Use this methods when you want to atomically execute code that uses different methods (e.g., check for state and then put message if open).

        Parameters:
        runnable - the runnable to execute