16 KiB
Druid's Checklist for Concurrency Code
Design
- Concurrency is rationalized in the PR description?
- Can use patterns to simplify concurrency?
- Immutability/Snapshotting
- Divide and conquer
- Producer-consumer
- Instance confinement
- Thread/Task/Serial thread confinement
- Active object
Documentation
- Thread safety is justified in comments?
- Class (method, field) has concurrent access documentation?
- Threading model of a subsystem (class) is described?
- Concurrent control flow (or data flow) of a subsystem (class) is described?
- Class is documented as immutable, thread-safe, or not thread-safe?
- Used concurrency patterns are pronounced?
@GuardedBy
annotation is used?- Safety of a benign race (e. g. unbalanced synchronization) is explained?
- Each use of
volatile
is justified? - Each field that is neither
volatile
nor annotated with@GuardedBy
has a comment?
Insufficient synchronization
- Static methods and fields are thread-safe?
- Thread doesn't wait in a loop for a non-volatile field to be updated by another thread?
- Read access to a non-volatile, concurrently updatable primitive field is protected?
@GET
/@POST
methods, Jetty Filters and Handlers are thread-safe?- Calls to
DateFormat.parse()
andformat()
are synchronized?
Excessive thread safety
- No "extra" (pseudo) thread safety?
- No atomics on which only
get()
andset()
are called? - Class (method) needs to be thread-safe?
ReentrantLock
(ReentrantReadWriteLock
,Semaphore
) needs to be fair?
Race conditions
- No
put()
orremove()
calls on aConcurrentMap
(or Cache) afterget()
orcontainsKey()
? - No point accesses to a non-thread-safe collection outside of critical sections?
- Iteration over a non-thread-safe collection doesn't leak outside of a critical section?
- Non-trivial object is not returned from a getter in a thread-safe class?
- No separate getters to an atomically updated state?
- No check-then-act race conditions (state used inside a critical section is read outside of it)?
coll.toArray(new E[coll.size()])
is not called on a synchronized collection?- No race conditions with user or programmatic input or interop between programs?
- No check-then-act race conditions with file system operations?
- No concurrent
invalidate(key)
andget()
calls on Guava's loadingCache
? Cache.put()
is not used (nor exposed in the own Cache interface)?- Concurrent invalidation race is not possible on a lazily initialized state?
- Iteration, Stream pipeline, or copying a
Collections.synchronized*()
collection is protected by a lock?
Testing
- Unit tests for thread-safe classes are multi-threaded?
- A shared
Random
instance is not used from concurrent test workers? - Concurrent test workers coordinate their start?
- There are more test threads than CPUs (if possible for the test)?
- Assertions in parallel threads and asynchronous code are handled properly?
Locks
- Can use
LifecycleLock
instead of a standard lock in a lifecycled object? - Can use some concurrency utility instead of a lock with conditional
wait
(await
) calls? - Can use Guava’s
Monitor
instead of a lock with conditionalwait
(await
) calls? - Can use
synchronized
instead of aReentrantLock
? lock()
is called outside oftry {}
? No statements betweenlock()
andtry {}
?
Avoiding deadlocks
- Can avoid nested critical sections?
- Locking order for nested critical sections is documented?
- Dynamically determined locks for nested critical sections are ordered?
- No extension API calls within critical sections?
- No calls to
ConcurrentHashMap
's methods (incl.get()
) incompute()
-like lambdas on the same map?
Improving scalability
- Critical section is as small as possible?
- Can use
ConcurrentHashMap.compute()
or Guava'sStriped
for per-key locking? - Can replace blocking collection or a queue with a concurrent one?
- Can use
ClassValue
instead ofConcurrentHashMap<Class, ...>
? - Considered
ReadWriteLock
(orStampedLock
) instead of a simple lock? StampedLock
is used instead ofReadWriteLock
when reentrancy is not needed?- Considered
LongAdder
instead of anAtomicLong
for a "hot field"? - Considered queues from JCTools instead of the standard concurrent queues?
- Caffeine cache is used instead of Guava?
- Can apply speculation (optimistic concurrency) technique?
Lazy initialization and double-checked locking
- Lazy initialization of a field should be thread-safe?
- Considered double-checked locking for a lazy initialization to improve performance?
- Double-checked locking follows the SafeLocalDCL pattern?
- Considered eager initialization instead of a lazy initialization to simplify code?
- Can do lazy initialization with a benign race and without locking to improve performance?
- Holder class idiom is used for lazy static fields rather than double-checked locking?
Non-blocking and partially blocking code
- Non-blocking code has enough comments to make line-by-line checking as easy as possible?
- Can use immutable POJO + compare-and-swap operations to simplify non-blocking code?
- Boundaries of non-blocking or benignly racy code are identified with WARNING comments?
Threads and Executors
- Thread is named?
- Thread is daemon?
- Using
Execs
to create anExecutorService
? - Can use
ExecutorService
instead of creating a newThread
each time some method is called? - No network I/O in a CachedThreadPool?
- No blocking (incl. I/O) operations in a
ForkJoinPool
or in a parallel Stream pipeline? - Can execute non-blocking computation in
FJP.commonPool()
instead of a custom thread pool? ExecutorService
is shutdown explicitly?
Parallel Streams
- Parallel Stream computation takes more than 100us in total?
- Comment before a parallel Streams pipeline explains how it takes more than 100us in total?
Thread interruption and Future
cancellation
- Interruption status is restored before wrapping
InterruptedException
with another exception? InterruptedException
is swallowed only in the following kinds of methods:Runnable.run()
,Callable.call()
, or methods to be passed to executors as lambda tasks; or- Methods with "try" or "best effort" semantics?
InterruptedException
swallowing is documented for a method?- Can use Guava's
Uninterruptibles
to avoidInterruptedException
swallowing? Future
is canceled upon catching anInterruptedException
or aTimeoutException
onget()
?
Time
nanoTime()
values are compared in an overflow-aware manner?currentTimeMillis()
is not used to measure time intervals and timeouts?- Units for a time variable are identified in the variable's name or via
TimeUnit
? - Negative timeouts and delays are treated as zeros?
Thread safety of Cleaners and native code
close()
is concurrently idempotent in a class with aCleaner
orfinalize()
?- Method accessing native state calls
reachabilityFence()
in a class with aCleaner
orfinalize()
? Cleaner
orfinalize()
is used for real cleanup, not mere reporting?- Considered making a class with native state thread-safe?
# Lk.D1. Is it possible to use Druid's LifecycleLock
utility instead of a standard lock and
"started" flag in lifecycled objects with start()
and stop()
methods? See the Javadoc comment for LifecycleLock
for more details.
# TE.D1. Are Threads created directly or via a ThreadFactory
configured to be daemon via
setDaemon(true)
? Note that by default, ThreadFactories constructed via Execs.makeThreadFactory()
methods create
daemon threads already.
# TE.D2. Is it possible to use one of the static factory methods in Druid's Execs
utility class to
create an ExecutorService
instead of Java's standard ExecutorServices
? This is recommended because Execs
configure
ThreadFactories to create daemon threads by default, as required by the previous item.