Interface Awaitable
-
- All Known Subinterfaces:
Condition
,ConditionAsConsumer<T>
,CountDownLatch
,Future<V>
,Promise<V>
,RunnableFuture<V>
,WaitQueue.Signal
- All Known Implementing Classes:
AbstractFuture
,AsymmetricRemoteSyncTask
,AsyncChannelPromise
,AsyncFuture
,AsyncOneResponse
,AsyncPromise
,AsyncPromise.WithExecutor
,Awaitable.AbstractAwaitable
,Awaitable.AsyncAwaitable
,Awaitable.SyncAwaitable
,BlockingPartitionRepair
,Condition.Async
,Condition.Sync
,ConditionAsConsumer.Async
,CountDownLatch.Async
,CountDownLatch.Sync
,FutureCombiner
,FutureTask
,FutureTaskWithResources
,ImmediateFuture
,LocalSyncTask
,PaxosCleanup
,PaxosCleanupComplete
,PaxosCleanupLocalCoordinator
,PaxosCleanupSession
,PaxosFinishPrepareCleanup
,PaxosStartPrepareCleanup
,RepairJob
,RepairSession
,Scheduler.Task
,SnapshotTask
,StreamResultFuture
,SymmetricRemoteSyncTask
,SyncFuture
,SyncFutureTask
,SyncPromise
,SyncPromise.WithExecutor
,SyncTask
,ValidationTask
,WaitQueue.Standard.AbstractSignal
public interface Awaitable
A generic signal consumer, supporting all of the typical patterns used in Cassandra. All of the methods defined inAwaitable
may be waited on without a loop, as this interface declares that there are no spurious wake-ups.
-
-
Nested Class Summary
Nested Classes Modifier and Type Interface Description static class
Awaitable.AbstractAwaitable
static class
Awaitable.AsyncAwaitable
A barebones asynchronousAwaitable
.static class
Awaitable.Defaults
static class
Awaitable.SyncAwaitable
A barebonesAwaitable
that uses mutual exclusion.
-
Method Summary
All Methods Instance Methods Abstract Methods Modifier and Type Method Description Awaitable
await()
Await indefinitely, throwing any interrupt.boolean
await(long time, java.util.concurrent.TimeUnit units)
Await for the specified period, throwing any interrupt.Awaitable
awaitThrowUncheckedOnInterrupt()
Await indefinitely, throwing any interrupt as an unchecked exception.boolean
awaitThrowUncheckedOnInterrupt(long time, java.util.concurrent.TimeUnit units)
Await for the specified period, throwing any interrupt as an unchecked exception.Awaitable
awaitUninterruptibly()
Await indefinitely, ignoring interrupts (but maintaining the interrupt flag on exit).boolean
awaitUninterruptibly(long time, java.util.concurrent.TimeUnit units)
Await until the deadline (in nanoTime), ignoring interrupts (but maintaining the interrupt flag on exit).boolean
awaitUntil(long nanoTimeDeadline)
Await until the deadline (in nanoTime), throwing any interrupt.boolean
awaitUntilThrowUncheckedOnInterrupt(long nanoTimeDeadline)
Await until the deadline (in nanoTime), throwing any interrupt as an unchecked exception.boolean
awaitUntilUninterruptibly(long nanoTimeDeadline)
Await until the deadline (in nanoTime), ignoring interrupts (but maintaining the interrupt flag on exit).
-
-
-
Method Detail
-
awaitUntil
boolean awaitUntil(long nanoTimeDeadline) throws java.lang.InterruptedException
Await until the deadline (in nanoTime), throwing any interrupt. No spurious wakeups.- Returns:
- true if we were signalled, false if the deadline elapsed
- Throws:
java.lang.InterruptedException
- if interrupted
-
awaitUntilThrowUncheckedOnInterrupt
boolean awaitUntilThrowUncheckedOnInterrupt(long nanoTimeDeadline) throws UncheckedInterruptedException
Await until the deadline (in nanoTime), throwing any interrupt as an unchecked exception. No spurious wakeups.- Returns:
- true if we were signalled, false if the deadline elapsed
- Throws:
UncheckedInterruptedException
- if interrupted
-
awaitUntilUninterruptibly
boolean awaitUntilUninterruptibly(long nanoTimeDeadline)
Await until the deadline (in nanoTime), ignoring interrupts (but maintaining the interrupt flag on exit). No spurious wakeups.- Returns:
- true if we were signalled, false if the deadline elapsed
-
await
boolean await(long time, java.util.concurrent.TimeUnit units) throws java.lang.InterruptedException
Await for the specified period, throwing any interrupt. No spurious wakeups.- Returns:
- true if we were signalled, false if the timeout elapses
- Throws:
java.lang.InterruptedException
- if interrupted
-
awaitThrowUncheckedOnInterrupt
boolean awaitThrowUncheckedOnInterrupt(long time, java.util.concurrent.TimeUnit units) throws UncheckedInterruptedException
Await for the specified period, throwing any interrupt as an unchecked exception. No spurious wakeups.- Returns:
- true if we were signalled, false if the timeout elapses
- Throws:
UncheckedInterruptedException
- if interrupted
-
awaitUninterruptibly
boolean awaitUninterruptibly(long time, java.util.concurrent.TimeUnit units)
Await until the deadline (in nanoTime), ignoring interrupts (but maintaining the interrupt flag on exit). No spurious wakeups.- Returns:
- true if we were signalled, false if the timeout elapses
-
await
Awaitable await() throws java.lang.InterruptedException
Await indefinitely, throwing any interrupt. No spurious wakeups.- Throws:
java.lang.InterruptedException
- if interrupted
-
awaitThrowUncheckedOnInterrupt
Awaitable awaitThrowUncheckedOnInterrupt() throws UncheckedInterruptedException
Await indefinitely, throwing any interrupt as an unchecked exception. No spurious wakeups.- Throws:
UncheckedInterruptedException
- if interrupted
-
awaitUninterruptibly
Awaitable awaitUninterruptibly()
Await indefinitely, ignoring interrupts (but maintaining the interrupt flag on exit). No spurious wakeups.
-
-