+#define DISPATCH_QUEUE_CACHELINE_PAD \
+ DISPATCH_CACHELINE_PAD_SIZE(struct _dispatch_unpadded_queue_s)
+
+#define DISPATCH_QUEUE_CACHELINE_PADDING \
+ char _dq_pad[DISPATCH_QUEUE_CACHELINE_PAD]
+
+/*
+ * dispatch queues `dq_state` demystified
+ *
+ *******************************************************************************
+ *
+ * Most Significant 32 bit Word
+ * ----------------------------
+ *
+ * sc: suspend count (bits 63 - 58)
+ * The suspend count unsurprisingly holds the suspend count of the queue
+ * Only 7 bits are stored inline. Extra counts are transfered in a side
+ * suspend count and when that has happened, the ssc: bit is set.
+ */
+#define DISPATCH_QUEUE_SUSPEND_INTERVAL 0x0400000000000000ull
+#define DISPATCH_QUEUE_SUSPEND_HALF 0x20u
+/*
+ * ssc: side suspend count (bit 57)
+ * This bit means that the total suspend count didn't fit in the inline
+ * suspend count, and that there are additional suspend counts stored in the
+ * `dq_side_suspend_cnt` field.
+ */
+#define DISPATCH_QUEUE_HAS_SIDE_SUSPEND_CNT 0x0200000000000000ull
+/*
+ * i: inactive bit (bit 56)
+ * This bit means that the object is inactive (see dispatch_activate)
+ */
+#define DISPATCH_QUEUE_INACTIVE 0x0100000000000000ull
+/*
+ * na: needs activation (bit 55)
+ * This bit is set if the object is created inactive. It tells
+ * dispatch_queue_wakeup to perform various tasks at first wakeup.
+ *
+ * This bit is cleared as part of the first wakeup. Having that bit prevents
+ * the object from being woken up (because _dq_state_should_wakeup will say
+ * no), except in the dispatch_activate/dispatch_resume codepath.
+ */
+#define DISPATCH_QUEUE_NEEDS_ACTIVATION 0x0080000000000000ull
+/*
+ * This mask covers the suspend count (sc), side suspend count bit (ssc),
+ * inactive (i) and needs activation (na) bits
+ */
+#define DISPATCH_QUEUE_SUSPEND_BITS_MASK 0xff80000000000000ull
+/*
+ * ib: in barrier (bit 54)
+ * This bit is set when the queue is currently executing a barrier
+ */
+#define DISPATCH_QUEUE_IN_BARRIER 0x0040000000000000ull
+/*
+ * qf: queue full (bit 53)
+ * This bit is a subtle hack that allows to check for any queue width whether
+ * the full width of the queue is used or reserved (depending on the context)
+ * In other words that the queue has reached or overflown its capacity.
+ */
+#define DISPATCH_QUEUE_WIDTH_FULL_BIT 0x0020000000000000ull
+#define DISPATCH_QUEUE_WIDTH_FULL 0x1000ull
+#define DISPATCH_QUEUE_WIDTH_POOL (DISPATCH_QUEUE_WIDTH_FULL - 1)
+#define DISPATCH_QUEUE_WIDTH_MAX (DISPATCH_QUEUE_WIDTH_FULL - 2)
+#define DISPATCH_QUEUE_USES_REDIRECTION(width) \
+ ({ uint16_t _width = (width); \
+ _width > 1 && _width < DISPATCH_QUEUE_WIDTH_POOL; })
+/*
+ * w: width (bits 52 - 41)
+ * This encodes how many work items are in flight. Barriers hold `dq_width`
+ * of them while they run. This is encoded as a signed offset with respect,
+ * to full use, where the negative values represent how many available slots
+ * are left, and the positive values how many work items are exceeding our
+ * capacity.
+ *
+ * When this value is positive, then `wo` is always set to 1.
+ */
+#define DISPATCH_QUEUE_WIDTH_INTERVAL 0x0000020000000000ull
+#define DISPATCH_QUEUE_WIDTH_MASK 0x003ffe0000000000ull
+#define DISPATCH_QUEUE_WIDTH_SHIFT 41
+/*
+ * pb: pending barrier (bit 40)
+ * Drainers set this bit when they couldn't run the next work item and it is
+ * a barrier. When this bit is set, `dq_width - 1` work item slots are
+ * reserved so that no wakeup happens until the last work item in flight
+ * completes.
+ */
+#define DISPATCH_QUEUE_PENDING_BARRIER 0x0000010000000000ull
+/*
+ * d: dirty bit (bit 39)
+ * This bit is set when a queue transitions from empty to not empty.
+ * This bit is set before dq_items_head is set, with appropriate barriers.
+ * Any thread looking at a queue head is responsible for unblocking any
+ * dispatch_*_sync that could be enqueued at the beginning.
+ *
+ * Drainer perspective
+ * ===================
+ *
+ * When done, any "Drainer", in particular for dispatch_*_sync() handoff
+ * paths, exits in 3 steps, and the point of the DIRTY bit is to make
+ * the Drainers take the slowpath at step 2 to take into account enqueuers
+ * that could have made the queue non idle concurrently.
+ *
+ * <code>
+ * // drainer-exit step 1
+ * if (slowpath(dq->dq_items_tail)) { // speculative test
+ * return handle_non_empty_queue_or_wakeup(dq);
+ * }
+ * // drainer-exit step 2
+ * if (!_dispatch_queue_drain_try_unlock(dq, ${owned}, ...)) {
+ * return handle_non_empty_queue_or_wakeup(dq);
+ * }
+ * // drainer-exit step 3
+ * // no need to wake up the queue, it's really empty for sure
+ * return;
+ * </code>
+ *
+ * The crux is _dispatch_queue_drain_try_unlock(), it is a function whose
+ * contract is to release everything the current thread owns from the queue
+ * state, so that when it's successful, any other thread can acquire
+ * width from that queue.
+ *
+ * But, that function must fail if it sees the DIRTY bit set, leaving
+ * the state untouched. Leaving the state untouched is vital as it ensures
+ * that no other Slayer^WDrainer can rise at the same time, because the
+ * resource stays locked.
+ *
+ *
+ * Note that releasing the DRAIN_LOCK or ENQUEUE_LOCK (see below) currently
+ * doesn't use that pattern, and always tries to requeue. It isn't a problem
+ * because while holding either of these locks prevents *some* sync (the
+ * barrier one) codepaths to acquire the resource, the retry they perform
+ * at their step D (see just below) isn't affected by the state of these bits
+ * at all.
+ *
+ *
+ * Sync items perspective
+ * ======================
+ *
+ * On the dispatch_*_sync() acquire side, the code must look like this:
+ *
+ * <code>
+ * // step A
+ * if (try_acquire_sync(dq)) {
+ * return sync_operation_fastpath(dq, item);
+ * }
+ *
+ * // step B
+ * if (queue_push_and_inline(dq, item)) {
+ * atomic_store(dq->dq_items_head, item, relaxed);
+ * // step C
+ * atomic_or(dq->dq_state, DIRTY, release);
+ *
+ * // step D
+ * if (try_acquire_sync(dq)) {
+ * try_lock_transfer_or_wakeup(dq);
+ * }
+ * }
+ *
+ * // step E
+ * wait_for_lock_transfer(dq);
+ * </code>
+ *
+ * A. If this code can acquire the resource it needs at step A, we're good.
+ *
+ * B. If the item isn't the first at enqueue time, then there is no issue
+ * At least another thread went through C, this thread isn't interesting
+ * for the possible races, responsibility to make progress is transfered
+ * to the thread which went through C-D.
+ *
+ * C. The DIRTY bit is set with a release barrier, after the head/tail
+ * has been set, so that seeing the DIRTY bit means that head/tail
+ * will be visible to any drainer that has the matching acquire barrier.
+ *
+ * Drainers may see the head/tail and fail to see DIRTY, in which
+ * case, their _dispatch_queue_drain_try_unlock() will clear the DIRTY
+ * bit, and fail, causing the caller to retry exactly once.
+ *
+ * D. At this stage, there's two possible outcomes:
+ *
+ * - either the acquire works this time, in which case this thread
+ * successfuly becomes a drainer. That's obviously the happy path.
+ * It means all drainers are after Step 2 (or there is no Drainer)
+ *
+ * - or the acquire fails, which means that another drainer is before
+ * its Step 2. Since we set the DIRTY bit on the dq_state by now,
+ * and that drainers manipulate the state atomically, at least one
+ * drainer that is still before its step 2 will fail its step 2, and
+ * be responsible for making progress.
+ *
+ *
+ * Async items perspective
+ * ======================
+ *
+ * On the async codepath, when the queue becomes non empty, the queue
+ * is always woken up. There is no point in trying to avoid that wake up
+ * for the async case, because it's required for the async()ed item to make
+ * progress: a drain of the queue must happen.
+ *
+ * So on the async "acquire" side, there is no subtlety at all.
+ */
+#define DISPATCH_QUEUE_DIRTY 0x0000008000000000ull
+/*
+ * md: enqueued/draining on manager (bit 38)
+ * Set when enqueued and draining on the manager hierarchy.
+ *
+ * Unlike the ENQUEUED bit, it is kept until the queue is unlocked from its
+ * invoke call on the manager. This is used to prevent stealing, and
+ * overrides to be applied down the target queue chain.
+ */
+#define DISPATCH_QUEUE_ENQUEUED_ON_MGR 0x0000004000000000ull
+/*
+ * r: queue graph role (bits 37 - 36)
+ * Queue role in the target queue graph
+ *
+ * 11: unused
+ * 10: WLH base
+ * 01: non wlh base
+ * 00: inner queue
+ */
+#define DISPATCH_QUEUE_ROLE_MASK 0x0000003000000000ull
+#define DISPATCH_QUEUE_ROLE_BASE_WLH 0x0000002000000000ull
+#define DISPATCH_QUEUE_ROLE_BASE_ANON 0x0000001000000000ull
+#define DISPATCH_QUEUE_ROLE_INNER 0x0000000000000000ull
+/*
+ * o: has override (bit 35, if role is DISPATCH_QUEUE_ROLE_BASE_ANON)
+ * Set when a queue has received a QOS override and needs to reset it.
+ * This bit is only cleared when the final drain_try_unlock() succeeds.
+ *
+ * sw: has received sync wait (bit 35, if role DISPATCH_QUEUE_ROLE_BASE_WLH)
+ * Set when a queue owner has been exposed to the kernel because of
+ * dispatch_sync() contention.
+ */
+#define DISPATCH_QUEUE_RECEIVED_OVERRIDE 0x0000000800000000ull
+#define DISPATCH_QUEUE_RECEIVED_SYNC_WAIT 0x0000000800000000ull
+/*
+ * max_qos: max qos (bits 34 - 32)
+ * This is the maximum qos that has been enqueued on the queue
+ */
+#define DISPATCH_QUEUE_MAX_QOS_MASK 0x0000000700000000ull
+#define DISPATCH_QUEUE_MAX_QOS_SHIFT 32
+/*
+ * dl: drain lock (bits 31-0)
+ * This is used by the normal drain to drain exlusively relative to other
+ * drain stealers (like the QoS Override codepath). It holds the identity
+ * (thread port) of the current drainer.
+ *
+ * st: sync transfer (bit 1 or 30)
+ * Set when a dispatch_sync() is transferred to
+ *
+ * e: enqueued bit (bit 0 or 31)
+ * Set when a queue is enqueued on its target queue
+ */
+#define DISPATCH_QUEUE_DRAIN_OWNER_MASK ((uint64_t)DLOCK_OWNER_MASK)
+#define DISPATCH_QUEUE_SYNC_TRANSFER ((uint64_t)DLOCK_FAILED_TRYLOCK_BIT)
+#define DISPATCH_QUEUE_ENQUEUED ((uint64_t)DLOCK_WAITERS_BIT)
+
+#define DISPATCH_QUEUE_DRAIN_PRESERVED_BITS_MASK \
+ (DISPATCH_QUEUE_ENQUEUED_ON_MGR | DISPATCH_QUEUE_ENQUEUED | \
+ DISPATCH_QUEUE_ROLE_MASK | DISPATCH_QUEUE_MAX_QOS_MASK)
+
+#define DISPATCH_QUEUE_DRAIN_UNLOCK_MASK \
+ (DISPATCH_QUEUE_DRAIN_OWNER_MASK | DISPATCH_QUEUE_RECEIVED_OVERRIDE | \
+ DISPATCH_QUEUE_RECEIVED_SYNC_WAIT | DISPATCH_QUEUE_SYNC_TRANSFER)
+
+/*
+ *******************************************************************************
+ *
+ * `Drainers`
+ *
+ * Drainers are parts of the code that hold the drain lock by setting its value
+ * to their thread port. There are two kinds:
+ * 1. async drainers,
+ * 2. lock transfer handlers.
+ *
+ * Drainers from the first category are _dispatch_queue_class_invoke and its
+ * stealers. Those drainers always try to reserve width at the same time they
+ * acquire the drain lock, to make sure they can make progress, and else exit
+ * quickly.
+ *
+ * Drainers from the second category are `slow` work items. Those run on the
+ * calling thread, and when done, try to transfer the width they own to the
+ * possible next `slow` work item, and if there is no such item, they reliquish
+ * that right. To do so, prior to taking any decision, they also try to own
+ * the full "barrier" width on the given queue.
+ *
+ *******************************************************************************
+ *
+ * Enqueuing and wakeup rules
+ *
+ * Nobody should enqueue any dispatch object if it has no chance to make any
+ * progress. That means that queues that:
+ * - are suspended
+ * - have reached or overflown their capacity
+ * - are currently draining
+ * - are already enqueued
+ *
+ * should not try to be enqueued.
+ *
+ *******************************************************************************
+ *
+ * Lock transfer
+ *
+ * The point of the lock transfer code is to allow pure dispatch_*_sync()
+ * callers to make progress without requiring the bring up of a drainer.
+ * There are two reason for that:
+ *
+ * - performance, as draining has to give up for dispatch_*_sync() work items,
+ * so waking up a queue for this is wasteful.
+ *
+ * - liveness, as with dispatch_*_sync() you burn threads waiting, you're more
+ * likely to hit various thread limits and may not have any drain being
+ * brought up if the process hits a limit.
+ *
+ *
+ * Lock transfer happens at the end on the dispatch_*_sync() codepaths:
+ *
+ * - obviously once a dispatch_*_sync() work item finishes, it owns queue
+ * width and it should try to transfer that ownership to the possible next
+ * queued item if it is a dispatch_*_sync() item
+ *
+ * - just before such a work item blocks to make sure that that work item
+ * itself isn't its own last chance to be woken up. That can happen when
+ * a Drainer pops up everything from the queue, and that a dispatch_*_sync()
+ * work item has taken the slow path then was preempted for a long time.
+ *
+ * That's why such work items, if first in the queue, must try a lock
+ * transfer procedure.
+ *
+ *
+ * For transfers where a partial width is owned, we give back that width.
+ * If the queue state is "idle" again, we attempt to acquire the full width.
+ * If that succeeds, this falls back to the full barrier lock
+ * transfer, else it wakes up the queue according to its state.
+ *
+ * For full barrier transfers, if items eligible for lock transfer are found,
+ * then they are woken up and the lock transfer is successful.
+ *
+ * If none are found, the full barrier width is released. If by doing so the
+ * DIRTY bit is found, releasing the full barrier width fails and transferring
+ * the lock is retried from scratch.
+ */
+
+#define DISPATCH_QUEUE_STATE_INIT_VALUE(width) \
+ ((DISPATCH_QUEUE_WIDTH_FULL - (width)) << DISPATCH_QUEUE_WIDTH_SHIFT)
+
+/* Magic dq_state values for global queues: they have QUEUE_FULL and IN_BARRIER
+ * set to force the slowpath in both dispatch_barrier_sync() and dispatch_sync()
+ */
+#define DISPATCH_ROOT_QUEUE_STATE_INIT_VALUE \
+ (DISPATCH_QUEUE_WIDTH_FULL_BIT | DISPATCH_QUEUE_IN_BARRIER)
+
+#define DISPATCH_QUEUE_SERIAL_DRAIN_OWNED \
+ (DISPATCH_QUEUE_IN_BARRIER | DISPATCH_QUEUE_WIDTH_INTERVAL)