Commit 9a8218df authored by Maged Michael's avatar Maged Michael Committed by Facebook Github Bot

Unbounded queue

Summary:
UnboundedQueue supports:
- SPSC, MPSC, SCMP, MPMC
- Non-waiting, waiting, and timed consumer operations.
- Producers never wait or fail (unless out-of-memory).
- Memory usage grows and shrinks dynamically

```
/// UnboundedQueue supports a variety of options for unbounded
/// dynamically expanding an shrinking queues, including variations of:
/// - Single vs. multiple producers
/// - Single vs. multiple consumers
/// - Blocking vs. spin-waiting
/// - Non-waiting, timed, and waiting consumer operations.
/// Producer operations never wait or fail (unless out-of-memory).
///
/// Template parameters:
/// - T: element type
/// - SingleProducer: true if there can be only one producer at a
///   time.
/// - SingleConsumer: true if there can be only one consumer at a
///   time.
/// - MayBlock: true if consumers may block, false if they only
///   spins. A performance tuning parameter.
/// - LgSegmentSize (default 8): Log base 2 of number of elements per
///   segment. A performance tuning parameter. See below.
///
/// When to use UnboundedQueue:
/// - If a small bound may lead to deadlock or performance degradation
///   under bursty patterns.
/// - If there is no risk of the queue growing too much.
///
/// When not to use UnboundedQueue:
/// - If there is risk of the queue growing too much and a large bound
///   is acceptable, then use DynamicBoundedQueue.
/// - If the queue must not allocate on enqueue or it must have a
///   small bound, then use fixed-size MPMCQueue or (if non-blocking
///   SPSC) ProducerConsumerQueue.
///
/// Template Aliases:
///   USPSCQueue<T, MayBlock, LgSegmentSize>
///   UMPSCQueue<T, MayBlock, LgSegmentSize>
///   USPMCQueue<T, MayBlock, LgSegmentSize>
///   UMPMCQueue<T, MayBlock, LgSegmentSize>
///
/// Functions:
///   Producer operations never wait or fail (unless OOM)
///     void enqueue(const T&);
///     void enqueue(T&&);
///         Adds an element to the end of the queue.
///
///   Consumer operations:
///     void dequeue(T&);
///         Extracts an element from the front of the queue. Waits
///         until an element is available if needed.
///     bool try_dequeue(T&);
///         Tries to extracts an element from the front of the queue
///         if available. Returns true if successful, false otherwise.
///     bool try_dequeue_until(T&, time_point& deadline);
///         Tries to extracts an element from the front of the queue
///         if available until the specified deadline.  Returns true
///         if successful, false otherwise.
///     bool try_dequeue_for(T&, duration&);
///         Tries to extracts an element from the front of the queue
///         if available for for the specified duration.  Returns true
///         if successful, false otherwise.
///
///   Secondary functions:
///     size_t size();
///         Returns an estimate of the size of the queue.
///     bool empty();
///         Returns true only if the queue was empty during the call.
///     Note: size() and empty() are guaranteed to be accurate only if
```

Reviewed By: djwatson

Differential Revision: D6157613

