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, useisMailboxThread()
, 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 withtryTakeFromBatch()
. Note that there is no blockingtakeFromBatch
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 aMailboxExecutor.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.
-
-
Nested Class Summary
Nested Classes Modifier and Type Interface Description static class
TaskMailbox.MailboxClosedException
Exception thrown whenTaskMailbox
is closed.static class
TaskMailbox.State
This enum represents the states of the mailbox lifecycle.
-
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 withtryTakeFromBatch()
.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 enumTaskMailbox.State
.boolean
hasMail()
Returnstrue
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()
Returnstrue
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.
-
take
@Nonnull Mail take(int priority) throws InterruptedException
This method returns the oldest mail from the mailbox (head of queue) or blocks until a mail is available.Must be called from the mailbox thread (
isMailboxThread()
.- Returns:
- the oldest mail from the mailbox (head of queue).
- Throws:
InterruptedException
- on interruption.IllegalStateException
- if mailbox is already closed.
-
createBatch
boolean createBatch()
Creates a batch of mails that can be taken withtryTakeFromBatch()
. The batch does not affecttryTake(int)
andtake(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 throwTaskMailbox.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 throwTaskMailbox.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 enumTaskMailbox.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
-
-