fbshipit-source-id: db423f86d1d0604d22f6b9c71ea0ed08be32e2a1
parent 192d78d4
......@@ -57,6 +57,7 @@ nobase_follyinclude_HEADERS = \
concurrency/ConcurrentHashMap.h \
concurrency/CoreCachedSharedPtr.h \
concurrency/detail/ConcurrentHashMap-detail.h \
concurrency/UnboundedQueue.h \
container/Access.h \
container/Array.h \
container/Iterator.h \
......
/*
* Copyright 2017 Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include <folly/experimental/hazptr/hazptr.h>
#include <folly/lang/Launder.h>
#include <folly/synchronization/SaturatingSemaphore.h>
#include <glog/logging.h>
namespace folly {
/* constructor */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
inline UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::UnboundedQueue() {
setProducerTicket(0);
setConsumerTicket(0);
auto s = new Segment(0);
DEBUG_PRINT(s);
setTail(s);
setHead(s);
}
/* destructor */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
inline UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::~UnboundedQueue() {
Segment* next;
for (auto s = head(); s; s = next) {
next = s->nextSegment();
if (SPSC) {
delete s;
} else {
s->retire(); // hazptr
}
}
}
/* dequeue */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
FOLLY_ALWAYS_INLINE void UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::dequeue(T& item) noexcept {
if (SPSC) {
auto s = head();
dequeueCommon(s, item);
} else {
// Using hazptr_holder instead of hazptr_local because it is
// possible to call ~T() and it may happen to use hazard pointers.
folly::hazptr::hazptr_holder hptr;
auto s = hptr.get_protected(head_);
dequeueCommon(s, item);
}
}
/* try_dequeue_until */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
template <typename Clock, typename Duration>
FOLLY_ALWAYS_INLINE bool UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::
try_dequeue_until(
T& item,
const std::chrono::time_point<Clock, Duration>& deadline) noexcept {
if (SingleConsumer) {
auto s = head();
return singleConsumerTryDequeueUntil(s, item, deadline);
} else {
// Using hazptr_holder instead of hazptr_local because it is
// possible to call ~T() and it may happen to use hazard pointers.
folly::hazptr::hazptr_holder hptr;
auto s = hptr.get_protected(head_);
return multiConsumerTryDequeueUntil(s, item, deadline);
}
}
/* enqueueImpl */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
template <typename Arg>
FOLLY_ALWAYS_INLINE void UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::enqueueImpl(Arg&& arg) {
if (SPSC) {
auto s = tail();
enqueueCommon(s, std::forward<Arg>(arg));
} else {
// Using hazptr_holder instead of hazptr_local because it is
// possible that the T construcctor happens to use hazard
// pointers.
folly::hazptr::hazptr_holder hptr;
auto s = hptr.get_protected(tail_);
enqueueCommon(s, std::forward<Arg>(arg));
}
}
/* enqueueCommon */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
template <typename Arg>
FOLLY_ALWAYS_INLINE void UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::enqueueCommon(Segment* s, Arg&& arg) {
auto t = fetchIncrementProducerTicket();
if (!SingleProducer) {
s = findSegment(s, t);
}
DCHECK_GE(t, s->minTicket());
DCHECK_LT(t, (s->minTicket() + SegmentSize));
size_t idx = index(t);
Entry& e = s->entry(idx);
e.putItem(std::forward<Arg>(arg));
if (responsibleForAlloc(t)) {
allocNextSegment(s, t + SegmentSize);
}
if (responsibleForAdvance(t)) {
advanceTail(s);
}
}
/* dequeueCommon */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
FOLLY_ALWAYS_INLINE void UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::dequeueCommon(Segment* s, T& item) noexcept {
auto t = fetchIncrementConsumerTicket();
if (!SingleConsumer) {
s = findSegment(s, t);
}
size_t idx = index(t);
Entry& e = s->entry(idx);
e.takeItem(item);
if (responsibleForAdvance(t)) {
advanceHead(s);
}
}
/* singleConsumerTryDequeueUntil */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
template <typename Clock, typename Duration>
FOLLY_ALWAYS_INLINE bool UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::
singleConsumerTryDequeueUntil(
Segment* s,
T& item,
const std::chrono::time_point<Clock, Duration>& deadline) noexcept {
auto t = consumerTicket();
DCHECK_GE(t, s->minTicket());
DCHECK_LT(t, (s->minTicket() + SegmentSize));
size_t idx = index(t);
Entry& e = s->entry(idx);
if (!e.tryWaitUntil(deadline)) {
return false;
}
setConsumerTicket(t + 1);
e.takeItem(item);
if (responsibleForAdvance(t)) {
advanceHead(s);
}
return true;
}
/* multiConsumerTryDequeueUntil */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
template <typename Clock, typename Duration>
FOLLY_ALWAYS_INLINE bool UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::
multiConsumerTryDequeueUntil(
Segment* s,
T& item,
const std::chrono::time_point<Clock, Duration>& deadline) noexcept {
while (true) {
auto t = consumerTicket();
if (UNLIKELY(t >= (s->minTicket() + SegmentSize))) {
Segment* next;
// Note that the following loop will not spin indefinitely (as
// long as the number of concurrently waiting consumers is not
// greater than SegmentSize). The algorithm guarantees in such a
// case that the producer reponsible for setting the next
// pointer is already running.
while ((next = s->nextSegment()) == nullptr) {
if (Clock::now() > deadline) {
return false;
}
asm_volatile_pause();
}
s = next;
DCHECK(s != nullptr);
continue;
}
size_t idx = index(t);
Entry& e = s->entry(idx);
if (!e.tryWaitUntil(deadline)) {
return false;
}
if (!consumerTicket_.compare_exchange_weak(
t, t + 1, std::memory_order_acq_rel, std::memory_order_acquire)) {
continue;
}
e.takeItem(item);
if (responsibleForAdvance(t)) {
advanceHead(s);
}
return true;
}
}
/* findSegment */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
inline typename UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::Segment*
UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::findSegment(Segment* s, const Ticket t) const noexcept {
while (t >= (s->minTicket() + SegmentSize)) {
Segment* next = s->nextSegment();
// Note that the following loop will not spin indefinitely. The
// algorithm guarantees that the producer reponsible for setting
// the next pointer is already running.
while (next == nullptr) {
asm_volatile_pause();
next = s->nextSegment();
}
DCHECK(next != nullptr);
s = next;
}
return s;
}
/* allocNextSegment */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
inline void UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::allocNextSegment(Segment* s, const Ticket t) {
auto next = new Segment(t);
if (!SPSC) {
next->acquire_ref_safe(); // hazptr
}
DEBUG_PRINT(s << " " << next);
DCHECK(s->nextSegment() == nullptr);
s->setNextSegment(next);
}
/* advanceTail */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
inline void UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::advanceTail(Segment* s) noexcept {
Segment* next = s->nextSegment();
if (!SingleProducer) {
// Note that the following loop will not spin indefinitely. The
// algorithm guarantees that the producer reponsible for setting
// the next pointer is already running.
while (next == nullptr) {
asm_volatile_pause();
next = s->nextSegment();
}
}
DCHECK(next != nullptr);
DEBUG_PRINT(s << " " << next);
setTail(next);
}
/* advanceHead */
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
inline void UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::advanceHead(Segment* s) noexcept {
// Note that the following loops will not spin indefinitely. The
// algorithm guarantees that the producers reponsible for advancing
// the tail pointer and setting the next pointer are already
// running.
while (tail() == s) {
asm_volatile_pause();
}
auto next = s->nextSegment();
while (next == nullptr) {
next = s->nextSegment();
}
DEBUG_PRINT(s << " " << next);
setHead(next);
if (SPSC) {
delete s;
} else {
s->retire(); // hazptr
}
}
/**
* Entry
*/
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
class UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::Entry {
folly::SaturatingSemaphore<MayBlock, Atom> flag_;
typename std::aligned_storage<sizeof(T), alignof(T)>::type item_;
public:
template <typename Arg>
FOLLY_ALWAYS_INLINE void putItem(Arg&& arg) {
new (&item_) T(std::forward<Arg>(arg));
flag_.post();
}
FOLLY_ALWAYS_INLINE void takeItem(T& item) noexcept {
flag_.wait();
getItem(item);
}
template <typename Clock, typename Duration>
FOLLY_ALWAYS_INLINE bool tryWaitUntil(
const std::chrono::time_point<Clock, Duration>& deadline) noexcept {
return flag_.try_wait_until(deadline);
}
private:
FOLLY_ALWAYS_INLINE void getItem(T& item) noexcept {
item = std::move(*(folly::launder(itemPtr())));
destroyItem();
}
FOLLY_ALWAYS_INLINE T* itemPtr() noexcept {
return static_cast<T*>(static_cast<void*>(&item_));
}
FOLLY_ALWAYS_INLINE void destroyItem() noexcept {
itemPtr()->~T();
}
}; // UnboundedQueue::Entry
/**
* Segment
*/
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize,
template <typename> class Atom>
class UnboundedQueue<
T,
SingleProducer,
SingleConsumer,
MayBlock,
LgSegmentSize,
Atom>::Segment : public folly::hazptr::hazptr_obj_base_refcounted<Segment> {
Atom<Segment*> next_;
const Ticket min_;
bool marked_; // used for iterative deletion
FOLLY_ALIGN_TO_AVOID_FALSE_SHARING
Entry b_[SegmentSize];
public:
explicit Segment(const Ticket t) : next_(nullptr), min_(t), marked_(false) {}
~Segment() {
if (!SPSC && !marked_) {
auto next = nextSegment();
while (next) {
if (!next->release_ref()) { // hazptr
return;
}
auto s = next;
next = s->nextSegment();
s->marked_ = true;
delete s;
}
}
}
Segment* nextSegment() const noexcept {
return next_.load(std::memory_order_acquire);
}
void setNextSegment(Segment* s) noexcept {
next_.store(s, std::memory_order_release);
}
FOLLY_ALWAYS_INLINE Ticket minTicket() const noexcept {
DCHECK_EQ((min_ & (SegmentSize - 1)), 0);
return min_;
}
FOLLY_ALWAYS_INLINE Entry& entry(size_t index) noexcept {
return b_[index];
}
}; // UnboundedQueue::Segment
} // namespace folly
/*
* Copyright 2017 Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include <atomic>
#include <chrono>
#include <memory>
#include <folly/concurrency/CacheLocality.h>
namespace folly {
/// UnboundedQueue supports a variety of options for unbounded
/// dynamically expanding an shrinking queues, including variations of:
/// - Single vs. multiple producers
/// - Single vs. multiple consumers
/// - Blocking vs. spin-waiting
/// - Non-waiting, timed, and waiting consumer operations.
/// Producer operations never wait or fail (unless out-of-memory).
///
/// Template parameters:
/// - T: element type
/// - SingleProducer: true if there can be only one producer at a
/// time.
/// - SingleConsumer: true if there can be only one consumer at a
/// time.
/// - MayBlock: true if consumers may block, false if they only
/// spins. A performance tuning parameter.
/// - LgSegmentSize (default 8): Log base 2 of number of elements per
/// segment. A performance tuning parameter. See below.
///
/// When to use UnboundedQueue:
/// - If a small bound may lead to deadlock or performance degradation
/// under bursty patterns.
/// - If there is no risk of the queue growing too much.
///
/// When not to use UnboundedQueue:
/// - If there is risk of the queue growing too much and a large bound
/// is acceptable, then use DynamicBoundedQueue.
/// - If the queue must not allocate on enqueue or it must have a
/// small bound, then use fixed-size MPMCQueue or (if non-blocking
/// SPSC) ProducerConsumerQueue.
///
/// Template Aliases:
/// USPSCQueue<T, MayBlock, LgSegmentSize>
/// UMPSCQueue<T, MayBlock, LgSegmentSize>
/// USPMCQueue<T, MayBlock, LgSegmentSize>
/// UMPMCQueue<T, MayBlock, LgSegmentSize>
///
/// Functions:
/// Producer operations never wait or fail (unless OOM)
/// void enqueue(const T&);
/// void enqueue(T&&);
/// Adds an element to the end of the queue.
///
/// Consumer operations:
/// void dequeue(T&);
/// Extracts an element from the front of the queue. Waits
/// until an element is available if needed.
/// bool try_dequeue(T&);
/// Tries to extracts an element from the front of the queue
/// if available. Returns true if successful, false otherwise.
/// bool try_dequeue_until(T&, time_point& deadline);
/// Tries to extracts an element from the front of the queue
/// if available until the specified deadline. Returns true
/// if successful, false otherwise.
/// bool try_dequeue_for(T&, duration&);
/// Tries to extracts an element from the front of the queue
/// if available for for the specified duration. Returns true
/// if successful, false otherwise.
///
/// Secondary functions:
/// size_t size();
/// Returns an estimate of the size of the queue.
/// bool empty();
/// Returns true only if the queue was empty during the call.
/// Note: size() and empty() are guaranteed to be accurate only if
/// the queue is not changed concurrently.
///
/// Usage examples:
/// @code
/// /* UMPSC, doesn't block, 1024 int elements per segment */
/// UMPSCQueue<int, false, 10> q;
/// q.enqueue(1);
/// q.enqueue(2);
/// q.enqueue(3);
/// ASSERT_FALSE(q.empty());
/// ASSERT_EQ(q.size(), 3);
/// int v;
/// q.dequeue(v);
/// ASSERT_EQ(v, 1);
/// ASSERT_TRUE(try_dequeue(v));
/// ASSERT_EQ(v, 2);
/// ASSERT_TRUE(try_dequeue_until(v, now() + seconds(1)));
/// ASSERT_EQ(v, 3);
/// ASSERT_TRUE(q.empty());
/// ASSERT_EQ(q.size(), 0);
/// ASSERT_FALSE(try_dequeue(v));
/// ASSERT_FALSE(try_dequeue_for(v, microseconds(100)));
/// @endcode
///
/// Design:
/// - The queue is composed of one or more segments. Each segment has
/// a fixed size of 2^LgSegmentSize entries. Each segment is used
/// exactly once.
/// - Each entry is composed of a futex and a single element.
/// - The queue contains two 64-bit ticket variables. The producer
/// ticket counts the number of producer tickets isued so far, and
/// the same for the consumer ticket. Each ticket number corresponds
/// to a specific entry in a specific segment.
/// - The queue maintains two pointers, head and tail. Head points to
/// the segment that corresponds to the current consumer
/// ticket. Similarly, tail pointer points to the segment that
/// corresponds to the producer ticket.
/// - Segments are organized as a singly linked list.
/// - The producer with the first ticket in the current producer
/// segment is solely responsible for allocating and linking the
/// next segment.
/// - The producer with the last ticket in the current producer
/// segment is solely responsible for advancing the tail pointer to
/// the next segment.
/// - Similarly, the consumer with the last ticket in the current
/// consumer segment is solely responsible for advancing the head
/// pointer to the next segment. It must ensure that head never
/// overtakes tail.
///
/// Memory Usage:
/// - An empty queue contains one segment. A nonempty queue contains
/// one or two more segment than fits its contents.
/// - Removed segments are not reclaimed until there are no threads,
/// producers or consumers, have references to them or their
/// predessors. That is, a lagging thread may delay the reclamation
/// of a chain of removed segments.
///
/// Performance considerations:
/// - All operations take constant time, excluding the costs of
/// allocation, reclamation, interence from other threads, and
/// waiting for actions by other threads.
/// - In general, using the single producer and or single consumer
/// variants yields better performance than the MP and MC
/// alternatives.
/// - SPSC without blocking is the fastest configuration. It doesn't
/// include any read-modify-write atomic operations, full fences, or
/// system calls in the critical path.
/// - MP adds a fetch_add to the critical path of each producer operation.
/// - MC adds a fetch_add or compare_exchange to the critical path of
/// each consumer operation.
/// - The possibility of consumers blocking, even if they never do,
/// adds a compare_exchange to the crtical path of each producer
/// operation.
/// - MPMC, SPMC, MPSC require the use of a deferred reclamation
/// mechanism to guarantee that segments removed from the linked
/// list, i.e., unreachable from the head pointer, are reclaimed
/// only after they are no longer needed by any lagging producers or
/// consumers.
/// - The overheads of segment allocation and reclamation are intended
/// to be mostly out of the critical path of the queue's throughput.
/// - If the template parameter LgSegmentSize is changed, it should be
/// set adequately high to keep the amortized cost of allocation and
/// reclamation low.
/// - Another consideration is that the queue is guaranteed to have
/// enough space for a number of consumers equal to 2^LgSegmentSize
/// for local blocking. Excess waiting consumers spin.
/// - It is recommended to measure perforamnce with different variants
/// when applicable, e.g., UMPMC vs UMPSC. Depending on the use
/// case, sometimes the variant with the higher sequential overhead
/// may yield better results due to, for example, more favorable
/// producer-consumer balance or favorable timining for avoiding
/// costly blocking.
template <
typename T,
bool SingleProducer,
bool SingleConsumer,
bool MayBlock,
size_t LgSegmentSize = 8,
template <typename> class Atom = std::atomic>
class UnboundedQueue {
using Ticket = uint64_t;
class Entry;
class Segment;
static constexpr bool SPSC = SingleProducer && SingleConsumer;
static constexpr size_t SegmentSize = 1 << LgSegmentSize;
static constexpr size_t Stride = SPSC || (LgSegmentSize <= 1) ? 1 : 27;
static_assert(
std::is_nothrow_destructible<T>::value,
"T must be nothrow_destructible");
static_assert((Stride & 1) == 1, "Stride must be odd");
static_assert(LgSegmentSize < 32, "LgSegmentSize must be < 32");
FOLLY_ALIGN_TO_AVOID_FALSE_SHARING
Atom<Segment*> head_;
Atom<Ticket> consumerTicket_;
FOLLY_ALIGN_TO_AVOID_FALSE_SHARING
Atom<Segment*> tail_;
Atom<Ticket> producerTicket_;
public:
UnboundedQueue();
~UnboundedQueue();
/** enqueue */
FOLLY_ALWAYS_INLINE void enqueue(const T& arg) {
enqueueImpl(arg);
}
FOLLY_ALWAYS_INLINE void enqueue(T&& arg) {
enqueueImpl(std::move(arg));
}
/** dequeue */
void dequeue(T& item) noexcept;
/** try_dequeue */
bool try_dequeue(T& item) noexcept {
return try_dequeue_until(
item, std::chrono::steady_clock::time_point::min());
}
/** try_dequeue_until */
template <typename Clock, typename Duration>
bool try_dequeue_until(
T& item,
const std::chrono::time_point<Clock, Duration>& deadline) noexcept;
/** try_dequeue_for */
template <typename Rep, typename Period>
bool try_dequeue_for(
T& item,
const std::chrono::duration<Rep, Period>& duration) noexcept {
return try_dequeue_until(item, std::chrono::steady_clock::now() + duration);
}
/** size */
size_t size() const noexcept {
auto p = producerTicket();
auto c = consumerTicket();
return p > c ? p - c : 0;
}
/** empty */
bool empty() const noexcept {
auto c = consumerTicket();
auto p = producerTicket();
return p <= c;
}
private:
template <typename Arg>
void enqueueImpl(Arg&& arg);
template <typename Arg>
void enqueueCommon(Segment* s, Arg&& arg);
void dequeueCommon(Segment* s, T& item) noexcept;
template <typename Clock, typename Duration>
bool singleConsumerTryDequeueUntil(
Segment* s,
T& item,
const std::chrono::time_point<Clock, Duration>& deadline) noexcept;
template <typename Clock, typename Duration>
bool multiConsumerTryDequeueUntil(
Segment* s,
T& item,
const std::chrono::time_point<Clock, Duration>& deadline) noexcept;
Segment* findSegment(Segment* s, const Ticket t) const noexcept;
void allocNextSegment(Segment* s, const Ticket t);
void advanceTail(Segment* s) noexcept;
void advanceHead(Segment* s) noexcept;
FOLLY_ALWAYS_INLINE size_t index(Ticket t) const noexcept {
return (t * Stride) & (SegmentSize - 1);
}
FOLLY_ALWAYS_INLINE bool responsibleForAlloc(Ticket t) const noexcept {
return (t & (SegmentSize - 1)) == 0;
}
FOLLY_ALWAYS_INLINE bool responsibleForAdvance(Ticket t) const noexcept {
return (t & (SegmentSize - 1)) == (SegmentSize - 1);
}
FOLLY_ALWAYS_INLINE Segment* head() const noexcept {
return head_.load(std::memory_order_acquire);
}
FOLLY_ALWAYS_INLINE Segment* tail() const noexcept {
return tail_.load(std::memory_order_acquire);
}
FOLLY_ALWAYS_INLINE Ticket producerTicket() const noexcept {
return producerTicket_.load(std::memory_order_acquire);
}
FOLLY_ALWAYS_INLINE Ticket consumerTicket() const noexcept {
return consumerTicket_.load(std::memory_order_acquire);
}
void setHead(Segment* s) noexcept {
head_.store(s, std::memory_order_release);
}
void setTail(Segment* s) noexcept {
tail_.store(s, std::memory_order_release);
}
FOLLY_ALWAYS_INLINE void setProducerTicket(Ticket t) noexcept {
producerTicket_.store(t, std::memory_order_release);
}
FOLLY_ALWAYS_INLINE void setConsumerTicket(Ticket t) noexcept {
consumerTicket_.store(t, std::memory_order_release);
}
FOLLY_ALWAYS_INLINE Ticket fetchIncrementConsumerTicket() noexcept {
if (SingleConsumer) {
auto oldval = consumerTicket();
setConsumerTicket(oldval + 1);
return oldval;
} else { // MC
return consumerTicket_.fetch_add(1, std::memory_order_acq_rel);
}
}
FOLLY_ALWAYS_INLINE Ticket fetchIncrementProducerTicket() noexcept {
if (SingleProducer) {
auto oldval = producerTicket();
setProducerTicket(oldval + 1);
return oldval;
} else { // MP
return producerTicket_.fetch_add(1, std::memory_order_acq_rel);
}
}
}; // UnboundedQueue
/* Aliases */
template <
typename T,
bool MayBlock,
size_t LgSegmentSize = 8,
template <typename> class Atom = std::atomic>
using USPSCQueue = UnboundedQueue<T, true, true, MayBlock, LgSegmentSize, Atom>;
template <
typename T,
bool MayBlock,
size_t LgSegmentSize = 8,
template <typename> class Atom = std::atomic>
using UMPSCQueue =
UnboundedQueue<T, false, true, MayBlock, LgSegmentSize, Atom>;
template <
typename T,
bool MayBlock,
size_t LgSegmentSize = 8,
template <typename> class Atom = std::atomic>
using USPMCQueue =
UnboundedQueue<T, true, false, MayBlock, LgSegmentSize, Atom>;
template <
typename T,
bool MayBlock,
size_t LgSegmentSize = 8,
template <typename> class Atom = std::atomic>
using UMPMCQueue =
UnboundedQueue<T, false, false, MayBlock, LgSegmentSize, Atom>;
} // namespace folly
#include <folly/concurrency/UnboundedQueue-inl.h>
/*
* Copyright 2017 Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <folly/concurrency/UnboundedQueue.h>
#include <folly/MPMCQueue.h>
#include <folly/ProducerConsumerQueue.h>
#include <folly/portability/GTest.h>
#include <glog/logging.h>
#include <atomic>
#include <thread>
DEFINE_bool(bench, false, "run benchmark");
DEFINE_int32(reps, 10, "number of reps");
DEFINE_int32(ops, 1000000, "number of operations per rep");
template <typename T, bool MayBlock>
using USPSC = folly::USPSCQueue<T, MayBlock>;
template <typename T, bool MayBlock>
using UMPSC = folly::UMPSCQueue<T, MayBlock>;
template <typename T, bool MayBlock>
using USPMC = folly::USPMCQueue<T, MayBlock>;
template <typename T, bool MayBlock>
using UMPMC = folly::UMPMCQueue<T, MayBlock>;
template <template <typename, bool> class Q, bool MayBlock>
void basic_test() {
Q<int, MayBlock> q;
ASSERT_TRUE(q.empty());
ASSERT_EQ(q.size(), 0);
int v = -1;
ASSERT_FALSE(q.try_dequeue(v));
q.enqueue(1);
ASSERT_FALSE(q.empty());
ASSERT_EQ(q.size(), 1);
q.enqueue(2);
ASSERT_EQ(q.size(), 2);
ASSERT_FALSE(q.empty());
ASSERT_TRUE(q.try_dequeue(v));
ASSERT_EQ(v, 1);
ASSERT_FALSE(q.empty());
ASSERT_EQ(q.size(), 1);
ASSERT_TRUE(q.try_dequeue(v));
ASSERT_EQ(v, 2);
ASSERT_TRUE(q.empty());
ASSERT_EQ(q.size(), 0);
}
TEST(UnboundedQueue, basic) {
basic_test<USPSC, false>();
basic_test<UMPSC, false>();
basic_test<USPMC, false>();
basic_test<UMPMC, false>();
basic_test<USPSC, true>();
basic_test<UMPSC, true>();
basic_test<USPMC, true>();
basic_test<UMPMC, true>();
}
template <template <typename, bool> class Q, bool MayBlock>
void timeout_test() {
Q<int, MayBlock> q;
int v = -1;
ASSERT_FALSE(q.try_dequeue_until(
v, std::chrono::steady_clock::now() + std::chrono::microseconds(1)));
ASSERT_FALSE(q.try_dequeue_for(v, std::chrono::microseconds(1)));
q.enqueue(10);
ASSERT_TRUE(q.try_dequeue_until(
v, std::chrono::steady_clock::now() + std::chrono::microseconds(1)));
ASSERT_EQ(v, 10);
}
TEST(UnboundedQueue, timeout) {
timeout_test<USPSC, false>();
timeout_test<UMPSC, false>();
timeout_test<USPMC, false>();
timeout_test<UMPMC, false>();
timeout_test<USPSC, true>();
timeout_test<UMPSC, true>();
timeout_test<USPMC, true>();
timeout_test<UMPMC, true>();
}
template <typename ProdFunc, typename ConsFunc, typename EndFunc>
inline uint64_t run_once(
int nprod,
int ncons,
const ProdFunc& prodFn,
const ConsFunc& consFn,
const EndFunc& endFn) {
std::atomic<bool> start{false};
std::atomic<int> ready{0};
/* producers */
std::vector<std::thread> prodThr(nprod);
for (int tid = 0; tid < nprod; ++tid) {
prodThr[tid] = std::thread([&, tid] {
++ready;
while (!start.load()) {
/* spin */;
}
prodFn(tid);
});
}
/* consumers */
std::vector<std::thread> consThr(ncons);
for (int tid = 0; tid < ncons; ++tid) {
consThr[tid] = std::thread([&, tid] {
++ready;
while (!start.load()) {
/* spin */;
}
consFn(tid);
});
}
/* wait for all producers and consumers to be ready */
while (ready.load() < (nprod + ncons)) {
/* spin */;
}
/* begin time measurement */
auto tbegin = std::chrono::steady_clock::now();
start.store(true);
/* wait for completion */
for (int i = 0; i < nprod; ++i) {
prodThr[i].join();
}
for (int i = 0; i < ncons; ++i) {
consThr[i].join();
}
/* end time measurement */
auto tend = std::chrono::steady_clock::now();
endFn();
return std::chrono::duration_cast<std::chrono::nanoseconds>(tend - tbegin)
.count();
}
template <bool SingleProducer, bool SingleConsumer, bool MayBlock>
void enq_deq_test(const int nprod, const int ncons) {
if (SingleProducer) {
ASSERT_EQ(nprod, 1);
}
if (SingleConsumer) {
ASSERT_EQ(ncons, 1);
}
int ops = 1000;
folly::UnboundedQueue<int, SingleProducer, SingleConsumer, MayBlock, 4> q;
std::atomic<uint64_t> sum(0);
auto prod = [&](int tid) {
for (int i = tid; i < ops; i += nprod) {
q.enqueue(i);
}
};
auto cons = [&](int tid) {
uint64_t mysum = 0;
for (int i = tid; i < ops; i += ncons) {
int v = -1;
if ((i % 3) == 0) {
while (!q.try_dequeue(v)) {
/* keep trying */;
}
} else if ((i % 3) == 1) {
std::chrono::steady_clock::time_point deadline =
std::chrono::steady_clock::now() + std::chrono::milliseconds(1);
while (!q.try_dequeue_until(v, deadline)) {
/* keep trying */;
}
} else {
q.dequeue(v);
}
if (nprod == 1 && ncons == 1) {
ASSERT_EQ(v, i);
}
mysum += v;
}
sum.fetch_add(mysum);
};
auto endfn = [&] {
uint64_t expected = ops;
expected *= ops - 1;
expected /= 2;
ASSERT_EQ(sum.load(), expected);
};
run_once(nprod, ncons, prod, cons, endfn);
}
TEST(UnboundedQueue, enq_deq) {
/* SPSC */
enq_deq_test<true, true, false>(1, 1);
enq_deq_test<true, true, true>(1, 1);
/* MPSC */
enq_deq_test<false, true, false>(1, 1);
enq_deq_test<false, true, true>(1, 1);
enq_deq_test<false, true, false>(2, 1);
enq_deq_test<false, true, true>(2, 1);
enq_deq_test<false, true, false>(10, 1);
enq_deq_test<false, true, true>(10, 1);
/* SPMC */
enq_deq_test<true, false, false>(1, 1);
enq_deq_test<true, false, true>(1, 1);
enq_deq_test<true, false, false>(1, 2);
enq_deq_test<true, false, true>(1, 2);
enq_deq_test<true, false, false>(1, 10);
enq_deq_test<true, false, true>(1, 10);
/* MPMC */
enq_deq_test<false, false, false>(1, 1);
enq_deq_test<false, false, true>(1, 1);
enq_deq_test<false, false, false>(2, 1);
enq_deq_test<false, false, true>(2, 1);
enq_deq_test<false, false, false>(10, 1);
enq_deq_test<false, false, true>(10, 1);
enq_deq_test<false, false, false>(1, 2);
enq_deq_test<false, false, true>(1, 2);
enq_deq_test<false, false, false>(1, 10);
enq_deq_test<false, false, true>(1, 10);
enq_deq_test<false, false, false>(2, 2);
enq_deq_test<false, false, true>(2, 2);
enq_deq_test<false, false, false>(10, 10);
enq_deq_test<false, false, true>(10, 10);
}
template <typename RepFunc>
uint64_t runBench(const std::string& name, int ops, const RepFunc& repFn) {
int reps = FLAGS_reps;
uint64_t min = UINTMAX_MAX;
uint64_t max = 0;
uint64_t sum = 0;
repFn(); // sometimes first run is outlier
for (int r = 0; r < reps; ++r) {
uint64_t dur = repFn();
sum += dur;
min = std::min(min, dur);
max = std::max(max, dur);
// if each rep takes too long run at least 3 reps
const uint64_t minute = 60000000000UL;
if (sum > minute && r >= 2) {
reps = r + 1;
break;
}
}
const std::string unit = " ns";
uint64_t avg = sum / reps;
uint64_t res = min;
std::cout << name;
std::cout << " " << std::setw(4) << max / ops << unit;
std::cout << " " << std::setw(4) << avg / ops << unit;
std::cout << " " << std::setw(4) << res / ops << unit;
std::cout << std::endl;
return res;
}
template <template <typename, bool> class Q, typename T, int Op>
uint64_t bench(const int nprod, const int ncons, const std::string& name) {
int ops = FLAGS_ops;
auto repFn = [&] {
Q<T, Op == 3 || Op == 4 || Op == 5> q;
std::atomic<uint64_t> sum(0);
auto prod = [&](int tid) {
for (int i = tid; i < ops; i += nprod) {
q.enqueue(i);
}
};
auto cons = [&](int tid) {
std::chrono::steady_clock::time_point deadline =
std::chrono::steady_clock::now() + std::chrono::hours(24);
uint64_t mysum = 0;
for (int i = tid; i < ops; i += ncons) {
T v;
if (Op == 0 || Op == 3) {
while (UNLIKELY(!q.try_dequeue(v))) {
/* keep trying */;
}
} else if (Op == 1 || Op == 4) {
while (UNLIKELY(!q.try_dequeue_until(v, deadline))) {
/* keep trying */;
}
} else {
ASSERT_TRUE(Op == 2 || Op == 5);
q.dequeue(v);
}
if (nprod == 1 && ncons == 1) {
DCHECK_EQ(int(v), i);
}
mysum += v;
}
sum.fetch_add(mysum);
};
auto endfn = [&] {
uint64_t expected = ops;
expected *= ops - 1;
expected /= 2;
ASSERT_EQ(sum.load(), expected);
};
return run_once(nprod, ncons, prod, cons, endfn);
};
return runBench(name, ops, repFn);
}
/* For performance comparison */
template <typename T, size_t capacity>
class MPMC {
folly::MPMCQueue<T> q_;
public:
MPMC() : q_(capacity) {}
template <typename... Args>
void enqueue(Args&&... args) {
q_.blockingWrite(std::forward<Args>(args)...);
}
void dequeue(T& item) {
q_.blockingRead(item);
}
bool try_dequeue(T& item) {
return q_.read(item);
}
template <typename Clock, typename Duration>
bool try_dequeue_until(
T& item,
const std::chrono::time_point<Clock, Duration>& deadline) {
return q_.tryReadUntil(deadline, item);
}
};
template <typename T, bool ignore>
using FMPMC = MPMC<T, 256 * 1024>;
template <typename T, size_t capacity>
class PCQ {
folly::ProducerConsumerQueue<T> q_;
public:
PCQ() : q_(capacity) {}
template <typename... Args>
void enqueue(Args&&... args) {
while (!q_.write(std::forward<Args>(args)...)) {
/* keep trying*/;
}
}
void dequeue(T&) {
ASSERT_TRUE(false);
}
bool try_dequeue(T& item) {
return q_.read(item);
}
template <typename Clock, typename Duration>
bool try_dequeue_until(T&, const std::chrono::time_point<Clock, Duration>&) {
return false;
}
};
template <typename T, bool ignore>
using FPCQ = PCQ<T, 256 * 1024>;
template <size_t M>
struct IntArray {
int a[M];
IntArray() {}
/* implicit */ IntArray(int v) {
for (size_t i = 0; i < M; ++i) {
a[i] = v;
}
}
operator int() {
return a[0];
}
};
void dottedLine() {
std::cout << ".............................................................."
<< std::endl;
}
template <typename T>
void type_benches(const int np, const int nc, const std::string& name) {
std::cout << name
<< "===========================================" << std::endl;
if (np == 1 && nc == 1) {
bench<USPSC, T, 0>(1, 1, "Unbounded SPSC try spin only ");
bench<USPSC, T, 1>(1, 1, "Unbounded SPSC timed spin only ");
bench<USPSC, T, 2>(1, 1, "Unbounded SPSC wait spin only ");
bench<USPSC, T, 3>(1, 1, "Unbounded SPSC try may block ");
bench<USPSC, T, 4>(1, 1, "Unbounded SPSC timed may block ");
bench<USPSC, T, 5>(1, 1, "Unbounded SPSC wait may block ");
dottedLine();
}
if (nc == 1) {
bench<UMPSC, T, 0>(np, 1, "Unbounded MPSC try spin only ");
bench<UMPSC, T, 1>(np, 1, "Unbounded MPSC timed spin only ");
bench<UMPSC, T, 2>(np, 1, "Unbounded MPSC wait spin only ");
bench<UMPSC, T, 3>(np, 1, "Unbounded MPSC try may block ");
bench<UMPSC, T, 4>(np, 1, "Unbounded MPSC timed may block ");
bench<UMPSC, T, 5>(np, 1, "Unbounded MPSC wait may block ");
dottedLine();
}
if (np == 1) {
bench<USPMC, T, 0>(1, nc, "Unbounded SPMC try spin only ");
bench<USPMC, T, 1>(1, nc, "Unbounded SPMC timed spin only ");
bench<USPMC, T, 2>(1, nc, "Unbounded SPMC wait spin only ");
bench<USPMC, T, 3>(1, nc, "Unbounded SPMC try may block ");
bench<USPMC, T, 4>(1, nc, "Unbounded SPMC timed may block ");
bench<USPMC, T, 5>(1, nc, "Unbounded SPMC wait may block ");
dottedLine();
}
bench<UMPMC, T, 0>(np, nc, "Unbounded MPMC try spin only ");
bench<UMPMC, T, 1>(np, nc, "Unbounded MPMC timed spin only ");
bench<UMPMC, T, 2>(np, nc, "Unbounded MPMC wait spin only ");
bench<UMPMC, T, 3>(np, nc, "Unbounded MPMC try may block ");
bench<UMPMC, T, 4>(np, nc, "Unbounded MPMC timed may block ");
bench<UMPMC, T, 5>(np, nc, "Unbounded MPMC wait may block ");
dottedLine();
if (np == 1 && nc == 1) {
bench<FPCQ, T, 0>(1, 1, "folly::PCQ read ");
dottedLine();
}
bench<FMPMC, T, 3>(np, nc, "folly::MPMC read ");
bench<FMPMC, T, 4>(np, nc, "folly::MPMC tryReadUntil ");
bench<FMPMC, T, 5>(np, nc, "folly::MPMC blockingRead ");
std::cout << "=============================================================="
<< std::endl;
}
void benches(const int np, const int nc) {
std::cout << "====================== " << std::setw(2) << np << " prod"
<< " " << std::setw(2) << nc << " cons"
<< " ======================" << std::endl;
type_benches<uint32_t>(np, nc, "=== uint32_t ======");
// Benchmarks for other element sizes can be added as follows:
// type_benches<IntArray<4>>(np, nc, "=== IntArray<4> ===");
}
TEST(UnboundedQueue, bench) {
if (!FLAGS_bench) {
return;
}
std::cout << "=============================================================="
<< std::endl;
std::cout << std::setw(2) << FLAGS_reps << " reps of " << std::setw(8)
<< FLAGS_ops << " handoffs\n";
dottedLine();
std::cout << "$ numactl -N 1 $dir/unbounded_queue_test --bench\n";
dottedLine();
std::cout << "Using a capacity of 256K for folly::ProducerConsumerQueue\n"
<< "and folly::MPMCQueue\n";
std::cout << "=============================================================="
<< std::endl;
std::cout << "Test name Max time Avg time Min time"
<< std::endl;
for (int nc : {1, 2, 4, 8, 16, 32}) {
int np = 1;
benches(np, nc);
}
for (int np : {1, 2, 4, 8, 16, 32}) {
int nc = 1;
benches(np, nc);
}
for (int np : {2, 4, 8, 16, 32}) {
for (int nc : {2, 4, 8, 16, 32}) {
benches(np, nc);
}
}
}
/*
==============================================================
10 reps of 1000000 handoffs
..............................................................
$ numactl -N 1 $dir/unbounded_queue_test --bench
..............................................................
Using a capacity of 256K for folly::ProducerConsumerQueue
and folly::MPMCQueue
==============================================================
Test name Max time Avg time Min time
====================== 1 prod 1 cons ======================
=== uint32_t =================================================
Unbounded SPSC try spin only 5 ns 5 ns 5 ns
Unbounded SPSC timed spin only 5 ns 5 ns 5 ns
Unbounded SPSC wait spin only 6 ns 6 ns 5 ns
Unbounded SPSC try may block 38 ns 37 ns 35 ns
Unbounded SPSC timed may block 38 ns 36 ns 34 ns
Unbounded SPSC wait may block 34 ns 34 ns 33 ns
..............................................................
Unbounded MPSC try spin only 45 ns 43 ns 42 ns
Unbounded MPSC timed spin only 47 ns 43 ns 42 ns
Unbounded MPSC wait spin only 45 ns 43 ns 41 ns
Unbounded MPSC try may block 55 ns 52 ns 51 ns
Unbounded MPSC timed may block 54 ns 52 ns 51 ns
Unbounded MPSC wait may block 51 ns 50 ns 49 ns
..............................................................
Unbounded SPMC try spin only 18 ns 17 ns 16 ns
Unbounded SPMC timed spin only 23 ns 21 ns 18 ns
Unbounded SPMC wait spin only 22 ns 19 ns 16 ns
Unbounded SPMC try may block 30 ns 26 ns 22 ns
Unbounded SPMC timed may block 32 ns 24 ns 20 ns
Unbounded SPMC wait may block 49 ns 35 ns 29 ns
..............................................................
Unbounded MPMC try spin only 25 ns 24 ns 24 ns
Unbounded MPMC timed spin only 38 ns 35 ns 30 ns
Unbounded MPMC wait spin only 41 ns 39 ns 37 ns
Unbounded MPMC try may block 53 ns 52 ns 51 ns
Unbounded MPMC timed may block 52 ns 51 ns 49 ns
Unbounded MPMC wait may block 53 ns 51 ns 50 ns
..............................................................
folly::PCQ read 16 ns 11 ns 7 ns
..............................................................
folly::MPMC read 52 ns 52 ns 51 ns
folly::MPMC tryReadUntil 96 ns 90 ns 55 ns
folly::MPMC blockingRead 61 ns 56 ns 50 ns
==============================================================
====================== 1 prod 2 cons ======================
=== uint32_t =================================================
Unbounded SPMC try spin only 76 ns 68 ns 53 ns
Unbounded SPMC timed spin only 79 ns 71 ns 65 ns
Unbounded SPMC wait spin only 39 ns 35 ns 32 ns
Unbounded SPMC try may block 83 ns 81 ns 76 ns
Unbounded SPMC timed may block 86 ns 63 ns 23 ns
Unbounded SPMC wait may block 38 ns 36 ns 34 ns
..............................................................
Unbounded MPMC try spin only 86 ns 79 ns 64 ns
Unbounded MPMC timed spin only 84 ns 77 ns 74 ns
Unbounded MPMC wait spin only 36 ns 35 ns 34 ns
Unbounded MPMC try may block 83 ns 79 ns 75 ns
Unbounded MPMC timed may block 83 ns 76 ns 63 ns
Unbounded MPMC wait may block 56 ns 48 ns 36 ns
..............................................................
folly::MPMC read 103 ns 93 ns 68 ns
folly::MPMC tryReadUntil 109 ns 102 ns 91 ns
folly::MPMC blockingRead 61 ns 58 ns 54 ns
==============================================================
====================== 1 prod 4 cons ======================
=== uint32_t =================================================
Unbounded SPMC try spin only 116 ns 109 ns 97 ns
Unbounded SPMC timed spin only 117 ns 111 ns 108 ns
Unbounded SPMC wait spin only 43 ns 40 ns 37 ns
Unbounded SPMC try may block 127 ns 113 ns 98 ns
Unbounded SPMC timed may block 116 ns 109 ns 97 ns
Unbounded SPMC wait may block 45 ns 43 ns 40 ns
..............................................................
Unbounded MPMC try spin only 121 ns 113 ns 102 ns
Unbounded MPMC timed spin only 118 ns 108 ns 88 ns
Unbounded MPMC wait spin only 45 ns 41 ns 34 ns
Unbounded MPMC try may block 117 ns 108 ns 96 ns
Unbounded MPMC timed may block 118 ns 109 ns 99 ns
Unbounded MPMC wait may block 62 ns 53 ns 43 ns
..............................................................
folly::MPMC read 139 ns 130 ns 111 ns
folly::MPMC tryReadUntil 205 ns 135 ns 115 ns
folly::MPMC blockingRead 104 ns 74 ns 54 ns
==============================================================
====================== 1 prod 8 cons ======================
=== uint32_t =================================================
Unbounded SPMC try spin only 169 ns 163 ns 157 ns
Unbounded SPMC timed spin only 167 ns 158 ns 133 ns
Unbounded SPMC wait spin only 44 ns 39 ns 36 ns
Unbounded SPMC try may block 170 ns 165 ns 156 ns
Unbounded SPMC timed may block 172 ns 163 ns 153 ns
Unbounded SPMC wait may block 49 ns 40 ns 35 ns
..............................................................
Unbounded MPMC try spin only 166 ns 158 ns 149 ns
Unbounded MPMC timed spin only 171 ns 161 ns 145 ns
Unbounded MPMC wait spin only 62 ns 52 ns 42 ns
Unbounded MPMC try may block 169 ns 161 ns 149 ns
Unbounded MPMC timed may block 170 ns 160 ns 147 ns
Unbounded MPMC wait may block 70 ns 63 ns 61 ns
..............................................................
folly::MPMC read 174 ns 167 ns 159 ns
folly::MPMC tryReadUntil 349 ns 171 ns 148 ns
folly::MPMC blockingRead 182 ns 138 ns 115 ns
==============================================================
====================== 1 prod 16 cons ======================
=== uint32_t =================================================
Unbounded SPMC try spin only 219 ns 198 ns 190 ns
Unbounded SPMC timed spin only 202 ns 198 ns 193 ns
Unbounded SPMC wait spin only 36 ns 36 ns 35 ns
Unbounded SPMC try may block 202 ns 195 ns 190 ns
Unbounded SPMC timed may block 208 ns 197 ns 190 ns
Unbounded SPMC wait may block 1645 ns 1427 ns 36 ns
..............................................................
Unbounded MPMC try spin only 204 ns 198 ns 194 ns
Unbounded MPMC timed spin only 202 ns 195 ns 190 ns
Unbounded MPMC wait spin only 61 ns 59 ns 57 ns
Unbounded MPMC try may block 206 ns 196 ns 191 ns
Unbounded MPMC timed may block 204 ns 198 ns 192 ns
Unbounded MPMC wait may block 1658 ns 1293 ns 70 ns
..............................................................
folly::MPMC read 210 ns 191 ns 182 ns
folly::MPMC tryReadUntil 574 ns 248 ns 192 ns
folly::MPMC blockingRead 1400 ns 1319 ns 1227 ns
==============================================================
====================== 1 prod 32 cons ======================
=== uint32_t =================================================
Unbounded SPMC try spin only 209 ns 205 ns 199 ns
Unbounded SPMC timed spin only 208 ns 205 ns 200 ns
Unbounded SPMC wait spin only 175 ns 51 ns 33 ns
Unbounded SPMC try may block 215 ns 203 ns 186 ns
Unbounded SPMC timed may block 453 ns 334 ns 204 ns
Unbounded SPMC wait may block 1601 ns 1514 ns 1373 ns
..............................................................
Unbounded MPMC try spin only 328 ns 218 ns 197 ns
Unbounded MPMC timed spin only 217 ns 206 ns 200 ns
Unbounded MPMC wait spin only 147 ns 85 ns 58 ns
Unbounded MPMC try may block 310 ns 223 ns 199 ns
Unbounded MPMC timed may block 461 ns 275 ns 196 ns
Unbounded MPMC wait may block 1623 ns 1526 ns 888 ns
..............................................................
folly::MPMC read 280 ns 215 ns 194 ns
folly::MPMC tryReadUntil 28740 ns 13508 ns 212 ns
folly::MPMC blockingRead 1343 ns 1293 ns 1269 ns
==============================================================
====================== 1 prod 1 cons ======================
=== uint32_t =================================================
Unbounded SPSC try spin only 5 ns 5 ns 5 ns
Unbounded SPSC timed spin only 8 ns 6 ns 6 ns
Unbounded SPSC wait spin only 6 ns 6 ns 5 ns
Unbounded SPSC try may block 37 ns 36 ns 35 ns
Unbounded SPSC timed may block 37 ns 36 ns 35 ns
Unbounded SPSC wait may block 35 ns 35 ns 34 ns
..............................................................
Unbounded MPSC try spin only 43 ns 42 ns 41 ns
Unbounded MPSC timed spin only 45 ns 42 ns 42 ns
Unbounded MPSC wait spin only 44 ns 43 ns 42 ns
Unbounded MPSC try may block 55 ns 51 ns 50 ns
Unbounded MPSC timed may block 61 ns 52 ns 50 ns
Unbounded MPSC wait may block 54 ns 52 ns 50 ns
..............................................................
Unbounded SPMC try spin only 18 ns 17 ns 17 ns
Unbounded SPMC timed spin only 23 ns 19 ns 17 ns
Unbounded SPMC wait spin only 20 ns 17 ns 15 ns
Unbounded SPMC try may block 30 ns 23 ns 19 ns
Unbounded SPMC timed may block 23 ns 19 ns 17 ns
Unbounded SPMC wait may block 36 ns 31 ns 26 ns
..............................................................
Unbounded MPMC try spin only 25 ns 23 ns 17 ns
Unbounded MPMC timed spin only 37 ns 34 ns 25 ns
Unbounded MPMC wait spin only 40 ns 38 ns 36 ns
Unbounded MPMC try may block 51 ns 49 ns 48 ns
Unbounded MPMC timed may block 53 ns 50 ns 48 ns
Unbounded MPMC wait may block 53 ns 49 ns 34 ns
..............................................................
folly::PCQ read 15 ns 12 ns 7 ns
..............................................................
folly::MPMC read 53 ns 51 ns 50 ns
folly::MPMC tryReadUntil 100 ns 96 ns 90 ns
folly::MPMC blockingRead 75 ns 59 ns 52 ns
==============================================================
====================== 2 prod 1 cons ======================
=== uint32_t =================================================
Unbounded MPSC try spin only 49 ns 49 ns 46 ns
Unbounded MPSC timed spin only 52 ns 50 ns 49 ns
Unbounded MPSC wait spin only 53 ns 52 ns 51 ns
Unbounded MPSC try may block 63 ns 60 ns 57 ns
Unbounded MPSC timed may block 64 ns 61 ns 54 ns
Unbounded MPSC wait may block 62 ns 59 ns 35 ns
..............................................................
Unbounded MPMC try spin only 44 ns 41 ns 38 ns
Unbounded MPMC timed spin only 50 ns 49 ns 49 ns
Unbounded MPMC wait spin only 51 ns 49 ns 49 ns
Unbounded MPMC try may block 63 ns 60 ns 57 ns
Unbounded MPMC timed may block 62 ns 60 ns 57 ns
Unbounded MPMC wait may block 62 ns 60 ns 58 ns
..............................................................
folly::MPMC read 78 ns 57 ns 52 ns
folly::MPMC tryReadUntil 78 ns 72 ns 70 ns
folly::MPMC blockingRead 56 ns 54 ns 52 ns
==============================================================
====================== 4 prod 1 cons ======================
=== uint32_t =================================================
Unbounded MPSC try spin only 48 ns 47 ns 46 ns
Unbounded MPSC timed spin only 47 ns 47 ns 46 ns
Unbounded MPSC wait spin only 49 ns 47 ns 47 ns
Unbounded MPSC try may block 61 ns 59 ns 55 ns
Unbounded MPSC timed may block 62 ns 58 ns 46 ns
Unbounded MPSC wait may block 62 ns 61 ns 59 ns
..............................................................
Unbounded MPMC try spin only 42 ns 42 ns 40 ns
Unbounded MPMC timed spin only 48 ns 47 ns 45 ns
Unbounded MPMC wait spin only 48 ns 47 ns 46 ns
Unbounded MPMC try may block 63 ns 62 ns 61 ns
Unbounded MPMC timed may block 63 ns 61 ns 51 ns
Unbounded MPMC wait may block 62 ns 61 ns 59 ns
..............................................................
folly::MPMC read 56 ns 55 ns 54 ns
folly::MPMC tryReadUntil 112 ns 106 ns 97 ns
folly::MPMC blockingRead 47 ns 47 ns 45 ns
==============================================================
====================== 8 prod 1 cons ======================
=== uint32_t =================================================
Unbounded MPSC try spin only 44 ns 43 ns 42 ns
Unbounded MPSC timed spin only 45 ns 44 ns 40 ns
Unbounded MPSC wait spin only 45 ns 44 ns 41 ns
Unbounded MPSC try may block 61 ns 60 ns 58 ns
Unbounded MPSC timed may block 61 ns 59 ns 56 ns
Unbounded MPSC wait may block 61 ns 59 ns 56 ns
..............................................................
Unbounded MPMC try spin only 43 ns 40 ns 36 ns
Unbounded MPMC timed spin only 45 ns 44 ns 41 ns
Unbounded MPMC wait spin only 45 ns 43 ns 41 ns
Unbounded MPMC try may block 62 ns 60 ns 58 ns
Unbounded MPMC timed may block 62 ns 59 ns 56 ns
Unbounded MPMC wait may block 61 ns 58 ns 54 ns
..............................................................
folly::MPMC read 147 ns 119 ns 63 ns
folly::MPMC tryReadUntil 152 ns 130 ns 97 ns
folly::MPMC blockingRead 135 ns 101 ns 48 ns
==============================================================
====================== 16 prod 1 cons ======================
=== uint32_t =================================================
Unbounded MPSC try spin only 47 ns 38 ns 35 ns
Unbounded MPSC timed spin only 36 ns 36 ns 35 ns
Unbounded MPSC wait spin only 46 ns 37 ns 35 ns
Unbounded MPSC try may block 58 ns 47 ns 45 ns
Unbounded MPSC timed may block 46 ns 46 ns 45 ns
Unbounded MPSC wait may block 47 ns 45 ns 45 ns
..............................................................
Unbounded MPMC try spin only 41 ns 39 ns 35 ns
Unbounded MPMC timed spin only 45 ns 41 ns 38 ns
Unbounded MPMC wait spin only 43 ns 40 ns 38 ns
Unbounded MPMC try may block 51 ns 49 ns 47 ns
Unbounded MPMC timed may block 52 ns 49 ns 47 ns
Unbounded MPMC wait may block 59 ns 50 ns 46 ns
..............................................................
folly::MPMC read 924 ns 839 ns 664 ns
folly::MPMC tryReadUntil 968 ns 865 ns 678 ns
folly::MPMC blockingRead 929 ns 727 ns 487 ns
==============================================================
====================== 32 prod 1 cons ======================
=== uint32_t =================================================
Unbounded MPSC try spin only 90 ns 44 ns 36 ns
Unbounded MPSC timed spin only 91 ns 43 ns 35 ns
Unbounded MPSC wait spin only 92 ns 55 ns 36 ns
Unbounded MPSC try may block 87 ns 52 ns 45 ns
Unbounded MPSC timed may block 70 ns 48 ns 45 ns
Unbounded MPSC wait may block 109 ns 60 ns 45 ns
..............................................................
Unbounded MPMC try spin only 47 ns 42 ns 37 ns
Unbounded MPMC timed spin only 50 ns 46 ns 38 ns
Unbounded MPMC wait spin only 50 ns 42 ns 36 ns
Unbounded MPMC try may block 103 ns 59 ns 50 ns
Unbounded MPMC timed may block 56 ns 52 ns 47 ns
Unbounded MPMC wait may block 59 ns 51 ns 46 ns
..............................................................
folly::MPMC read 1029 ns 911 ns 694 ns
folly::MPMC tryReadUntil 1023 ns 969 ns 907 ns
folly::MPMC blockingRead 1024 ns 921 ns 790 ns
==============================================================
====================== 2 prod 2 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 83 ns 66 ns 24 ns
Unbounded MPMC timed spin only 84 ns 74 ns 49 ns
Unbounded MPMC wait spin only 50 ns 49 ns 47 ns
Unbounded MPMC try may block 86 ns 81 ns 77 ns
Unbounded MPMC timed may block 82 ns 74 ns 59 ns
Unbounded MPMC wait may block 62 ns 59 ns 56 ns
..............................................................
folly::MPMC read 98 ns 85 ns 63 ns
folly::MPMC tryReadUntil 105 ns 94 ns 83 ns
folly::MPMC blockingRead 59 ns 56 ns 54 ns
==============================================================
====================== 2 prod 4 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 114 ns 105 ns 91 ns
Unbounded MPMC timed spin only 119 ns 107 ns 102 ns
Unbounded MPMC wait spin only 54 ns 53 ns 52 ns
Unbounded MPMC try may block 114 ns 106 ns 93 ns
Unbounded MPMC timed may block 111 ns 100 ns 92 ns
Unbounded MPMC wait may block 70 ns 64 ns 60 ns
..............................................................
folly::MPMC read 133 ns 125 ns 120 ns
folly::MPMC tryReadUntil 130 ns 125 ns 114 ns
folly::MPMC blockingRead 69 ns 68 ns 66 ns
==============================================================
====================== 2 prod 8 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 169 ns 160 ns 152 ns
Unbounded MPMC timed spin only 165 ns 158 ns 149 ns
Unbounded MPMC wait spin only 59 ns 54 ns 45 ns
Unbounded MPMC try may block 166 ns 158 ns 131 ns
Unbounded MPMC timed may block 168 ns 163 ns 158 ns
Unbounded MPMC wait may block 73 ns 66 ns 60 ns
..............................................................
folly::MPMC read 170 ns 167 ns 160 ns
folly::MPMC tryReadUntil 163 ns 154 ns 146 ns
folly::MPMC blockingRead 82 ns 73 ns 60 ns
==============================================================
====================== 2 prod 16 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 207 ns 198 ns 191 ns
Unbounded MPMC timed spin only 211 ns 198 ns 192 ns
Unbounded MPMC wait spin only 57 ns 55 ns 54 ns
Unbounded MPMC try may block 197 ns 193 ns 188 ns
Unbounded MPMC timed may block 201 ns 195 ns 188 ns
Unbounded MPMC wait may block 89 ns 78 ns 70 ns
..............................................................
folly::MPMC read 196 ns 189 ns 181 ns
folly::MPMC tryReadUntil 202 ns 184 ns 173 ns
folly::MPMC blockingRead 267 ns 100 ns 76 ns
==============================================================
====================== 2 prod 32 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 228 ns 207 ns 193 ns
Unbounded MPMC timed spin only 210 ns 205 ns 198 ns
Unbounded MPMC wait spin only 102 ns 71 ns 56 ns
Unbounded MPMC try may block 268 ns 211 ns 198 ns
Unbounded MPMC timed may block 226 ns 205 ns 183 ns
Unbounded MPMC wait may block 502 ns 164 ns 67 ns
..............................................................
folly::MPMC read 228 ns 205 ns 195 ns
folly::MPMC tryReadUntil 207 ns 200 ns 192 ns
folly::MPMC blockingRead 830 ns 612 ns 192 ns
==============================================================
====================== 4 prod 2 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 87 ns 65 ns 33 ns
Unbounded MPMC timed spin only 79 ns 60 ns 36 ns
Unbounded MPMC wait spin only 47 ns 46 ns 44 ns
Unbounded MPMC try may block 87 ns 77 ns 52 ns
Unbounded MPMC timed may block 86 ns 79 ns 57 ns
Unbounded MPMC wait may block 62 ns 61 ns 60 ns
..............................................................
folly::MPMC read 110 ns 95 ns 60 ns
folly::MPMC tryReadUntil 108 ns 104 ns 96 ns
folly::MPMC blockingRead 60 ns 57 ns 47 ns
==============================================================
====================== 4 prod 4 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 110 ns 100 ns 86 ns
Unbounded MPMC timed spin only 113 ns 104 ns 93 ns
Unbounded MPMC wait spin only 49 ns 46 ns 45 ns
Unbounded MPMC try may block 115 ns 105 ns 84 ns
Unbounded MPMC timed may block 119 ns 108 ns 89 ns
Unbounded MPMC wait may block 63 ns 61 ns 54 ns
..............................................................
folly::MPMC read 140 ns 131 ns 113 ns
folly::MPMC tryReadUntil 132 ns 129 ns 121 ns
folly::MPMC blockingRead 58 ns 53 ns 48 ns
==============================================================
====================== 4 prod 8 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 170 ns 162 ns 151 ns
Unbounded MPMC timed spin only 174 ns 158 ns 139 ns
Unbounded MPMC wait spin only 51 ns 50 ns 48 ns
Unbounded MPMC try may block 164 ns 160 ns 154 ns
Unbounded MPMC timed may block 165 ns 158 ns 144 ns
Unbounded MPMC wait may block 67 ns 62 ns 52 ns
..............................................................
folly::MPMC read 174 ns 166 ns 156 ns
folly::MPMC tryReadUntil 165 ns 160 ns 150 ns
folly::MPMC blockingRead 58 ns 56 ns 49 ns
==============================================================
====================== 4 prod 16 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 200 ns 195 ns 181 ns
Unbounded MPMC timed spin only 200 ns 195 ns 191 ns
Unbounded MPMC wait spin only 51 ns 49 ns 45 ns
Unbounded MPMC try may block 198 ns 192 ns 188 ns
Unbounded MPMC timed may block 199 ns 190 ns 182 ns
Unbounded MPMC wait may block 77 ns 66 ns 60 ns
..............................................................
folly::MPMC read 195 ns 186 ns 175 ns
folly::MPMC tryReadUntil 204 ns 187 ns 167 ns
folly::MPMC blockingRead 66 ns 60 ns 57 ns
==============================================================
====================== 4 prod 32 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 246 ns 210 ns 195 ns
Unbounded MPMC timed spin only 217 ns 207 ns 199 ns
Unbounded MPMC wait spin only 66 ns 52 ns 46 ns
Unbounded MPMC try may block 250 ns 207 ns 197 ns
Unbounded MPMC timed may block 208 ns 202 ns 195 ns
Unbounded MPMC wait may block 80 ns 66 ns 56 ns
..............................................................
folly::MPMC read 231 ns 201 ns 190 ns
folly::MPMC tryReadUntil 202 ns 199 ns 196 ns
folly::MPMC blockingRead 65 ns 61 ns 57 ns
==============================================================
====================== 8 prod 2 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 50 ns 41 ns 39 ns
Unbounded MPMC timed spin only 73 ns 49 ns 40 ns
Unbounded MPMC wait spin only 46 ns 43 ns 39 ns
Unbounded MPMC try may block 81 ns 62 ns 56 ns
Unbounded MPMC timed may block 75 ns 61 ns 53 ns
Unbounded MPMC wait may block 61 ns 57 ns 50 ns
..............................................................
folly::MPMC read 120 ns 102 ns 58 ns
folly::MPMC tryReadUntil 119 ns 112 ns 103 ns
folly::MPMC blockingRead 85 ns 71 ns 58 ns
==============================================================
====================== 8 prod 4 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 104 ns 87 ns 39 ns
Unbounded MPMC timed spin only 109 ns 89 ns 40 ns
Unbounded MPMC wait spin only 46 ns 45 ns 43 ns
Unbounded MPMC try may block 121 ns 101 ns 74 ns
Unbounded MPMC timed may block 116 ns 103 ns 72 ns
Unbounded MPMC wait may block 62 ns 57 ns 52 ns
..............................................................
folly::MPMC read 136 ns 130 ns 118 ns
folly::MPMC tryReadUntil 132 ns 127 ns 118 ns
folly::MPMC blockingRead 68 ns 61 ns 51 ns
==============================================================
====================== 8 prod 8 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 175 ns 171 ns 162 ns
Unbounded MPMC timed spin only 177 ns 169 ns 159 ns
Unbounded MPMC wait spin only 49 ns 47 ns 45 ns
Unbounded MPMC try may block 175 ns 171 ns 156 ns
Unbounded MPMC timed may block 180 ns 170 ns 162 ns
Unbounded MPMC wait may block 63 ns 62 ns 59 ns
..............................................................
folly::MPMC read 177 ns 162 ns 147 ns
folly::MPMC tryReadUntil 170 ns 162 ns 148 ns
folly::MPMC blockingRead 57 ns 53 ns 49 ns
==============================================================
====================== 8 prod 16 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 203 ns 192 ns 185 ns
Unbounded MPMC timed spin only 199 ns 193 ns 185 ns
Unbounded MPMC wait spin only 48 ns 46 ns 44 ns
Unbounded MPMC try may block 204 ns 194 ns 182 ns
Unbounded MPMC timed may block 198 ns 187 ns 171 ns
Unbounded MPMC wait may block 63 ns 61 ns 57 ns
..............................................................
folly::MPMC read 193 ns 185 ns 167 ns
folly::MPMC tryReadUntil 199 ns 188 ns 164 ns
folly::MPMC blockingRead 57 ns 52 ns 49 ns
==============================================================
====================== 8 prod 32 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 222 ns 208 ns 198 ns
Unbounded MPMC timed spin only 234 ns 212 ns 203 ns
Unbounded MPMC wait spin only 89 ns 58 ns 45 ns
Unbounded MPMC try may block 234 ns 207 ns 196 ns
Unbounded MPMC timed may block 205 ns 203 ns 197 ns
Unbounded MPMC wait may block 65 ns 63 ns 61 ns
..............................................................
folly::MPMC read 240 ns 204 ns 194 ns
folly::MPMC tryReadUntil 205 ns 202 ns 199 ns
folly::MPMC blockingRead 56 ns 52 ns 49 ns
==============================================================
====================== 16 prod 2 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 52 ns 40 ns 34 ns
Unbounded MPMC timed spin only 63 ns 47 ns 36 ns
Unbounded MPMC wait spin only 45 ns 39 ns 36 ns
Unbounded MPMC try may block 62 ns 51 ns 47 ns
Unbounded MPMC timed may block 77 ns 52 ns 46 ns
Unbounded MPMC wait may block 63 ns 50 ns 46 ns
..............................................................
folly::MPMC read 114 ns 103 ns 77 ns
folly::MPMC tryReadUntil 116 ns 106 ns 85 ns
folly::MPMC blockingRead 85 ns 79 ns 63 ns
==============================================================
====================== 16 prod 4 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 106 ns 68 ns 33 ns
Unbounded MPMC timed spin only 88 ns 56 ns 36 ns
Unbounded MPMC wait spin only 46 ns 39 ns 35 ns
Unbounded MPMC try may block 95 ns 66 ns 47 ns
Unbounded MPMC timed may block 80 ns 57 ns 46 ns
Unbounded MPMC wait may block 52 ns 48 ns 45 ns
..............................................................
folly::MPMC read 121 ns 113 ns 104 ns
folly::MPMC tryReadUntil 119 ns 110 ns 101 ns
folly::MPMC blockingRead 65 ns 62 ns 57 ns
==============================================================
====================== 16 prod 8 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 153 ns 109 ns 46 ns
Unbounded MPMC timed spin only 167 ns 110 ns 36 ns
Unbounded MPMC wait spin only 43 ns 39 ns 36 ns
Unbounded MPMC try may block 159 ns 125 ns 100 ns
Unbounded MPMC timed may block 127 ns 82 ns 52 ns
Unbounded MPMC wait may block 51 ns 50 ns 46 ns
..............................................................
folly::MPMC read 149 ns 139 ns 129 ns
folly::MPMC tryReadUntil 141 ns 134 ns 112 ns
folly::MPMC blockingRead 59 ns 54 ns 49 ns
==============================================================
====================== 16 prod 16 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 193 ns 169 ns 148 ns
Unbounded MPMC timed spin only 221 ns 175 ns 106 ns
Unbounded MPMC wait spin only 45 ns 41 ns 37 ns
Unbounded MPMC try may block 204 ns 171 ns 133 ns
Unbounded MPMC timed may block 184 ns 162 ns 104 ns
Unbounded MPMC wait may block 61 ns 52 ns 49 ns
..............................................................
folly::MPMC read 181 ns 164 ns 157 ns
folly::MPMC tryReadUntil 185 ns 173 ns 157 ns
folly::MPMC blockingRead 56 ns 50 ns 45 ns
==============================================================
====================== 16 prod 32 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 255 ns 217 ns 181 ns
Unbounded MPMC timed spin only 225 ns 205 ns 182 ns
Unbounded MPMC wait spin only 115 ns 57 ns 40 ns
Unbounded MPMC try may block 215 ns 199 ns 184 ns
Unbounded MPMC timed may block 218 ns 196 ns 179 ns
Unbounded MPMC wait may block 63 ns 54 ns 47 ns
..............................................................
folly::MPMC read 260 ns 205 ns 185 ns
folly::MPMC tryReadUntil 205 ns 200 ns 192 ns
folly::MPMC blockingRead 53 ns 48 ns 43 ns
==============================================================
====================== 32 prod 2 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 95 ns 66 ns 45 ns
Unbounded MPMC timed spin only 95 ns 62 ns 45 ns
Unbounded MPMC wait spin only 56 ns 44 ns 36 ns
Unbounded MPMC try may block 123 ns 86 ns 50 ns
Unbounded MPMC timed may block 109 ns 73 ns 47 ns
Unbounded MPMC wait may block 95 ns 58 ns 47 ns
..............................................................
folly::MPMC read 445 ns 380 ns 315 ns
folly::MPMC tryReadUntil 459 ns 341 ns 153 ns
folly::MPMC blockingRead 351 ns 286 ns 218 ns
==============================================================
====================== 32 prod 4 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 114 ns 92 ns 59 ns
Unbounded MPMC timed spin only 135 ns 99 ns 47 ns
Unbounded MPMC wait spin only 139 ns 55 ns 38 ns
Unbounded MPMC try may block 165 ns 113 ns 72 ns
Unbounded MPMC timed may block 119 ns 94 ns 51 ns
Unbounded MPMC wait may block 61 ns 52 ns 47 ns
..............................................................
folly::MPMC read 127 ns 112 ns 93 ns
folly::MPMC tryReadUntil 116 ns 107 ns 96 ns
folly::MPMC blockingRead 67 ns 59 ns 51 ns
==============================================================
====================== 32 prod 8 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 226 ns 140 ns 57 ns
Unbounded MPMC timed spin only 176 ns 126 ns 61 ns
Unbounded MPMC wait spin only 86 ns 50 ns 39 ns
Unbounded MPMC try may block 170 ns 131 ns 76 ns
Unbounded MPMC timed may block 201 ns 141 ns 110 ns
Unbounded MPMC wait may block 94 ns 55 ns 47 ns
..............................................................
folly::MPMC read 148 ns 131 ns 120 ns
folly::MPMC tryReadUntil 132 ns 126 ns 121 ns
folly::MPMC blockingRead 59 ns 54 ns 51 ns
==============================================================
====================== 32 prod 16 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 209 ns 174 ns 146 ns
Unbounded MPMC timed spin only 214 ns 189 ns 154 ns
Unbounded MPMC wait spin only 138 ns 51 ns 38 ns
Unbounded MPMC try may block 247 ns 191 ns 144 ns
Unbounded MPMC timed may block 245 ns 180 ns 123 ns
Unbounded MPMC wait may block 74 ns 51 ns 46 ns
..............................................................
folly::MPMC read 164 ns 148 ns 135 ns
folly::MPMC tryReadUntil 156 ns 149 ns 140 ns
folly::MPMC blockingRead 55 ns 50 ns 47 ns
==============================================================
====================== 32 prod 32 cons ======================
=== uint32_t =================================================
Unbounded MPMC try spin only 255 ns 212 ns 179 ns
Unbounded MPMC timed spin only 391 ns 223 ns 147 ns
Unbounded MPMC wait spin only 78 ns 44 ns 38 ns
Unbounded MPMC try may block 516 ns 249 ns 195 ns
Unbounded MPMC timed may block 293 ns 210 ns 171 ns
Unbounded MPMC wait may block 54 ns 51 ns 48 ns
..............................................................
folly::MPMC read 195 ns 183 ns 164 ns
folly::MPMC tryReadUntil 191 ns 175 ns 159 ns
folly::MPMC blockingRead 49 ns 45 ns 43 ns
==============================================================
$ lscpu
Architecture: x86_64
CPU op-mode(s): 32-bit, 64-bit
Byte Order: Little Endian
CPU(s): 32
On-line CPU(s) list: 0-31
Thread(s) per core: 2
Core(s) per socket: 8
Socket(s): 2
NUMA node(s): 2
Vendor ID: GenuineIntel
CPU family: 6
Model: 45
Model name: Intel(R) Xeon(R) CPU E5-2660 0 @ 2.20GHz
Stepping: 6
CPU MHz: 2200.000
CPU max MHz: 2200.0000
CPU min MHz: 1200.0000
BogoMIPS: 4399.92
Virtualization: VT-x
L1d cache: 32K
L1i cache: 32K
L2 cache: 256K
L3 cache: 20480K
NUMA node0 CPU(s): 0-7,16-23
NUMA node1 CPU(s): 8-15,24-31
Flags: fpu vme de pse tsc msr pae mce cx8 apic sep mtrr
pge mca cmov pat pse36 clflush dts acpi mmx fxsr
sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp
lm constant_tsc arch_perfmon pebs bts rep_good
nopl xtopology nonstop_tsc aperfmperf eagerfpu
pni pclmulqdq dtes64 monitor ds_cpl vmx smx est
tm2 ssse3 cx16 xtpr pdcm pcid dca sse4_1 sse4_2
x2apic popcnt tsc_deadline_timer aes xsave avx
lahf_lm epb tpr_shadow vnmi flexpriority ept vpid
xsaveopt dtherm arat pln pts
*/
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment