From 886a70d9164fb1e25e7d450e3841937e9fb6f55a Mon Sep 17 00:00:00 2001 From: Aaryaman Sagar <aary@instagram.com> Date: Wed, 12 Jun 2019 23:33:02 -0700 Subject: [PATCH] Optimizations, support for exceptions and big return values for lock_combine Summary: Some optimizations and changes to make mutex migrations easier: - Add exception handling support, this allows using lock_combine pretty much anywhere a unique_lock would be used and makes transitioning between lock methods easier and more efficient as users aren't required to maintain their own unions anywhere (eg. with folly::Try) - Add support for big return values so people can return anything from the critical section. Without this, users would have to use code of the following form, which is prone to false sharing with metadata for the waiting thread ``` auto value = ReturnValue{}; mutex.lock_combine([&]() { value = critical_section(); }); ``` - Add some optimizations like inlining the combine codepath and an optimistic load to elide a branch. This gets us a ~8% throughput improvement from before. More importantly, This prevents compilers from messing up the generated code to dereference the waiter node whenever they feel like. - Defer time publishing for combinable threads until a preemption. This gets us to the same level of efficiency as std::atomic even on broadwell, takes us to 7x of the baseline (std::mutex) on the NUMA-less machines and almost perfectly parallel in the moderate concurrency levels. I suspect we can do better with NUMA-awareness, but that's for another diff Reviewed By: yfeldblum Differential Revision: D15522658 fbshipit-source-id: 420f4202503305d57b6bd59a9a4ecb67d4dd3c2e --- folly/synchronization/DistributedMutex-inl.h | 387 +++++-- folly/synchronization/DistributedMutex.h | 10 +- .../test/DistributedMutexTest.cpp | 505 ++++++--- .../test/SmallLocksBenchmark.cpp | 968 +++++++++--------- 4 files changed, 1185 insertions(+), 685 deletions(-) diff --git a/folly/synchronization/DistributedMutex-inl.h b/folly/synchronization/DistributedMutex-inl.h index 0ede38b04..ae2ef1c68 100644 --- a/folly/synchronization/DistributedMutex-inl.h +++ b/folly/synchronization/DistributedMutex-inl.h @@ -129,6 +129,18 @@ constexpr auto kCombineUninitialized = std::uint32_t{0b1000}; // lock holder that the thread has set its next_ pointer in the contention // chain constexpr auto kCombineWaiting = std::uint32_t{0b1001}; +// kExceptionOccurred is set on the waiter futex when the remote task throws +// an exception. It is the caller's responsibility to retrieve the exception +// and rethrow it in their own context. Note that when the caller uses a +// noexcept function as their critical section, they can avoid checking for +// this value +// +// This allows us to avoid all cost of exceptions in the memory layout of the +// fast path (no errors) as exceptions are stored as an std::exception_ptr in +// the same union that stores the return value of the critical section. We +// also avoid all CPU overhead because the combiner uses a try-catch block +// without any additional branching to handle exceptions +constexpr auto kExceptionOccurred = std::uint32_t{0b1010}; // The number of spins that we are allowed to do before we resort to marking a // thread as having slept @@ -244,7 +256,29 @@ class Waiter { kUninitialized}; // The successor of this node. This will be the thread that had its address // on the mutex previously - std::uintptr_t next_{0}; + // + // We can do without making this atomic since the remote thread synchronizes + // on the futex variable above. If this were not atomic, the remote thread + // would only be allowed to read from it after the waiter has moved into the + // waiting state to avoid risk of a load racing with a write. However, it + // helps to make this atomic because we can use an unconditional load and make + // full use of the load buffer to coalesce both reads into a single clock + // cycle after the line arrives in the combiner core. This is a heavily + // contended line, so an RFO from the enqueueing thread is highly likely and + // has the potential to cause an immediate invalidation; blocking the combiner + // thread from making progress until the line is pulled back to read this + // value + // + // Further, making this atomic prevents the compiler from making an incorrect + // optimization where it does not load the value as written in the code, but + // rather dereferences it through a pointer whenever needed (since the value + // of the pointer to this is readily available on the stack). Doing this + // causes multiple invalidation requests from the enqueueing thread, blocking + // remote progress + // + // Note that we use relaxed loads and stores, so this should not have any + // additional overhead compared to a regular load on most architectures + std::atomic<std::uintptr_t> next_{0}; // We use an anonymous union for the combined critical section request and // the metadata that will be filled in from the leader's end. Only one is // active at a time - if a leader decides to combine the requested critical @@ -410,9 +444,11 @@ using Request = std::conditional_t< /** * A template that helps us to transform a callable returning a value to one - * that returns void so it can be type erased and passed on to the waker. The - * return value gets coalesced into the wait struct when it is small enough - * for optimal data transfer + * that returns void so it can be type erased and passed on to the waker. If + * the return value is small enough, it gets coalesced into the wait struct + * for optimal data transfer. When it's not small enough to fit in the waiter + * storage buffer, we place it on it's own cacheline with isolation to prevent + * false-sharing with the on-stack metadata of the waiter thread * * This helps a combined critical section feel more normal in the case where * the user wants to return a value, for example @@ -437,6 +473,7 @@ template <typename Func, typename Waiter> class TaskWithCoalesce { public: using ReturnType = folly::invoke_result_t<const Func&>; + using StorageType = folly::Unit; explicit TaskWithCoalesce(Func func, Waiter& waiter) : func_{std::move(func)}, waiter_{waiter} {} @@ -449,6 +486,7 @@ class TaskWithCoalesce { Func func_; Waiter& waiter_; + static_assert(!std::is_void<ReturnType>{}, ""); static_assert(alignof(decltype(waiter_.storage_)) >= alignof(ReturnType), ""); static_assert(sizeof(decltype(waiter_.storage_)) >= sizeof(ReturnType), ""); }; @@ -457,6 +495,7 @@ template <typename Func, typename Waiter> class TaskWithoutCoalesce { public: using ReturnType = void; + using StorageType = folly::Unit; explicit TaskWithoutCoalesce(Func func, Waiter&) : func_{std::move(func)} {} void operator()() const { @@ -467,6 +506,52 @@ class TaskWithoutCoalesce { Func func_; }; +template <typename Func, typename Waiter> +class TaskWithBigReturnValue { + public: + // Using storage that is aligned on the cacheline boundary helps us avoid a + // situation where the data ends up being allocated on two separate + // cachelines. This would require the remote thread to pull in both lines + // to issue a write. + // + // We also isolate the storage by appending some padding to the end to + // ensure we avoid false-sharing with the metadata used while the waiter + // waits + using ReturnType = folly::invoke_result_t<const Func&>; + static const auto kReturnValueAlignment = std::max( + alignof(ReturnType), + folly::hardware_destructive_interference_size); + using StorageType = std::aligned_storage_t< + sizeof(std::aligned_storage_t<sizeof(ReturnType), kReturnValueAlignment>), + kReturnValueAlignment>; + + explicit TaskWithBigReturnValue(Func func, Waiter&) + : func_{std::move(func)} {} + + void operator()() const { + DCHECK(storage_); + auto value = func_(); + new (storage_) ReturnType{std::move(value)}; + } + + void attach(StorageType* storage) { + DCHECK(!storage_); + storage_ = storage; + } + + private: + Func func_; + StorageType* storage_{nullptr}; + + static_assert(!std::is_void<ReturnType>{}, ""); + static_assert(sizeof(Waiter::storage_) < sizeof(ReturnType), ""); +}; + +template <typename T, typename = std::enable_if_t<true>> +constexpr const auto Sizeof = sizeof(T); +template <typename T> +constexpr const auto Sizeof<T, std::enable_if_t<std::is_void<T>{}>> = 0; + // we need to use std::integral_constant::value here as opposed to // std::integral_constant::operator T() because MSVC errors out with the // implicit conversion @@ -474,7 +559,10 @@ template <typename Func, typename Waiter> using CoalescedTask = std::conditional_t< std::is_void<folly::invoke_result_t<const Func&>>::value, TaskWithoutCoalesce<Func, Waiter>, - TaskWithCoalesce<Func, Waiter>>; + std::conditional_t< + Sizeof<folly::invoke_result_t<const Func&>> <= sizeof(Waiter::storage_), + TaskWithCoalesce<Func, Waiter>, + TaskWithBigReturnValue<Func, Waiter>>>; /** * Given a request and a wait node, coalesce them into a CoalescedTask that @@ -497,26 +585,120 @@ CoalescedTask<Func, Waiter> coalesce(Request& request, Waiter& waiter) { return CoalescedTask<Func, Waiter>{request.func_, waiter}; } +/** + * Given a task, create storage for the return value. When we get a type + * of CoalescedTask, this returns an instance of CoalescedTask::StorageType. + * std::nullptr_t otherwise + */ +inline std::nullptr_t makeReturnValueStorageFor(std::nullptr_t&) { + return {}; +} + +template < + typename CoalescedTask, + typename StorageType = typename CoalescedTask::StorageType> +StorageType makeReturnValueStorageFor(CoalescedTask&) { + return {}; +} + +/** + * Given a task and storage, attach them together if needed. This only helps + * when we have a task that returns a value bigger than can be coalesced. In + * that case, we need to attach the storage with the task so the return value + * can be transferred to this thread from the remote thread + */ +template <typename Task, typename Storage> +void attach(Task&, Storage&) { + static_assert( + std::is_same<Storage, std::nullptr_t>{} || + std::is_same<Storage, folly::Unit>{}, + ""); +} + +template < + typename R, + typename W, + typename StorageType = typename TaskWithBigReturnValue<R, W>::StorageType> +void attach(TaskWithBigReturnValue<R, W>& task, StorageType& storage) { + task.attach(&storage); +} + +template <typename Request, typename Waiter> +void throwIfExceptionOccurred(Request&, Waiter& waiter, bool exception) { + using Storage = decltype(waiter.storage_); + using F = typename Request::F; + static_assert(sizeof(Storage) >= sizeof(std::exception_ptr), ""); + static_assert(alignof(Storage) >= alignof(std::exception_ptr), ""); + + // we only need to check for an exception in the waiter struct if the passed + // callable is not noexcept + // + // we need to make another instance of the exception with automatic storage + // duration and destroy the exception held in the storage *before throwing* to + // avoid leaks. If we don't destroy the exception_ptr in storage, the + // refcount for the internal exception will never hit zero, thereby leaking + // memory + if (UNLIKELY(!folly::is_nothrow_invocable<const F&>{} && exception)) { + auto storage = &waiter.storage_; + auto exc = folly::launder(reinterpret_cast<std::exception_ptr*>(storage)); + auto copy = std::move(*exc); + exc->std::exception_ptr::~exception_ptr(); + std::rethrow_exception(std::move(copy)); + } +} + /** * Given a CoalescedTask, a wait node and a request. Detach the return value * into the request from the wait node and task. */ template <typename Waiter> -void detach(std::nullptr_t&, Waiter&) {} +void detach(std::nullptr_t&, Waiter&, bool exception, std::nullptr_t&) { + DCHECK(!exception); +} template <typename Waiter, typename F> -void detach(RequestWithoutReturn<F>&, Waiter&) {} +void detach( + RequestWithoutReturn<F>& request, + Waiter& waiter, + bool exception, + folly::Unit&) { + throwIfExceptionOccurred(request, waiter, exception); +} template <typename Waiter, typename F> -void detach(RequestWithReturn<F>& request, Waiter& waiter) { +void detach( + RequestWithReturn<F>& request, + Waiter& waiter, + bool exception, + folly::Unit&) { + throwIfExceptionOccurred(request, waiter, exception); + using ReturnType = typename RequestWithReturn<F>::ReturnType; static_assert(!std::is_same<ReturnType, void>{}, ""); + static_assert(sizeof(waiter.storage_) >= sizeof(ReturnType), ""); auto& val = *folly::launder(reinterpret_cast<ReturnType*>(&waiter.storage_)); new (&request.value_) ReturnType{std::move(val)}; val.~ReturnType(); } +template <typename Waiter, typename F, typename Storage> +void detach( + RequestWithReturn<F>& request, + Waiter& waiter, + bool exception, + Storage& storage) { + throwIfExceptionOccurred(request, waiter, exception); + + using ReturnType = typename RequestWithReturn<F>::ReturnType; + static_assert(!std::is_same<ReturnType, void>{}, ""); + static_assert(sizeof(storage) >= sizeof(ReturnType), ""); + + auto& val = *folly::launder(reinterpret_cast<ReturnType*>(&storage)); + new (&request.value_) ReturnType{std::move(val)}; + val.~ReturnType(); +} + /** * Get the time since epoch in nanoseconds * @@ -662,27 +844,73 @@ template <template <typename> class Atomic, bool TimePublishing> DistributedMutex<Atomic, TimePublishing>::DistributedMutex() : state_{kUnlocked} {} +template <typename Waiter> +std::uint64_t publish( + std::uint64_t spins, + bool& shouldPublish, + std::chrono::nanoseconds& previous, + Waiter& waiter, + std::uint32_t waitMode) { + // time publishing has some overhead because it executes an atomic exchange on + // the futex word. If this line is in a remote thread (eg. the combiner), + // then each time we publish a timestamp, this thread has to submit an RFO to + // the remote core for the cacheline, blocking progress for both threads. + // + // the remote core uses a store in the fast path - why then does an RFO make a + // difference? The only educated guess we have here is that the added + // roundtrip delays draining of the store buffer, which essentially exerts + // backpressure on future stores, preventing parallelization + // + // if we have requested a combine, time publishing is less important as it + // only comes into play when the combiner has exhausted their max combine + // passes. So we defer time publishing to the point when the current thread + // gets preempted + auto current = time(); + if ((current - previous) >= kScheduledAwaySpinThreshold) { + shouldPublish = true; + } + previous = current; + + // if we have requested a combine, and this is the first iteration of the + // wait-loop, we publish a max timestamp to optimistically convey that we have + // not yet been preempted (the remote knows the meaning of max timestamps) + // + // then if we are under the maximum number of spins allowed before sleeping, + // we publish the exact timestamp, otherwise we publish the minimum possible + // timestamp to force the waking thread to skip us + auto now = ((waitMode == kCombineWaiting) && !spins) + ? decltype(time())::max() + : (spins < kMaxSpins) ? previous : decltype(time())::zero(); + + // the wait mode information is published in the bottom 8 bits of the futex + // word, the rest contains time information as computed above. Overflows are + // not really a correctness concern because time publishing is only a + // heuristic. This leaves us 56 bits of nanoseconds (2 years) before we hit + // two consecutive wraparounds, so the lack of bits to respresent time is + // neither a performance nor correctness concern + auto data = strip(now) | waitMode; + auto signal = (shouldPublish || !spins || (waitMode != kCombineWaiting)) + ? waiter.futex_.exchange(data, std::memory_order_acq_rel) + : waiter.futex_.load(std::memory_order_acquire); + return signal & std::numeric_limits<std::uint8_t>::max(); +} + template <typename Waiter> bool spin(Waiter& waiter, std::uint32_t& sig, std::uint32_t mode) { - auto spins = 0; + auto spins = std::uint64_t{0}; auto waitMode = (mode == kCombineUninitialized) ? kCombineWaiting : kWaiting; + auto previous = time(); + auto shouldPublish = false; while (true) { - // publish our current time in the futex as a part of the spin waiting - // process - // - // if we are under the maximum number of spins allowed before sleeping, we - // publish the exact timestamp, otherwise we publish the minimum possible - // timestamp to force the waking thread to skip us - ++spins; - auto now = (spins < kMaxSpins) ? time() : decltype(time())::zero(); - auto data = strip(now) | waitMode; - auto signal = waiter.futex_.exchange(data, std::memory_order_acq_rel); - signal &= std::numeric_limits<std::uint8_t>::max(); + auto signal = publish(spins++, shouldPublish, previous, waiter, waitMode); // if we got skipped, make a note of it and return if we got a skipped // signal or a signal to wake up auto skipped = (signal == kSkipped); - if (skipped || (signal == kWake) || (signal == kCombined)) { + auto combined = (signal == kCombined); + auto exceptionOccurred = (signal == kExceptionOccurred); + auto woken = (signal == kWake); + if (skipped || woken || combined || exceptionOccurred) { sig = static_cast<std::uint32_t>(signal); return !skipped; } @@ -781,7 +1009,7 @@ bool doFutexWait(Waiter* waiter, Waiter*& next) { // when coming out of a futex, we might have some other sleeping threads // that we were supposed to wake up, assign that to the next pointer DCHECK(next == nullptr); - next = extractPtr<Waiter>(waiter->next_); + next = extractPtr<Waiter>(waiter->next_.load(std::memory_order_relaxed)); return false; } @@ -819,7 +1047,7 @@ void wakeTimedWaiters(Atomic* state, bool timedWaiters) { template <template <typename> class Atomic, bool TimePublishing> template <typename Func> -auto DistributedMutex<Atomic, TimePublishing>::lock_combine(Func func) noexcept +auto DistributedMutex<Atomic, TimePublishing>::lock_combine(Func func) -> folly::invoke_result_t<const Func&> { // invoke the lock implementation function and check whether we came out of // it with our task executed as a combined critical section. This usually @@ -867,7 +1095,7 @@ template <typename Rep, typename Period, typename Func, typename ReturnType> folly::Optional<ReturnType> DistributedMutex<Atomic, TimePublishing>::try_lock_combine_for( const std::chrono::duration<Rep, Period>& duration, - Func func) noexcept { + Func func) { auto state = try_lock_for(duration); if (state) { SCOPE_EXIT { @@ -884,7 +1112,7 @@ template <typename Clock, typename Duration, typename Func, typename ReturnType> folly::Optional<ReturnType> DistributedMutex<Atomic, TimePublishing>::try_lock_combine_until( const std::chrono::time_point<Clock, Duration>& deadline, - Func func) noexcept { + Func func) { auto state = try_lock_until(deadline); if (state) { SCOPE_EXIT { @@ -967,7 +1195,9 @@ lockImplementation( // constructor Waiter<Atomic> state{}; auto&& task = coalesce(request, state); + auto&& storage = makeReturnValueStorageFor(task); auto&& address = folly::bit_cast<std::uintptr_t>(&state); + attach(task, storage); state.initialize(waitMode, std::move(task)); DCHECK(!(address & 0b1)); @@ -986,7 +1216,7 @@ lockImplementation( // was unsuccessful previous = atomic.exchange(address, std::memory_order_acq_rel); recordTimedWaiterAndClearTimedBit(timedWaiter, previous); - state.next_ = previous; + state.next_.store(previous, std::memory_order_relaxed); if (previous == kUnlocked) { return {/* next */ nullptr, /* expected */ address, @@ -1034,8 +1264,10 @@ lockImplementation( // if we were given a combine signal, detach the return value from the // wait struct into the request, so the current thread can access it // outside this function - if (signal == kCombined) { - detach(request, state); + auto combined = (signal == kCombined); + auto exceptionOccurred = (signal == kExceptionOccurred); + if (combined || exceptionOccurred) { + detach(request, state, exceptionOccurred, storage); } // if we are just coming out of a futex call, then it means that the next @@ -1045,7 +1277,7 @@ lockImplementation( return {/* next */ extractPtr<Waiter<Atomic>>(next), /* expected */ expected, /* timedWaiter */ timedWaiter, - /* combined */ combineRequested && (signal == kCombined), + /* combined */ combineRequested && (combined || exceptionOccurred), /* waker */ state.metadata_.waker_, /* waiters */ extractPtr<Waiter<Atomic>>(state.metadata_.waiters_), /* ready */ nextSleeper}; @@ -1055,7 +1287,9 @@ lockImplementation( inline bool preempted(std::uint64_t value, std::chrono::nanoseconds now) { auto currentTime = recover(strip(now)); auto nodeTime = recover(value); - auto preempted = currentTime > nodeTime + kScheduledAwaySpinThreshold.count(); + auto preempted = + (currentTime > nodeTime + kScheduledAwaySpinThreshold.count()) && + (nodeTime != recover(strip(std::chrono::nanoseconds::max()))); // we say that the thread has been preempted if its timestamp says so, and // also if it is neither uninitialized nor skipped @@ -1093,46 +1327,21 @@ CombineFunction loadTask(Waiter* current, std::uintptr_t value) { return nullptr; } +template <typename Waiter> +FOLLY_COLD void transferCurrentException(Waiter* waiter) { + DCHECK(std::current_exception()); + new (&waiter->storage_) std::exception_ptr{std::current_exception()}; + waiter->futex_.store(kExceptionOccurred, std::memory_order_release); +} + template <template <typename> class Atomic> -std::uintptr_t tryCombine( - std::uintptr_t value, +FOLLY_ALWAYS_INLINE std::uintptr_t tryCombine( Waiter<Atomic>* waiter, + std::uintptr_t value, + std::uintptr_t next, std::uint64_t iteration, std::chrono::nanoseconds now, CombineFunction task) { - // it is important to load the value of next_ before checking the value of - // function_ in the next if condition. This is because of two things, the - // first being cache locality - it is helpful to read the value of the - // variable that is closer to futex_, since we just loaded from that before - // entering this function. The second is cache coherence, the wait struct - // is shared between two threads, one thread is spinning on the futex - // waiting for a signal while the other is possibly combining the requested - // critical section into its own. This means that there is a high chance - // we would cause the cachelines to bounce between the threads in the next - // if block. - // - // This leads to a degenerate case where the FunctionRef object ends up in a - // different cacheline thereby making it seem like benchmarks avoid this - // problem. When compiled differently (eg. with link time optimization) - // the wait struct ends up on the stack in a manner that causes the - // FunctionRef object to be in the same cacheline as the other data, thereby - // forcing the current thread to bounce on the cacheline twice (first to - // load the data from the other thread, that presumably owns the cacheline - // due to timestamp publishing) and then to signal the thread - // - // To avoid this sort of non-deterministic behavior based on compilation and - // stack layout, we load the value before executing the other thread's - // critical section - // - // Note that the waiting thread writes the value to the wait struct after - // enqueuing, but never writes to it after the value in the futex_ is - // initialised (showing that the thread is in the spin loop), this makes it - // safe for us to read next_ without synchronization - auto next = std::uintptr_t{0}; - if (isInitialized(value)) { - next = waiter->next_; - } - // if the waiter has asked for a combine operation, we should combine its // critical section and move on to the next waiter // @@ -1147,14 +1356,18 @@ std::uintptr_t tryCombine( // leading to further delays in critical section completion // // if all the above are satisfied, then we can combine the critical section. - // Note that it is only safe to read from the waiter struct if the value is - // not uninitialized. If the state is uninitialized, we synchronize with - // the write to the next_ member in the lock function. If the value is not - // uninitialized, there is a race in reading the next_ value + // Note that if the waiter is in a combineable state, that means that it had + // finished its writes to both the task and the next_ value. And observing + // a waiting state also means that we have acquired the writes to the other + // members of the waiter struct, so it's fine to use those values here if (isWaitingCombiner(value) && (iteration <= kMaxCombineIterations || preempted(value, now))) { - task(); - waiter->futex_.store(kCombined, std::memory_order_release); + try { + task(); + waiter->futex_.store(kCombined, std::memory_order_release); + } catch (...) { + transferCurrentException(waiter); + } return next; } @@ -1162,10 +1375,11 @@ std::uintptr_t tryCombine( } template <typename Waiter> -std::uintptr_t tryWake( +FOLLY_ALWAYS_INLINE std::uintptr_t tryWake( bool publishing, Waiter* waiter, std::uintptr_t value, + std::uintptr_t next, std::uintptr_t waker, Waiter*& sleepers, std::uint64_t iteration, @@ -1174,7 +1388,7 @@ std::uintptr_t tryWake( // we have successfully executed their critical section and can move on to // the rest of the chain auto now = time(); - if (auto next = tryCombine(value, waiter, iteration, now, task)) { + if (tryCombine(waiter, value, next, iteration, now, task)) { return next; } @@ -1217,7 +1431,7 @@ std::uintptr_t tryWake( // Can we relax this? DCHECK(preempted(value, now)); DCHECK(!isCombiner(value)); - auto next = waiter->next_; + next = waiter->next_.load(std::memory_order_relaxed); waiter->futex_.store(kSkipped, std::memory_order_release); return next; } @@ -1260,8 +1474,9 @@ std::uintptr_t tryWake( // // we also need to collect this sleeper in the list of sleepers being built // up - auto next = waiter->next_; - waiter->next_ = folly::bit_cast<std::uintptr_t>(sleepers); + next = waiter->next_.load(std::memory_order_relaxed); + auto head = folly::bit_cast<std::uintptr_t>(sleepers); + waiter->next_.store(head, std::memory_order_relaxed); sleepers = waiter; return next; } @@ -1278,13 +1493,23 @@ bool wake( // the last published timestamp of the node) auto current = &waiter; while (current) { - // it is important that we load the value of function after the initial - // acquire load. This is required because we need to synchronize with the - // construction of the waiter struct before reading from it + // it is important that we load the value of function and next_ after the + // initial acquire load. This is required because we need to synchronize + // with the construction of the waiter struct before reading from it + // + // the load from the next_ variable is an optimistic load that assumes + // that the waiting thread has probably gone to the waiting state. If the + // waiitng thread is in the waiting state (as revealed by the acquire load + // from the futex word), we will see a well formed next_ value because it + // happens-before the release store to the futex word. The atomic load from + // next_ is an optimization to avoid branching before loading and prevent + // the compiler from eliding the load altogether (and using a pointer + // dereference when needed) auto value = current->futex_.load(std::memory_order_acquire); + auto next = current->next_.load(std::memory_order_relaxed); auto task = loadTask(current, value); - auto next = - tryWake(publishing, current, value, waker, sleepers, iter, task); + next = + tryWake(publishing, current, value, next, waker, sleepers, iter, task); // if there is no next node, we have managed to wake someone up and have // successfully migrated the lock to another thread diff --git a/folly/synchronization/DistributedMutex.h b/folly/synchronization/DistributedMutex.h index 535e8d56f..155e67238 100644 --- a/folly/synchronization/DistributedMutex.h +++ b/folly/synchronization/DistributedMutex.h @@ -276,6 +276,10 @@ class DistributedMutex { * Here, because we used a combined critical section, we have introduced a * dependency from one -> three that might not obvious to the reader * + * This function is exception-safe. If the passed task throws an exception, + * it will be propagated to the caller, even if the task is running on + * another thread + * * There are three notable cases where this method causes undefined * behavior: * @@ -291,7 +295,7 @@ class DistributedMutex { * at compile time or runtime, so we have no checks against this */ template <typename Task> - auto lock_combine(Task task) noexcept -> folly::invoke_result_t<const Task&>; + auto lock_combine(Task task) -> folly::invoke_result_t<const Task&>; /** * Try to combine a task as a combined critical section untill the given time @@ -311,7 +315,7 @@ class DistributedMutex { typename ReturnType = decltype(std::declval<Task&>()())> folly::Optional<ReturnType> try_lock_combine_for( const std::chrono::duration<Rep, Period>& duration, - Task task) noexcept; + Task task); /** * Try to combine a task as a combined critical section untill the given time @@ -326,7 +330,7 @@ class DistributedMutex { typename ReturnType = decltype(std::declval<Task&>()())> folly::Optional<ReturnType> try_lock_combine_until( const std::chrono::time_point<Clock, Duration>& deadline, - Task task) noexcept; + Task task); private: Atomic<std::uintptr_t> state_{0}; diff --git a/folly/synchronization/test/DistributedMutexTest.cpp b/folly/synchronization/test/DistributedMutexTest.cpp index d1a2e7b44..44cf5df1d 100644 --- a/folly/synchronization/test/DistributedMutexTest.cpp +++ b/folly/synchronization/test/DistributedMutexTest.cpp @@ -187,8 +187,8 @@ void atomic_notify_one(const ManualAtomic<std::uintptr_t>*) { } // namespace test namespace { -DEFINE_int32(stress_factor, 1000, "The stress test factor for tests"); -DEFINE_int32(stress_test_seconds, 2, "Duration for stress tests"); +constexpr auto kStressFactor = 1000; +constexpr auto kStressTestSeconds = 2; constexpr auto kForever = 100h; using DSched = test::DeterministicSchedule; @@ -198,7 +198,7 @@ int sum(int n) { } template <template <typename> class Atom = std::atomic> -void basicNThreads(int numThreads, int iterations = FLAGS_stress_factor) { +void basicNThreads(int numThreads, int iterations = kStressFactor) { auto&& mutex = detail::distributed_mutex::DistributedMutex<Atom>{}; auto&& barrier = std::atomic<int>{0}; auto&& threads = std::vector<std::thread>{}; @@ -307,8 +307,12 @@ void combineNThreads(int numThreads, std::chrono::seconds duration) { auto current = mutex.lock_combine([&]() { result.fetch_add(1); EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 0); + EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 1); std::this_thread::yield(); - EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + SCOPE_EXIT { + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + }; + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 2); return local.fetch_add(1); }); EXPECT_EQ(current, expected - 1); @@ -355,8 +359,12 @@ void combineWithLockNThreads(int numThreads, std::chrono::seconds duration) { auto current = mutex.lock_combine([&]() { auto iteration = total.fetch_add(1); EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 0); + EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 1); std::this_thread::yield(); - EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + SCOPE_EXIT { + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + }; + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 2); return iteration; }); @@ -406,8 +414,12 @@ void combineWithTryLockNThreads(int numThreads, std::chrono::seconds duration) { auto current = mutex.lock_combine([&]() { auto iteration = total.fetch_add(1); EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 0); + EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 1); std::this_thread::yield(); - EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + SCOPE_EXIT { + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + }; + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 2); return iteration; }); @@ -474,8 +486,12 @@ void combineWithLockTryAndTimedNThreads( auto current = mutex.lock_combine([&]() { auto iteration = total.fetch_add(1); EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 0); + EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 1); std::this_thread::yield(); - EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + SCOPE_EXIT { + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + }; + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 2); // return a non-trivially-copyable object that occupies all the // storage we use to coalesce returns to test that codepath @@ -753,162 +769,150 @@ TEST(DistributedMutex, StressHardwareConcurrencyThreads) { } TEST(DistributedMutex, StressThreeThreadsLockTryAndTimed) { - lockWithTryAndTimedNThreads( - 3, std::chrono::seconds{FLAGS_stress_test_seconds}); + lockWithTryAndTimedNThreads(3, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSixThreadsLockTryAndTimed) { - lockWithTryAndTimedNThreads( - 6, std::chrono::seconds{FLAGS_stress_test_seconds}); + lockWithTryAndTimedNThreads(6, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressTwelveThreadsLockTryAndTimed) { - lockWithTryAndTimedNThreads( - 12, std::chrono::seconds{FLAGS_stress_test_seconds}); + lockWithTryAndTimedNThreads(12, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressTwentyFourThreadsLockTryAndTimed) { - lockWithTryAndTimedNThreads( - 24, std::chrono::seconds{FLAGS_stress_test_seconds}); + lockWithTryAndTimedNThreads(24, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressFourtyEightThreadsLockTryAndTimed) { - lockWithTryAndTimedNThreads( - 48, std::chrono::seconds{FLAGS_stress_test_seconds}); + lockWithTryAndTimedNThreads(48, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSixtyFourThreadsLockTryAndTimed) { - lockWithTryAndTimedNThreads( - 64, std::chrono::seconds{FLAGS_stress_test_seconds}); + lockWithTryAndTimedNThreads(64, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressHwConcThreadsLockTryAndTimed) { lockWithTryAndTimedNThreads( std::thread::hardware_concurrency(), - std::chrono::seconds{FLAGS_stress_test_seconds}); + std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressTwoThreadsCombine) { - combineNThreads(2, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreads(2, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressThreeThreadsCombine) { - combineNThreads(3, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreads(3, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressFourThreadsCombine) { - combineNThreads(4, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreads(4, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressFiveThreadsCombine) { - combineNThreads(5, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreads(5, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSixThreadsCombine) { - combineNThreads(6, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreads(6, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSevenThreadsCombine) { - combineNThreads(7, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreads(7, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressEightThreadsCombine) { - combineNThreads(8, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreads(8, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSixteenThreadsCombine) { - combineNThreads(16, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreads(16, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressThirtyTwoThreadsCombine) { - combineNThreads(32, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreads(32, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSixtyFourThreadsCombine) { - combineNThreads(64, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreads(64, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressHundredThreadsCombine) { - combineNThreads(100, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreads(100, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressHardwareConcurrencyThreadsCombine) { combineNThreads( std::thread::hardware_concurrency(), - std::chrono::seconds{FLAGS_stress_test_seconds}); + std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressTwoThreadsCombineAndLock) { - combineWithLockNThreads(2, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithLockNThreads(2, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressFourThreadsCombineAndLock) { - combineWithLockNThreads(4, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithLockNThreads(4, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressEightThreadsCombineAndLock) { - combineWithLockNThreads(8, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithLockNThreads(8, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSixteenThreadsCombineAndLock) { - combineWithLockNThreads(16, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithLockNThreads(16, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressThirtyTwoThreadsCombineAndLock) { - combineWithLockNThreads(32, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithLockNThreads(32, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSixtyFourThreadsCombineAndLock) { - combineWithLockNThreads(64, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithLockNThreads(64, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressHardwareConcurrencyThreadsCombineAndLock) { combineWithLockNThreads( std::thread::hardware_concurrency(), - std::chrono::seconds{FLAGS_stress_test_seconds}); + std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressThreeThreadsCombineTryLockAndLock) { - combineWithTryLockNThreads( - 3, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithTryLockNThreads(3, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSixThreadsCombineTryLockAndLock) { - combineWithTryLockNThreads( - 6, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithTryLockNThreads(6, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressTwelveThreadsCombineTryLockAndLock) { - combineWithTryLockNThreads( - 12, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithTryLockNThreads(12, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressTwentyFourThreadsCombineTryLockAndLock) { - combineWithTryLockNThreads( - 24, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithTryLockNThreads(24, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressFourtyEightThreadsCombineTryLockAndLock) { - combineWithTryLockNThreads( - 48, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithTryLockNThreads(48, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSixtyFourThreadsCombineTryLockAndLock) { - combineWithTryLockNThreads( - 64, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineWithTryLockNThreads(64, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressHardwareConcurrencyThreadsCombineTryLockAndLock) { combineWithTryLockNThreads( std::thread::hardware_concurrency(), - std::chrono::seconds{FLAGS_stress_test_seconds}); + std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressThreeThreadsCombineTryLockLockAndTimed) { combineWithLockTryAndTimedNThreads( - 3, std::chrono::seconds{FLAGS_stress_test_seconds}); + 3, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSixThreadsCombineTryLockLockAndTimed) { combineWithLockTryAndTimedNThreads( - 6, std::chrono::seconds{FLAGS_stress_test_seconds}); + 6, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressTwelveThreadsCombineTryLockLockAndTimed) { combineWithLockTryAndTimedNThreads( - 12, std::chrono::seconds{FLAGS_stress_test_seconds}); + 12, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressTwentyFourThreadsCombineTryLockLockAndTimed) { combineWithLockTryAndTimedNThreads( - 24, std::chrono::seconds{FLAGS_stress_test_seconds}); + 24, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressFourtyEightThreadsCombineTryLockLockAndTimed) { combineWithLockTryAndTimedNThreads( - 48, std::chrono::seconds{FLAGS_stress_test_seconds}); + 48, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressSixtyFourThreadsCombineTryLockLockAndTimed) { combineWithLockTryAndTimedNThreads( - 64, std::chrono::seconds{FLAGS_stress_test_seconds}); + 64, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressHwConcurrencyThreadsCombineTryLockLockAndTimed) { combineWithLockTryAndTimedNThreads( std::thread::hardware_concurrency(), - std::chrono::seconds{FLAGS_stress_test_seconds}); + std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressTryLock) { auto&& mutex = DistributedMutex{}; - for (auto i = 0; i < FLAGS_stress_factor; ++i) { + for (auto i = 0; i < kStressFactor; ++i) { while (true) { auto state = mutex.try_lock(); if (state) { @@ -1022,152 +1026,146 @@ TEST(DistributedMutex, DeterministicStressThirtyTwoThreads) { TEST(DistributedMutex, DeterministicStressThreeThreadsLockTryAndTimed) { lockWithTryAndTimedNThreadsDeterministic( - 3, std::chrono::seconds{FLAGS_stress_test_seconds}); + 3, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, DeterministicStressSixThreadsLockTryAndTimed) { lockWithTryAndTimedNThreadsDeterministic( - 6, std::chrono::seconds{FLAGS_stress_test_seconds}); + 6, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, DeterministicStressTwelveThreadsLockTryAndTimed) { lockWithTryAndTimedNThreadsDeterministic( - 12, std::chrono::seconds{FLAGS_stress_test_seconds}); + 12, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, DeterministicStressTwentyFourThreadsLockTryAndTimed) { lockWithTryAndTimedNThreadsDeterministic( - 24, std::chrono::seconds{FLAGS_stress_test_seconds}); + 24, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, DeterministicStressFourtyEightThreadsLockTryAndTimed) { lockWithTryAndTimedNThreadsDeterministic( - 48, std::chrono::seconds{FLAGS_stress_test_seconds}); + 48, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, DeterministicStressSixtyFourThreadsLockTryAndTimed) { lockWithTryAndTimedNThreadsDeterministic( - 64, std::chrono::seconds{FLAGS_stress_test_seconds}); + 64, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, DeterministicStressHwConcThreadsLockTryAndTimed) { lockWithTryAndTimedNThreadsDeterministic( std::thread::hardware_concurrency(), - std::chrono::seconds{FLAGS_stress_test_seconds}); + std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineDeterministicStressTwoThreads) { - combineNThreadsDeterministic( - 2, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreadsDeterministic(2, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineDeterministicStressFourThreads) { - combineNThreadsDeterministic( - 4, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreadsDeterministic(4, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineDeterministicStressEightThreads) { - combineNThreadsDeterministic( - 8, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreadsDeterministic(8, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineDeterministicStressSixteenThreads) { - combineNThreadsDeterministic( - 16, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreadsDeterministic(16, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineDeterministicStressThirtyTwoThreads) { - combineNThreadsDeterministic( - 32, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreadsDeterministic(32, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineDeterministicStressSixtyFourThreads) { - combineNThreadsDeterministic( - 64, std::chrono::seconds{FLAGS_stress_test_seconds}); + combineNThreadsDeterministic(64, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineDeterministicStressHardwareConcurrencyThreads) { combineNThreadsDeterministic( std::thread::hardware_concurrency(), - std::chrono::seconds{FLAGS_stress_test_seconds}); + std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineAndLockDeterministicStressTwoThreads) { combineAndLockNThreadsDeterministic( - 2, std::chrono::seconds{FLAGS_stress_test_seconds}); + 2, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineAndLockDeterministicStressFourThreads) { combineAndLockNThreadsDeterministic( - 4, std::chrono::seconds{FLAGS_stress_test_seconds}); + 4, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineAndLockDeterministicStressEightThreads) { combineAndLockNThreadsDeterministic( - 8, std::chrono::seconds{FLAGS_stress_test_seconds}); + 8, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineAndLockDeterministicStressSixteenThreads) { combineAndLockNThreadsDeterministic( - 16, std::chrono::seconds{FLAGS_stress_test_seconds}); + 16, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineAndLockDeterministicStressThirtyTwoThreads) { combineAndLockNThreadsDeterministic( - 32, std::chrono::seconds{FLAGS_stress_test_seconds}); + 32, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineAndLockDeterministicStressSixtyFourThreads) { combineAndLockNThreadsDeterministic( - 64, std::chrono::seconds{FLAGS_stress_test_seconds}); + 64, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineAndLockDeterministicStressHWConcurrencyThreads) { combineAndLockNThreadsDeterministic( std::thread::hardware_concurrency(), - std::chrono::seconds{FLAGS_stress_test_seconds}); + std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndLockDeterministicStressThreeThreads) { combineTryLockAndLockNThreadsDeterministic( - 3, std::chrono::seconds{FLAGS_stress_test_seconds}); + 3, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndLockDeterministicStressSixThreads) { combineTryLockAndLockNThreadsDeterministic( - 6, std::chrono::seconds{FLAGS_stress_test_seconds}); + 6, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndLockDeterministicStressTwelveThreads) { combineTryLockAndLockNThreadsDeterministic( - 12, std::chrono::seconds{FLAGS_stress_test_seconds}); + 12, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndLockDeterministicStressTwentyThreads) { combineTryLockAndLockNThreadsDeterministic( - 24, std::chrono::seconds{FLAGS_stress_test_seconds}); + 24, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndLockDeterministicStressFortyThreads) { combineTryLockAndLockNThreadsDeterministic( - 48, std::chrono::seconds{FLAGS_stress_test_seconds}); + 48, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndLockDeterministicStressSixtyThreads) { combineTryLockAndLockNThreadsDeterministic( - 64, std::chrono::seconds{FLAGS_stress_test_seconds}); + 64, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndLockDeterministicStressHWConcThreads) { combineTryLockAndLockNThreadsDeterministic( std::thread::hardware_concurrency(), - std::chrono::seconds{FLAGS_stress_test_seconds}); + std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndTimedDeterministicStressThreeThreads) { combineWithTryLockAndTimedNThreadsDeterministic( - 3, std::chrono::seconds{FLAGS_stress_test_seconds}); + 3, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndTimedDeterministicStressSixThreads) { combineWithTryLockAndTimedNThreadsDeterministic( - 6, std::chrono::seconds{FLAGS_stress_test_seconds}); + 6, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndTimedDeterministicStressTwelveThreads) { combineWithTryLockAndTimedNThreadsDeterministic( - 12, std::chrono::seconds{FLAGS_stress_test_seconds}); + 12, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndTimedDeterministicStressTwentyThreads) { combineWithTryLockAndTimedNThreadsDeterministic( - 24, std::chrono::seconds{FLAGS_stress_test_seconds}); + 24, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndTimedDeterministicStressFortyThreads) { combineWithTryLockAndTimedNThreadsDeterministic( - 48, std::chrono::seconds{FLAGS_stress_test_seconds}); + 48, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndTimedDeterministicStressSixtyThreads) { combineWithTryLockAndTimedNThreadsDeterministic( - 64, std::chrono::seconds{FLAGS_stress_test_seconds}); + 64, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, CombineTryLockAndTimedDeterministicStressHWConcThreads) { combineWithTryLockAndTimedNThreadsDeterministic( std::thread::hardware_concurrency(), - std::chrono::seconds{FLAGS_stress_test_seconds}); + std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, TimedLockTimeout) { @@ -1308,7 +1306,7 @@ namespace { template <template <typename> class Atom = std::atomic> void stressTryLockWithConcurrentLocks( int numThreads, - int iterations = FLAGS_stress_factor) { + int iterations = kStressFactor) { auto&& threads = std::vector<std::thread>{}; auto&& mutex = detail::distributed_mutex::DistributedMutex<Atom>{}; auto&& atomic = std::atomic<std::uint64_t>{0}; @@ -1419,7 +1417,7 @@ TEST(DistributedMutex, DeterministicTryLockWithLocksSixtyFourThreads) { namespace { template <template <typename> class Atom = std::atomic> -void concurrentTryLocks(int numThreads, int iterations = FLAGS_stress_factor) { +void concurrentTryLocks(int numThreads, int iterations = kStressFactor) { auto&& threads = std::vector<std::thread>{}; auto&& mutex = detail::distributed_mutex::DistributedMutex<Atom>{}; auto&& atomic = std::atomic<std::uint64_t>{0}; @@ -1643,7 +1641,7 @@ TEST(DistributedMutex, TestAppropriateDestructionAndConstructionWithCombine) { }}; /* sleep override */ - std::this_thread::sleep_for(std::chrono::seconds{FLAGS_stress_test_seconds}); + std::this_thread::sleep_for(std::chrono::seconds{kStressTestSeconds}); stop.store(true); thread.join(); } @@ -1670,8 +1668,12 @@ void concurrentLocksManyMutexes(int numThreads, std::chrono::seconds duration) { ++expected; auto result = mutex.lock_combine([&]() { EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 0); + EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 1); std::this_thread::yield(); - EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + SCOPE_EXIT { + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + }; + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 2); return total.fetch_add(1, std::memory_order_relaxed); }); EXPECT_EQ(result, expected - 1); @@ -1691,28 +1693,22 @@ void concurrentLocksManyMutexes(int numThreads, std::chrono::seconds duration) { } // namespace TEST(DistributedMutex, StressWithManyMutexesAlternatingTwoThreads) { - concurrentLocksManyMutexes( - 2, std::chrono::seconds{FLAGS_stress_test_seconds}); + concurrentLocksManyMutexes(2, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressWithManyMutexesAlternatingFourThreads) { - concurrentLocksManyMutexes( - 4, std::chrono::seconds{FLAGS_stress_test_seconds}); + concurrentLocksManyMutexes(4, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressWithManyMutexesAlternatingEightThreads) { - concurrentLocksManyMutexes( - 8, std::chrono::seconds{FLAGS_stress_test_seconds}); + concurrentLocksManyMutexes(8, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressWithManyMutexesAlternatingSixteenThreads) { - concurrentLocksManyMutexes( - 16, std::chrono::seconds{FLAGS_stress_test_seconds}); + concurrentLocksManyMutexes(16, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressWithManyMutexesAlternatingThirtyTwoThreads) { - concurrentLocksManyMutexes( - 32, std::chrono::seconds{FLAGS_stress_test_seconds}); + concurrentLocksManyMutexes(32, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, StressWithManyMutexesAlternatingSixtyFourThreads) { - concurrentLocksManyMutexes( - 64, std::chrono::seconds{FLAGS_stress_test_seconds}); + concurrentLocksManyMutexes(64, std::chrono::seconds{kStressTestSeconds}); } namespace { @@ -1733,26 +1729,289 @@ void concurrentLocksManyMutexesDeterministic( TEST(DistributedMutex, DeterministicWithManyMutexesAlternatingTwoThreads) { concurrentLocksManyMutexesDeterministic( - 2, std::chrono::seconds{FLAGS_stress_test_seconds}); + 2, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, DeterministicWithManyMutexesAlternatingFourThreads) { concurrentLocksManyMutexesDeterministic( - 4, std::chrono::seconds{FLAGS_stress_test_seconds}); + 4, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, DeterministicWithManyMutexesAlternatingEightThreads) { concurrentLocksManyMutexesDeterministic( - 8, std::chrono::seconds{FLAGS_stress_test_seconds}); + 8, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, DeterministicWithManyMutexesAlternatingSixteenThreads) { concurrentLocksManyMutexesDeterministic( - 16, std::chrono::seconds{FLAGS_stress_test_seconds}); + 16, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, DeterministicWithManyMtxAlternatingThirtyTwoThreads) { concurrentLocksManyMutexesDeterministic( - 32, std::chrono::seconds{FLAGS_stress_test_seconds}); + 32, std::chrono::seconds{kStressTestSeconds}); } TEST(DistributedMutex, DeterministicWithManyMtxAlternatingSixtyFourThreads) { concurrentLocksManyMutexesDeterministic( - 64, std::chrono::seconds{FLAGS_stress_test_seconds}); + 64, std::chrono::seconds{kStressTestSeconds}); +} + +namespace { +class ExceptionWithConstructionTrack : public std::exception { + public: + explicit ExceptionWithConstructionTrack(int id) + : id_{folly::to<std::string>(id)}, constructionTrack_{id} {} + + const char* what() const noexcept override { + return id_.c_str(); + } + + private: + std::string id_; + TestConstruction constructionTrack_; +}; +} // namespace + +TEST(DistributedMutex, TestExceptionPropagationUncontended) { + TestConstruction::reset(); + auto&& mutex = folly::DistributedMutex{}; + + auto&& thread = std::thread{[&]() { + try { + mutex.lock_combine([&]() { throw ExceptionWithConstructionTrack{46}; }); + } catch (std::exception& exc) { + auto integer = folly::to<std::uint64_t>(exc.what()); + EXPECT_EQ(integer, 46); + EXPECT_GT(TestConstruction::defaultConstructs(), 0); + } + + EXPECT_EQ( + TestConstruction::defaultConstructs(), TestConstruction::destructs()); + }}; + + thread.join(); +} + +namespace { +template <template <typename> class Atom = std::atomic> +void concurrentExceptionPropagationStress( + int numThreads, + std::chrono::milliseconds t) { + TestConstruction::reset(); + auto&& mutex = detail::distributed_mutex::DistributedMutex<Atom>{}; + auto&& threads = std::vector<std::thread>{}; + auto&& stop = std::atomic<bool>{false}; + auto&& barrier = std::atomic<std::uint64_t>{0}; + + for (auto i = 0; i < numThreads; ++i) { + threads.push_back(DSched::thread([&]() { + for (auto j = 0; !stop.load(); ++j) { + auto value = int{0}; + try { + value = mutex.lock_combine([&]() { + EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 0); + EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 1); + std::this_thread::yield(); + SCOPE_EXIT { + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + }; + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 2); + + // we only throw an exception once every 3 times + if (!(j % 3)) { + throw ExceptionWithConstructionTrack{j}; + } + + return j; + }); + } catch (std::exception& exc) { + value = folly::to<int>(exc.what()); + } + + EXPECT_EQ(value, j); + } + })); + } + + /* sleep override */ + std::this_thread::sleep_for(t); + stop.store(true); + for (auto& thread : threads) { + DSched::join(thread); + } +} +} // namespace + +TEST(DistributedMutex, TestExceptionPropagationStressTwoThreads) { + concurrentExceptionPropagationStress( + 2, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, TestExceptionPropagationStressFourThreads) { + concurrentExceptionPropagationStress( + 4, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, TestExceptionPropagationStressEightThreads) { + concurrentExceptionPropagationStress( + 8, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, TestExceptionPropagationStressSixteenThreads) { + concurrentExceptionPropagationStress( + 16, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, TestExceptionPropagationStressThirtyTwoThreads) { + concurrentExceptionPropagationStress( + 32, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, TestExceptionPropagationStressSixtyFourThreads) { + concurrentExceptionPropagationStress( + 64, std::chrono::seconds{kStressTestSeconds}); +} + +namespace { +void concurrentExceptionPropagationDeterministic( + int threads, + std::chrono::seconds t) { + const auto kNumPasses = 3.0; + const auto seconds = std::ceil(static_cast<double>(t.count()) / kNumPasses); + const auto time = std::chrono::seconds{static_cast<std::uint64_t>(seconds)}; + + for (auto pass = 0; pass < kNumPasses; ++pass) { + auto&& schedule = DSched{DSched::uniform(pass)}; + concurrentExceptionPropagationStress<test::DeterministicAtomic>( + threads, time); + static_cast<void>(schedule); + } +} +} // namespace + +TEST(DistributedMutex, TestExceptionPropagationDeterministicTwoThreads) { + concurrentExceptionPropagationDeterministic( + 2, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, TestExceptionPropagationDeterministicFourThreads) { + concurrentExceptionPropagationDeterministic( + 4, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, TestExceptionPropagationDeterministicEightThreads) { + concurrentExceptionPropagationDeterministic( + 8, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, TestExceptionPropagationDeterministicSixteenThreads) { + concurrentExceptionPropagationDeterministic( + 16, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, TestExceptionPropagationDeterministicThirtyTwoThreads) { + concurrentExceptionPropagationDeterministic( + 32, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, TestExceptionPropagationDeterministicSixtyFourThreads) { + concurrentExceptionPropagationDeterministic( + 64, std::chrono::seconds{kStressTestSeconds}); +} + +namespace { +std::array<std::uint64_t, 8> makeMonotonicArray(int start) { + auto array = std::array<std::uint64_t, 8>{}; + folly::for_each(array, [&](auto& element) { element = start++; }); + return array; +} + +template <template <typename> class Atom = std::atomic> +void concurrentBigValueReturnStress( + int numThreads, + std::chrono::milliseconds t) { + auto&& mutex = detail::distributed_mutex::DistributedMutex<Atom>{}; + auto&& threads = std::vector<std::thread>{}; + auto&& stop = std::atomic<bool>{false}; + auto&& barrier = std::atomic<std::uint64_t>{0}; + + for (auto i = 0; i < numThreads; ++i) { + threads.push_back(DSched::thread([&]() { + auto&& value = std::atomic<std::uint64_t>{0}; + + for (auto j = 0; !stop.load(); ++j) { + auto returned = mutex.lock_combine([&]() { + EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 0); + EXPECT_EQ(barrier.fetch_add(1, std::memory_order_relaxed), 1); + std::this_thread::yield(); + // return an entire cacheline worth of data + auto current = value.fetch_add(1, std::memory_order_relaxed); + SCOPE_EXIT { + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 1); + }; + EXPECT_EQ(barrier.fetch_sub(1, std::memory_order_relaxed), 2); + return makeMonotonicArray(current); + }); + + auto expected = value.load() - 1; + folly::for_each( + returned, [&](auto& element) { EXPECT_EQ(element, expected++); }); + } + })); + } + + /* sleep override */ + std::this_thread::sleep_for(t); + stop.store(true); + for (auto& thread : threads) { + DSched::join(thread); + } +} +} // namespace + +TEST(DistributedMutex, StressBigValueReturnTwoThreads) { + concurrentBigValueReturnStress(2, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, StressBigValueReturnFourThreads) { + concurrentBigValueReturnStress(4, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, StressBigValueReturnEightThreads) { + concurrentBigValueReturnStress(8, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, StressBigValueReturnSixteenThreads) { + concurrentBigValueReturnStress(16, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, StressBigValueReturnThirtyTwoThreads) { + concurrentBigValueReturnStress(32, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, StressBigValueReturnSixtyFourThreads) { + concurrentBigValueReturnStress(64, std::chrono::seconds{kStressTestSeconds}); +} + +namespace { +void concurrentBigValueReturnDeterministic( + int threads, + std::chrono::seconds t) { + const auto kNumPasses = 3.0; + const auto seconds = std::ceil(static_cast<double>(t.count()) / kNumPasses); + const auto time = std::chrono::seconds{static_cast<std::uint64_t>(seconds)}; + + for (auto pass = 0; pass < kNumPasses; ++pass) { + auto&& schedule = DSched{DSched::uniform(pass)}; + concurrentBigValueReturnStress<test::DeterministicAtomic>(threads, time); + static_cast<void>(schedule); + } +} +} // namespace + +TEST(DistributedMutex, DeterministicBigValueReturnTwoThreads) { + concurrentBigValueReturnDeterministic( + 2, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, DeterministicBigValueReturnFourThreads) { + concurrentBigValueReturnDeterministic( + 4, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, DeterministicBigValueReturnEightThreads) { + concurrentBigValueReturnDeterministic( + 8, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, DeterministicBigValueReturnSixteenThreads) { + concurrentBigValueReturnDeterministic( + 16, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, DeterministicBigValueReturnThirtyTwoThreads) { + concurrentBigValueReturnDeterministic( + 32, std::chrono::seconds{kStressTestSeconds}); +} +TEST(DistributedMutex, DeterministicBigValueReturnSixtyFourThreads) { + concurrentBigValueReturnDeterministic( + 64, std::chrono::seconds{kStressTestSeconds}); } } // namespace folly diff --git a/folly/synchronization/test/SmallLocksBenchmark.cpp b/folly/synchronization/test/SmallLocksBenchmark.cpp index 0f31421cf..92dec49fe 100644 --- a/folly/synchronization/test/SmallLocksBenchmark.cpp +++ b/folly/synchronization/test/SmallLocksBenchmark.cpp @@ -167,13 +167,6 @@ auto lock_and(FlatCombiningMutexCaching& mutex, std::size_t i, F func) { return mutex.lock_combine(func, i); } -template <typename Mutex> -std::unique_lock<Mutex> lock(Mutex& mutex) { - return std::unique_lock<Mutex>{mutex}; -} -template <typename Mutex, typename Other> -void unlock(Mutex&, Other) {} - /** * Functions to initialize, write and read from data * @@ -285,10 +278,11 @@ runContended(size_t numOps, size_t numThreads, size_t work = FLAGS_work) { lockstruct* mutex = &locks[t % threadgroups]; runbarrier.wait(); for (size_t op = 0; op < numOps; op += 1) { - auto val = lock_and(mutex->mutex, t, [& value = mutex->value, work] { - burn(work); - return write(value); - }); + auto val = lock_and( + mutex->mutex, t, [& value = mutex->value, work ]() noexcept { + burn(work); + return write(value); + }); read(val); burn(FLAGS_unlocked_work); } @@ -344,7 +338,10 @@ static void runFairness(std::size_t numThreads) { while (!stop) { std::chrono::steady_clock::time_point prelock = std::chrono::steady_clock::now(); - auto state = lock(mutex->lock); + lock_and(mutex->lock, t, [&]() { + burn(FLAGS_work); + value++; + }); std::chrono::steady_clock::time_point postlock = std::chrono::steady_clock::now(); auto diff = std::chrono::duration_cast<std::chrono::microseconds>( @@ -354,10 +351,6 @@ static void runFairness(std::size_t numThreads) { if (diff > max) { max = diff; } - burn(FLAGS_work); - value++; - unlock(mutex->lock, std::move(state)); - burn(FLAGS_unlocked_work); } { std::lock_guard<std::mutex> g(rlock); @@ -410,9 +403,8 @@ void runUncontended(std::size_t iters) { auto&& mutex = Mutex{}; for (auto i = std::size_t{0}; i < iters; ++i) { folly::makeUnpredictable(mutex); - auto state = lock(mutex); + auto lck = std::unique_lock<Mutex>{mutex}; folly::makeUnpredictable(mutex); - unlock(mutex, std::move(state)); } } @@ -777,6 +769,8 @@ int main(int argc, char** argv) { fairnessTest<folly::SharedMutex>("folly::SharedMutex", numThreads); fairnessTest<folly::DistributedMutex>( "folly::DistributedMutex", numThreads); + fairnessTest<DistributedMutexFlatCombining>( + "folly::DistributedMutex (Combining)", numThreads); std::cout << std::string(76, '=') << std::endl; } @@ -792,349 +786,367 @@ int main(int argc, char** argv) { Intel(R) Xeon(R) CPU E5-2680 v4 @ 2.40GHz ------- std::mutex 2 threads -Sum: 107741003 Mean: 1923946 stddev: 99873 -Lock time stats in us: mean 1 stddev 40 max 53562 +Sum: 361854376 Mean: 6461685 stddev: 770837 +Lock time stats in us: mean 0 stddev 1 max 63002 ------- GoogleSpinLock 2 threads -Sum: 129434359 Mean: 2311327 stddev: 74053 -Lock time stats in us: mean 0 stddev 4 max 53102 +Sum: 463530598 Mean: 8277332 stddev: 759139 +Lock time stats in us: mean 0 stddev 9 max 44995 ------- folly::MicroSpinLock 2 threads -Sum: 225366606 Mean: 4024403 stddev: 1884122 -Lock time stats in us: mean 0 stddev 19 max 2278444 +Sum: 454928254 Mean: 8123718 stddev: 1568978 +Lock time stats in us: mean 0 stddev 9 max 118006 ------- folly::PicoSpinLock<std::uint16_t> 2 threads -Sum: 150216526 Mean: 2682437 stddev: 216045 -Lock time stats in us: mean 0 stddev 28 max 36826 +Sum: 376990850 Mean: 6731979 stddev: 1295859 +Lock time stats in us: mean 0 stddev 1 max 83007 ------- folly::MicroLock 2 threads -Sum: 132299209 Mean: 2362485 stddev: 496423 -Lock time stats in us: mean 0 stddev 32 max 68123 +Sum: 316081944 Mean: 5644320 stddev: 1249068 +Lock time stats in us: mean 0 stddev 13 max 53930 ------- folly::SharedMutex 2 threads -Sum: 132465497 Mean: 2365455 stddev: 556997 -Lock time stats in us: mean 0 stddev 32 max 24447 +Sum: 389298695 Mean: 6951762 stddev: 3031794 +Lock time stats in us: mean 0 stddev 2 max 55004 ------- folly::DistributedMutex 2 threads -Sum: 166667563 Mean: 2976206 stddev: 183292 -Lock time stats in us: mean 0 stddev 3 max 2834 +Sum: 512343772 Mean: 9148995 stddev: 1168346 +Lock time stats in us: mean 0 stddev 8 max 50830 +------- folly::DistributedMutex (Combining) 2 threads +Sum: 475079423 Mean: 8483561 stddev: 899288 +Lock time stats in us: mean 0 stddev 1 max 26006 ============================================================================ ------- std::mutex 4 threads -Sum: 56176633 Mean: 1003154 stddev: 20354 -Lock time stats in us: mean 2 stddev 76 max 10151 +Sum: 164126417 Mean: 2930828 stddev: 208327 +Lock time stats in us: mean 0 stddev 2 max 11759 ------- GoogleSpinLock 4 threads -Sum: 65060684 Mean: 1161797 stddev: 95631 -Lock time stats in us: mean 1 stddev 66 max 9624 +Sum: 200210044 Mean: 3575179 stddev: 472142 +Lock time stats in us: mean 0 stddev 21 max 16715 ------- folly::MicroSpinLock 4 threads -Sum: 124794912 Mean: 2228480 stddev: 752355 -Lock time stats in us: mean 1 stddev 2 max 1973546 +Sum: 168795789 Mean: 3014210 stddev: 825455 +Lock time stats in us: mean 0 stddev 3 max 152163 ------- folly::PicoSpinLock<std::uint16_t> 4 threads -Sum: 86858717 Mean: 1551048 stddev: 417050 -Lock time stats in us: mean 1 stddev 2 max 87873 +Sum: 125788231 Mean: 2246218 stddev: 755074 +Lock time stats in us: mean 1 stddev 3 max 151004 ------- folly::MicroLock 4 threads -Sum: 64529361 Mean: 1152310 stddev: 363331 -Lock time stats in us: mean 2 stddev 66 max 34196 +Sum: 109091138 Mean: 1948056 stddev: 465388 +Lock time stats in us: mean 1 stddev 39 max 60029 ------- folly::SharedMutex 4 threads -Sum: 64509031 Mean: 1151946 stddev: 551973 -Lock time stats in us: mean 2 stddev 5 max 58400 +Sum: 107870343 Mean: 1926256 stddev: 1039541 +Lock time stats in us: mean 1 stddev 2 max 57002 ------- folly::DistributedMutex 4 threads -Sum: 76778688 Mean: 1371048 stddev: 89767 -Lock time stats in us: mean 2 stddev 56 max 4038 +Sum: 207229191 Mean: 3700521 stddev: 182811 +Lock time stats in us: mean 0 stddev 21 max 16231 +------- folly::DistributedMutex (Combining) 4 threads +Sum: 204144735 Mean: 3645441 stddev: 619224 +Lock time stats in us: mean 0 stddev 0 max 27008 ============================================================================ ------- std::mutex 8 threads -Sum: 27905504 Mean: 498312 stddev: 12266 -Lock time stats in us: mean 4 stddev 154 max 10915 +Sum: 82709846 Mean: 1476961 stddev: 173483 +Lock time stats in us: mean 2 stddev 52 max 9404 ------- GoogleSpinLock 8 threads -Sum: 34900763 Mean: 623227 stddev: 34990 -Lock time stats in us: mean 3 stddev 4 max 11047 +Sum: 98373671 Mean: 1756672 stddev: 65326 +Lock time stats in us: mean 1 stddev 43 max 20805 ------- folly::MicroSpinLock 8 threads -Sum: 65703639 Mean: 1173279 stddev: 367466 -Lock time stats in us: mean 2 stddev 65 max 1985454 +Sum: 94805197 Mean: 1692949 stddev: 633249 +Lock time stats in us: mean 1 stddev 3 max 104517 ------- folly::PicoSpinLock<std::uint16_t> 8 threads -Sum: 46642042 Mean: 832893 stddev: 258465 -Lock time stats in us: mean 3 stddev 5 max 90012 +Sum: 41587796 Mean: 742639 stddev: 191868 +Lock time stats in us: mean 4 stddev 103 max 317025 ------- folly::MicroLock 8 threads -Sum: 28727093 Mean: 512983 stddev: 105746 -Lock time stats in us: mean 6 stddev 149 max 24648 +Sum: 42414128 Mean: 757395 stddev: 234934 +Lock time stats in us: mean 4 stddev 101 max 39660 ------- folly::SharedMutex 8 threads -Sum: 35789774 Mean: 639103 stddev: 420746 -Lock time stats in us: mean 5 stddev 120 max 95030 +Sum: 58861445 Mean: 1051097 stddev: 491231 +Lock time stats in us: mean 3 stddev 73 max 34007 ------- folly::DistributedMutex 8 threads -Sum: 33288752 Mean: 594442 stddev: 20581 -Lock time stats in us: mean 5 stddev 129 max 7018 +Sum: 93377108 Mean: 1667448 stddev: 113502 +Lock time stats in us: mean 1 stddev 46 max 11075 +------- folly::DistributedMutex (Combining) 8 threads +Sum: 131093487 Mean: 2340955 stddev: 187841 +Lock time stats in us: mean 1 stddev 3 max 25004 ============================================================================ ------- std::mutex 16 threads -Sum: 10886472 Mean: 194401 stddev: 9357 -Lock time stats in us: mean 12 stddev 394 max 13293 +Sum: 36606221 Mean: 653682 stddev: 65154 +Lock time stats in us: mean 5 stddev 117 max 13603 ------- GoogleSpinLock 16 threads -Sum: 13436731 Mean: 239941 stddev: 25068 -Lock time stats in us: mean 10 stddev 319 max 10127 +Sum: 29830088 Mean: 532680 stddev: 19614 +Lock time stats in us: mean 7 stddev 2 max 10338 ------- folly::MicroSpinLock 16 threads -Sum: 28766414 Mean: 513685 stddev: 109667 -Lock time stats in us: mean 7 stddev 149 max 453504 +Sum: 27935153 Mean: 498842 stddev: 197304 +Lock time stats in us: mean 7 stddev 3 max 257433 ------- folly::PicoSpinLock<std::uint16_t> 16 threads -Sum: 19795815 Mean: 353496 stddev: 110097 -Lock time stats in us: mean 10 stddev 217 max 164821 +Sum: 12265416 Mean: 219025 stddev: 146399 +Lock time stats in us: mean 17 stddev 350 max 471793 ------- folly::MicroLock 16 threads -Sum: 11380567 Mean: 203224 stddev: 25356 -Lock time stats in us: mean 15 stddev 377 max 13342 +Sum: 18180611 Mean: 324653 stddev: 32123 +Lock time stats in us: mean 11 stddev 236 max 40166 ------- folly::SharedMutex 16 threads -Sum: 13734684 Mean: 245262 stddev: 132500 -Lock time stats in us: mean 15 stddev 312 max 75465 +Sum: 21734734 Mean: 388120 stddev: 190252 +Lock time stats in us: mean 9 stddev 197 max 107045 ------- folly::DistributedMutex 16 threads -Sum: 13463633 Mean: 240422 stddev: 8070 -Lock time stats in us: mean 15 stddev 319 max 17020 +Sum: 42823745 Mean: 764709 stddev: 64251 +Lock time stats in us: mean 4 stddev 100 max 19986 +------- folly::DistributedMutex (Combining) 16 threads +Sum: 63515255 Mean: 1134200 stddev: 37905 +Lock time stats in us: mean 2 stddev 3 max 32005 ============================================================================ ------- std::mutex 32 threads -Sum: 3584545 Mean: 64009 stddev: 1099 -Lock time stats in us: mean 39 stddev 1197 max 12949 +Sum: 10307832 Mean: 184068 stddev: 2431 +Lock time stats in us: mean 21 stddev 416 max 18397 ------- GoogleSpinLock 32 threads -Sum: 4537642 Mean: 81029 stddev: 7258 -Lock time stats in us: mean 28 stddev 946 max 10736 +Sum: 10911809 Mean: 194853 stddev: 2968 +Lock time stats in us: mean 20 stddev 393 max 10765 ------- folly::MicroSpinLock 32 threads -Sum: 9493894 Mean: 169533 stddev: 42004 -Lock time stats in us: mean 23 stddev 452 max 934519 +Sum: 7318139 Mean: 130681 stddev: 24742 +Lock time stats in us: mean 29 stddev 586 max 230672 ------- folly::PicoSpinLock<std::uint16_t> 32 threads -Sum: 7159818 Mean: 127853 stddev: 20791 -Lock time stats in us: mean 30 stddev 599 max 116982 +Sum: 6424015 Mean: 114714 stddev: 138460 +Lock time stats in us: mean 34 stddev 668 max 879632 ------- folly::MicroLock 32 threads -Sum: 4052635 Mean: 72368 stddev: 10196 -Lock time stats in us: mean 38 stddev 1059 max 13123 +Sum: 4893744 Mean: 87388 stddev: 6935 +Lock time stats in us: mean 45 stddev 876 max 14902 ------- folly::SharedMutex 32 threads -Sum: 4207373 Mean: 75131 stddev: 36441 -Lock time stats in us: mean 51 stddev 1019 max 89781 +Sum: 6393363 Mean: 114167 stddev: 80211 +Lock time stats in us: mean 34 stddev 671 max 75777 ------- folly::DistributedMutex 32 threads -Sum: 4499483 Mean: 80347 stddev: 1684 -Lock time stats in us: mean 48 stddev 954 max 18793 +Sum: 14394775 Mean: 257049 stddev: 36723 +Lock time stats in us: mean 15 stddev 298 max 54654 +------- folly::DistributedMutex (Combining) 32 threads +Sum: 24232845 Mean: 432729 stddev: 11398 +Lock time stats in us: mean 8 stddev 177 max 35008 ============================================================================ ------- std::mutex 64 threads -Sum: 3584393 Mean: 56006 stddev: 989 -Lock time stats in us: mean 48 stddev 1197 max 12681 +Sum: 10656640 Mean: 166510 stddev: 3340 +Lock time stats in us: mean 23 stddev 402 max 10797 ------- GoogleSpinLock 64 threads -Sum: 4541415 Mean: 70959 stddev: 2042 -Lock time stats in us: mean 34 stddev 945 max 12997 +Sum: 11263029 Mean: 175984 stddev: 4669 +Lock time stats in us: mean 22 stddev 381 max 26844 ------- folly::MicroSpinLock 64 threads -Sum: 9464010 Mean: 147875 stddev: 43363 -Lock time stats in us: mean 26 stddev 453 max 464213 +Sum: 23284721 Mean: 363823 stddev: 62670 +Lock time stats in us: mean 10 stddev 184 max 168470 ------- folly::PicoSpinLock<std::uint16_t> 64 threads -Sum: 6915111 Mean: 108048 stddev: 15833 -Lock time stats in us: mean 36 stddev 620 max 162031 +Sum: 2322545 Mean: 36289 stddev: 6272 +Lock time stats in us: mean 109 stddev 1846 max 1157157 ------- folly::MicroLock 64 threads -Sum: 4008803 Mean: 62637 stddev: 6055 -Lock time stats in us: mean 46 stddev 1070 max 25289 +Sum: 4835136 Mean: 75549 stddev: 3484 +Lock time stats in us: mean 52 stddev 887 max 23895 ------- folly::SharedMutex 64 threads -Sum: 3580719 Mean: 55948 stddev: 23224 -Lock time stats in us: mean 68 stddev 1198 max 63328 +Sum: 7047147 Mean: 110111 stddev: 53207 +Lock time stats in us: mean 35 stddev 608 max 85181 ------- folly::DistributedMutex 64 threads -Sum: 4464065 Mean: 69751 stddev: 2299 -Lock time stats in us: mean 56 stddev 960 max 32873 +Sum: 14491662 Mean: 226432 stddev: 27098 +Lock time stats in us: mean 17 stddev 296 max 55078 +------- folly::DistributedMutex (Combining) 64 threads +Sum: 23885026 Mean: 373203 stddev: 14431 +Lock time stats in us: mean 10 stddev 179 max 62008 ============================================================================ ============================================================================ folly/synchronization/test/SmallLocksBenchmark.cpprelative time/iter iters/s ============================================================================ -StdMutexUncontendedBenchmark 16.40ns 60.98M -GoogleSpinUncontendedBenchmark 11.23ns 89.02M -MicroSpinLockUncontendedBenchmark 10.94ns 91.45M -PicoSpinLockUncontendedBenchmark 20.37ns 49.08M -MicroLockUncontendedBenchmark 29.21ns 34.24M -SharedMutexUncontendedBenchmark 19.44ns 51.45M -DistributedMutexUncontendedBenchmark 29.49ns 33.91M -AtomicFetchAddUncontendedBenchmark 5.45ns 183.56M +StdMutexUncontendedBenchmark 16.42ns 60.90M +GoogleSpinUncontendedBenchmark 11.25ns 88.86M +MicroSpinLockUncontendedBenchmark 10.95ns 91.33M +PicoSpinLockUncontendedBenchmark 20.38ns 49.07M +MicroLockUncontendedBenchmark 28.92ns 34.58M +SharedMutexUncontendedBenchmark 19.47ns 51.36M +DistributedMutexUncontendedBenchmark 28.89ns 34.62M +AtomicFetchAddUncontendedBenchmark 5.47ns 182.91M ---------------------------------------------------------------------------- ---------------------------------------------------------------------------- -std_mutex(1thread) 706.81ns 1.41M -google_spin(1thread) 103.09% 685.63ns 1.46M -folly_microspin(1thread) 117.03% 603.96ns 1.66M -folly_picospin(1thread) 102.72% 688.12ns 1.45M -folly_microlock(1thread) 103.40% 683.59ns 1.46M -folly_sharedmutex(1thread) 103.64% 682.01ns 1.47M -folly_distributedmutex(1thread) 101.07% 699.32ns 1.43M -folly_distributedmutex_combining(1thread) 102.75% 687.89ns 1.45M -folly_flatcombining_no_caching(1thread) 94.78% 745.77ns 1.34M -folly_flatcombining_caching(1thread) 100.95% 700.15ns 1.43M +std_mutex(1thread) 900.28ns 1.11M +google_spin(1thread) 94.91% 948.60ns 1.05M +folly_microspin(1thread) 109.53% 821.97ns 1.22M +folly_picospin(1thread) 101.86% 883.88ns 1.13M +folly_microlock(1thread) 102.54% 878.02ns 1.14M +folly_sharedmutex(1thread) 132.03% 681.86ns 1.47M +folly_distributedmutex(1thread) 129.50% 695.23ns 1.44M +folly_distributedmutex_combining(1thread) 130.73% 688.68ns 1.45M +folly_flatcombining_no_caching(1thread) 106.73% 843.49ns 1.19M +folly_flatcombining_caching(1thread) 125.22% 718.96ns 1.39M ---------------------------------------------------------------------------- -std_mutex(2thread) 1.28us 779.95K -google_spin(2thread) 137.96% 929.38ns 1.08M -folly_microspin(2thread) 151.64% 845.52ns 1.18M -folly_picospin(2thread) 140.81% 910.52ns 1.10M -folly_microlock(2thread) 131.62% 974.11ns 1.03M -folly_sharedmutex(2thread) 143.97% 890.53ns 1.12M -folly_distributedmutex(2thread) 129.20% 992.39ns 1.01M -folly_distributedmutex_combining(2thread) 131.27% 976.71ns 1.02M -folly_flatcombining_no_caching(2thread) 93.85% 1.37us 732.01K -folly_flatcombining_caching(2thread) 97.05% 1.32us 756.98K +std_mutex(2thread) 1.27us 784.90K +google_spin(2thread) 126.84% 1.00us 995.55K +folly_microspin(2thread) 147.93% 861.24ns 1.16M +folly_picospin(2thread) 146.10% 872.06ns 1.15M +folly_microlock(2thread) 131.35% 970.00ns 1.03M +folly_sharedmutex(2thread) 135.07% 943.23ns 1.06M +folly_distributedmutex(2thread) 135.88% 937.63ns 1.07M +folly_distributedmutex_combining(2thread) 130.37% 977.27ns 1.02M +folly_flatcombining_no_caching(2thread) 85.64% 1.49us 672.22K +folly_flatcombining_caching(2thread) 91.98% 1.39us 721.93K ---------------------------------------------------------------------------- -std_mutex(4thread) 2.65us 376.96K -google_spin(4thread) 125.03% 2.12us 471.33K -folly_microspin(4thread) 118.43% 2.24us 446.44K -folly_picospin(4thread) 122.04% 2.17us 460.05K -folly_microlock(4thread) 102.38% 2.59us 385.94K -folly_sharedmutex(4thread) 101.76% 2.61us 383.60K -folly_distributedmutex(4thread) 137.07% 1.94us 516.71K -folly_distributedmutex_combining(4thread) 191.98% 1.38us 723.71K -folly_flatcombining_no_caching(4thread) 106.91% 2.48us 403.02K -folly_flatcombining_caching(4thread) 111.66% 2.38us 420.91K +std_mutex(4thread) 2.40us 417.44K +google_spin(4thread) 111.99% 2.14us 467.49K +folly_microspin(4thread) 101.55% 2.36us 423.92K +folly_picospin(4thread) 97.89% 2.45us 408.64K +folly_microlock(4thread) 79.64% 3.01us 332.45K +folly_sharedmutex(4thread) 75.10% 3.19us 313.49K +folly_distributedmutex(4thread) 126.16% 1.90us 526.63K +folly_distributedmutex_combining(4thread) 166.56% 1.44us 695.28K +folly_flatcombining_no_caching(4thread) 91.79% 2.61us 383.17K +folly_flatcombining_caching(4thread) 103.95% 2.30us 433.95K ---------------------------------------------------------------------------- -std_mutex(8thread) 5.21us 191.97K -google_spin(8thread) 102.12% 5.10us 196.05K -folly_microspin(8thread) 97.02% 5.37us 186.26K -folly_picospin(8thread) 83.62% 6.23us 160.53K -folly_microlock(8thread) 69.32% 7.51us 133.08K -folly_sharedmutex(8thread) 64.22% 8.11us 123.29K -folly_distributedmutex(8thread) 175.50% 2.97us 336.91K -folly_distributedmutex_combining(8thread) 258.13% 2.02us 495.55K -folly_flatcombining_no_caching(8thread) 137.21% 3.80us 263.41K -folly_flatcombining_caching(8thread) 174.75% 2.98us 335.48K +std_mutex(8thread) 4.85us 206.37K +google_spin(8thread) 111.05% 4.36us 229.18K +folly_microspin(8thread) 105.28% 4.60us 217.28K +folly_picospin(8thread) 89.06% 5.44us 183.80K +folly_microlock(8thread) 73.95% 6.55us 152.62K +folly_sharedmutex(8thread) 67.17% 7.21us 138.62K +folly_distributedmutex(8thread) 162.16% 2.99us 334.66K +folly_distributedmutex_combining(8thread) 251.93% 1.92us 519.92K +folly_flatcombining_no_caching(8thread) 141.99% 3.41us 293.02K +folly_flatcombining_caching(8thread) 166.26% 2.91us 343.12K ---------------------------------------------------------------------------- -std_mutex(16thread) 10.06us 99.37K -google_spin(16thread) 97.24% 10.35us 96.63K -folly_microspin(16thread) 91.23% 11.03us 90.65K -folly_picospin(16thread) 58.31% 17.26us 57.94K -folly_microlock(16thread) 51.59% 19.51us 51.26K -folly_sharedmutex(16thread) 49.87% 20.18us 49.56K -folly_distributedmutex(16thread) 155.47% 6.47us 154.49K -folly_distributedmutex_combining(16thread) 316.70% 3.18us 314.70K -folly_flatcombining_no_caching(16thread) 198.94% 5.06us 197.68K -folly_flatcombining_caching(16thread) 184.72% 5.45us 183.55K +std_mutex(16thread) 11.36us 88.01K +google_spin(16thread) 99.95% 11.37us 87.96K +folly_microspin(16thread) 102.73% 11.06us 90.42K +folly_picospin(16thread) 44.00% 25.83us 38.72K +folly_microlock(16thread) 52.42% 21.67us 46.14K +folly_sharedmutex(16thread) 53.46% 21.26us 47.05K +folly_distributedmutex(16thread) 166.17% 6.84us 146.24K +folly_distributedmutex_combining(16thread) 352.82% 3.22us 310.52K +folly_flatcombining_no_caching(16thread) 218.07% 5.21us 191.92K +folly_flatcombining_caching(16thread) 217.69% 5.22us 191.58K ---------------------------------------------------------------------------- -std_mutex(32thread) 33.80us 29.59K -google_spin(32thread) 109.19% 30.95us 32.31K -folly_microspin(32thread) 110.23% 30.66us 32.62K -folly_picospin(32thread) 39.94% 84.62us 11.82K -folly_microlock(32thread) 56.56% 59.75us 16.74K -folly_sharedmutex(32thread) 73.92% 45.72us 21.87K -folly_distributedmutex(32thread) 192.60% 17.55us 56.99K -folly_distributedmutex_combining(32thread) 402.79% 8.39us 119.19K -folly_flatcombining_no_caching(32thread) 235.30% 14.36us 69.63K -folly_flatcombining_caching(32thread) 259.02% 13.05us 76.64K +std_mutex(32thread) 32.12us 31.13K +google_spin(32thread) 115.23% 27.88us 35.87K +folly_microspin(32thread) 104.52% 30.74us 32.54K +folly_picospin(32thread) 32.81% 97.91us 10.21K +folly_microlock(32thread) 57.40% 55.96us 17.87K +folly_sharedmutex(32thread) 63.68% 50.45us 19.82K +folly_distributedmutex(32thread) 180.17% 17.83us 56.09K +folly_distributedmutex_combining(32thread) 394.34% 8.15us 122.76K +folly_flatcombining_no_caching(32thread) 216.41% 14.84us 67.37K +folly_flatcombining_caching(32thread) 261.99% 12.26us 81.56K ---------------------------------------------------------------------------- -std_mutex(64thread) 38.86us 25.73K -google_spin(64thread) 109.06% 35.63us 28.06K -folly_microspin(64thread) 109.92% 35.36us 28.28K -folly_picospin(64thread) 37.02% 104.99us 9.53K -folly_microlock(64thread) 56.33% 68.99us 14.49K -folly_sharedmutex(64thread) 69.39% 56.00us 17.86K -folly_distributedmutex(64thread) 194.31% 20.00us 50.00K -folly_distributedmutex_combining(64thread) 397.54% 9.78us 102.29K -folly_flatcombining_no_caching(64thread) 230.64% 16.85us 59.35K -folly_flatcombining_caching(64thread) 254.03% 15.30us 65.37K +std_mutex(64thread) 36.76us 27.20K +google_spin(64thread) 115.38% 31.86us 31.39K +folly_microspin(64thread) 112.14% 32.78us 30.51K +folly_picospin(64thread) 32.34% 113.65us 8.80K +folly_microlock(64thread) 57.21% 64.26us 15.56K +folly_sharedmutex(64thread) 60.93% 60.33us 16.57K +folly_distributedmutex(64thread) 179.79% 20.45us 48.91K +folly_distributedmutex_combining(64thread) 392.64% 9.36us 106.81K +folly_flatcombining_no_caching(64thread) 211.85% 17.35us 57.63K +folly_flatcombining_caching(64thread) 241.45% 15.22us 65.68K ---------------------------------------------------------------------------- -std_mutex(128thread) 76.62us 13.05K -google_spin(128thread) 109.31% 70.09us 14.27K -folly_microspin(128thread) 102.86% 74.49us 13.43K -folly_picospin(128thread) 42.23% 181.42us 5.51K -folly_microlock(128thread) 55.01% 139.29us 7.18K -folly_sharedmutex(128thread) 63.50% 120.65us 8.29K -folly_distributedmutex(128thread) 183.63% 41.72us 23.97K -folly_distributedmutex_combining(128thread) 388.41% 19.73us 50.69K -folly_flatcombining_no_caching(128thread) 183.56% 41.74us 23.96K -folly_flatcombining_caching(128thread) 198.02% 38.69us 25.84K +std_mutex(128thread) 73.05us 13.69K +google_spin(128thread) 116.19% 62.87us 15.91K +folly_microspin(128thread) 97.45% 74.96us 13.34K +folly_picospin(128thread) 31.46% 232.19us 4.31K +folly_microlock(128thread) 56.50% 129.29us 7.73K +folly_sharedmutex(128thread) 59.54% 122.69us 8.15K +folly_distributedmutex(128thread) 166.59% 43.85us 22.80K +folly_distributedmutex_combining(128thread) 379.86% 19.23us 52.00K +folly_flatcombining_no_caching(128thread) 179.10% 40.79us 24.52K +folly_flatcombining_caching(128thread) 189.64% 38.52us 25.96K ---------------------------------------------------------------------------- -std_mutex_simple(1thread) 634.77ns 1.58M -google_spin_simple(1thread) 104.06% 610.01ns 1.64M -folly_microspin_simple(1thread) 104.59% 606.89ns 1.65M -folly_picospin_simple(1thread) 99.37% 638.81ns 1.57M -folly_microlock_simple(1thread) 104.08% 609.86ns 1.64M -folly_sharedmutex_simple(1thread) 91.77% 691.73ns 1.45M -folly_distributedmutex_simple(1thread) 98.10% 647.04ns 1.55M -folly_distributedmutex_combining_simple(1thread 101.90% 622.93ns 1.61M -folly_flatcombining_no_caching_simple(1thread) 93.71% 677.40ns 1.48M -folly_flatcombining_caching_simple(1thread) 101.81% 623.46ns 1.60M -atomics_fetch_add(1thread) 102.23% 620.90ns 1.61M -atomic_fetch_xor(1thread) 104.67% 606.43ns 1.65M -atomic_cas(1thread) 84.68% 749.58ns 1.33M +std_mutex_simple(1thread) 666.33ns 1.50M +google_spin_simple(1thread) 110.03% 605.58ns 1.65M +folly_microspin_simple(1thread) 109.80% 606.87ns 1.65M +folly_picospin_simple(1thread) 108.89% 611.94ns 1.63M +folly_microlock_simple(1thread) 108.42% 614.59ns 1.63M +folly_sharedmutex_simple(1thread) 93.00% 716.47ns 1.40M +folly_distributedmutex_simple(1thread) 90.08% 739.68ns 1.35M +folly_distributedmutex_combining_simple(1thread 90.20% 738.73ns 1.35M +folly_flatcombining_no_caching_simple(1thread) 98.04% 679.68ns 1.47M +folly_flatcombining_caching_simple(1thread) 105.59% 631.04ns 1.58M +atomics_fetch_add(1thread) 108.30% 615.29ns 1.63M +atomic_fetch_xor(1thread) 110.52% 602.90ns 1.66M +atomic_cas(1thread) 109.86% 606.52ns 1.65M ---------------------------------------------------------------------------- -std_mutex_simple(2thread) 1.24us 803.81K -google_spin_simple(2thread) 123.09% 1.01us 989.38K -folly_microspin_simple(2thread) 138.46% 898.48ns 1.11M -folly_picospin_simple(2thread) 121.05% 1.03us 973.01K -folly_microlock_simple(2thread) 112.54% 1.11us 904.60K -folly_sharedmutex_simple(2thread) 112.16% 1.11us 901.60K -folly_distributedmutex_simple(2thread) 119.86% 1.04us 963.47K -folly_distributedmutex_combining_simple(2thread 130.78% 951.25ns 1.05M -folly_flatcombining_no_caching_simple(2thread) 93.25% 1.33us 749.54K -folly_flatcombining_caching_simple(2thread) 102.34% 1.22us 822.65K -atomics_fetch_add(2thread) 113.81% 1.09us 914.83K -atomic_fetch_xor(2thread) 161.97% 768.09ns 1.30M -atomic_cas(2thread) 150.00% 829.41ns 1.21M +std_mutex_simple(2thread) 1.19us 841.25K +google_spin_simple(2thread) 107.33% 1.11us 902.89K +folly_microspin_simple(2thread) 130.73% 909.27ns 1.10M +folly_picospin_simple(2thread) 112.39% 1.06us 945.48K +folly_microlock_simple(2thread) 113.89% 1.04us 958.14K +folly_sharedmutex_simple(2thread) 119.48% 994.86ns 1.01M +folly_distributedmutex_simple(2thread) 112.44% 1.06us 945.91K +folly_distributedmutex_combining_simple(2thread 123.12% 965.48ns 1.04M +folly_flatcombining_no_caching_simple(2thread) 90.56% 1.31us 761.82K +folly_flatcombining_caching_simple(2thread) 100.66% 1.18us 846.83K +atomics_fetch_add(2thread) 119.15% 997.67ns 1.00M +atomic_fetch_xor(2thread) 179.85% 660.93ns 1.51M +atomic_cas(2thread) 179.40% 662.58ns 1.51M ---------------------------------------------------------------------------- -std_mutex_simple(4thread) 2.39us 418.75K -google_spin_simple(4thread) 109.55% 2.18us 458.74K -folly_microspin_simple(4thread) 110.15% 2.17us 461.26K -folly_picospin_simple(4thread) 115.62% 2.07us 484.17K -folly_microlock_simple(4thread) 88.54% 2.70us 370.77K -folly_sharedmutex_simple(4thread) 100.50% 2.38us 420.86K -folly_distributedmutex_simple(4thread) 114.93% 2.08us 481.26K -folly_distributedmutex_combining_simple(4thread 161.11% 1.48us 674.64K -folly_flatcombining_no_caching_simple(4thread) 106.27% 2.25us 445.02K -folly_flatcombining_caching_simple(4thread) 113.01% 2.11us 473.23K -atomics_fetch_add(4thread) 156.29% 1.53us 654.48K -atomic_fetch_xor(4thread) 285.69% 835.89ns 1.20M -atomic_cas(4thread) 270.31% 883.45ns 1.13M +std_mutex_simple(4thread) 2.37us 422.81K +google_spin_simple(4thread) 106.35% 2.22us 449.64K +folly_microspin_simple(4thread) 110.42% 2.14us 466.89K +folly_picospin_simple(4thread) 111.77% 2.12us 472.58K +folly_microlock_simple(4thread) 82.17% 2.88us 347.44K +folly_sharedmutex_simple(4thread) 93.40% 2.53us 394.89K +folly_distributedmutex_simple(4thread) 121.00% 1.95us 511.58K +folly_distributedmutex_combining_simple(4thread 187.65% 1.26us 793.42K +folly_flatcombining_no_caching_simple(4thread) 104.81% 2.26us 443.13K +folly_flatcombining_caching_simple(4thread) 112.90% 2.09us 477.34K +atomics_fetch_add(4thread) 178.61% 1.32us 755.20K +atomic_fetch_xor(4thread) 323.62% 730.84ns 1.37M +atomic_cas(4thread) 300.43% 787.23ns 1.27M ---------------------------------------------------------------------------- -std_mutex_simple(8thread) 4.83us 207.09K -google_spin_simple(8thread) 117.15% 4.12us 242.60K -folly_microspin_simple(8thread) 106.41% 4.54us 220.37K -folly_picospin_simple(8thread) 88.31% 5.47us 182.88K -folly_microlock_simple(8thread) 77.90% 6.20us 161.33K -folly_sharedmutex_simple(8thread) 72.21% 6.69us 149.55K -folly_distributedmutex_simple(8thread) 138.98% 3.47us 287.83K -folly_distributedmutex_combining_simple(8thread 289.79% 1.67us 600.12K -folly_flatcombining_no_caching_simple(8thread) 134.25% 3.60us 278.03K -folly_flatcombining_caching_simple(8thread) 149.74% 3.22us 310.10K -atomics_fetch_add(8thread) 318.11% 1.52us 658.78K -atomic_fetch_xor(8thread) 373.98% 1.29us 774.47K -atomic_cas(8thread) 241.00% 2.00us 499.09K +std_mutex_simple(8thread) 5.02us 199.09K +google_spin_simple(8thread) 108.93% 4.61us 216.88K +folly_microspin_simple(8thread) 116.44% 4.31us 231.82K +folly_picospin_simple(8thread) 80.84% 6.21us 160.94K +folly_microlock_simple(8thread) 77.18% 6.51us 153.66K +folly_sharedmutex_simple(8thread) 76.09% 6.60us 151.48K +folly_distributedmutex_simple(8thread) 145.27% 3.46us 289.21K +folly_distributedmutex_combining_simple(8thread 310.65% 1.62us 618.48K +folly_flatcombining_no_caching_simple(8thread) 139.83% 3.59us 278.39K +folly_flatcombining_caching_simple(8thread) 163.72% 3.07us 325.95K +atomics_fetch_add(8thread) 337.67% 1.49us 672.28K +atomic_fetch_xor(8thread) 380.66% 1.32us 757.87K +atomic_cas(8thread) 238.04% 2.11us 473.93K ---------------------------------------------------------------------------- -std_mutex_simple(16thread) 12.03us 83.13K -google_spin_simple(16thread) 98.34% 12.23us 81.75K -folly_microspin_simple(16thread) 115.19% 10.44us 95.76K -folly_picospin_simple(16thread) 54.50% 22.07us 45.31K -folly_microlock_simple(16thread) 58.38% 20.61us 48.53K -folly_sharedmutex_simple(16thread) 69.90% 17.21us 58.11K -folly_distributedmutex_simple(16thread) 155.15% 7.75us 128.97K -folly_distributedmutex_combining_simple(16threa 463.66% 2.59us 385.43K -folly_flatcombining_no_caching_simple(16thread) 279.15% 4.31us 232.05K -folly_flatcombining_caching_simple(16thread) 207.72% 5.79us 172.67K -atomics_fetch_add(16thread) 538.64% 2.23us 447.76K -atomic_fetch_xor(16thread) 570.85% 2.11us 474.53K -atomic_cas(16thread) 334.73% 3.59us 278.25K +std_mutex_simple(16thread) 12.26us 81.59K +google_spin_simple(16thread) 104.37% 11.74us 85.16K +folly_microspin_simple(16thread) 116.32% 10.54us 94.91K +folly_picospin_simple(16thread) 53.67% 22.83us 43.79K +folly_microlock_simple(16thread) 66.39% 18.46us 54.17K +folly_sharedmutex_simple(16thread) 65.00% 18.85us 53.04K +folly_distributedmutex_simple(16thread) 171.32% 7.15us 139.79K +folly_distributedmutex_combining_simple(16threa 445.11% 2.75us 363.17K +folly_flatcombining_no_caching_simple(16thread) 206.11% 5.95us 168.17K +folly_flatcombining_caching_simple(16thread) 245.09% 5.00us 199.97K +atomics_fetch_add(16thread) 494.82% 2.48us 403.73K +atomic_fetch_xor(16thread) 489.90% 2.50us 399.72K +atomic_cas(16thread) 232.76% 5.27us 189.91K ---------------------------------------------------------------------------- -std_mutex_simple(32thread) 30.92us 32.34K -google_spin_simple(32thread) 107.22% 28.84us 34.68K -folly_microspin_simple(32thread) 106.48% 29.04us 34.44K -folly_picospin_simple(32thread) 32.90% 93.97us 10.64K -folly_microlock_simple(32thread) 55.77% 55.44us 18.04K -folly_sharedmutex_simple(32thread) 63.85% 48.42us 20.65K -folly_distributedmutex_simple(32thread) 170.50% 18.13us 55.14K -folly_distributedmutex_combining_simple(32threa 562.55% 5.50us 181.94K -folly_flatcombining_no_caching_simple(32thread) 296.57% 10.43us 95.92K -folly_flatcombining_caching_simple(32thread) 295.25% 10.47us 95.49K -atomics_fetch_add(32thread) 952.20% 3.25us 307.96K -atomic_fetch_xor(32thread) 818.15% 3.78us 264.61K -atomic_cas(32thread) 634.91% 4.87us 205.34K +std_mutex_simple(32thread) 30.28us 33.03K +google_spin_simple(32thread) 106.34% 28.47us 35.12K +folly_microspin_simple(32thread) 102.20% 29.62us 33.76K +folly_picospin_simple(32thread) 31.56% 95.92us 10.43K +folly_microlock_simple(32thread) 53.99% 56.07us 17.83K +folly_sharedmutex_simple(32thread) 67.49% 44.86us 22.29K +folly_distributedmutex_simple(32thread) 161.63% 18.73us 53.38K +folly_distributedmutex_combining_simple(32threa 605.26% 5.00us 199.92K +folly_flatcombining_no_caching_simple(32thread) 234.62% 12.90us 77.49K +folly_flatcombining_caching_simple(32thread) 332.21% 9.11us 109.73K +atomics_fetch_add(32thread) 909.18% 3.33us 300.30K +atomic_fetch_xor(32thread) 779.56% 3.88us 257.49K +atomic_cas(32thread) 622.19% 4.87us 205.51K ---------------------------------------------------------------------------- -std_mutex_simple(64thread) 35.29us 28.33K -google_spin_simple(64thread) 107.33% 32.88us 30.41K -folly_microspin_simple(64thread) 106.02% 33.29us 30.04K -folly_picospin_simple(64thread) 32.93% 107.17us 9.33K -folly_microlock_simple(64thread) 54.76% 64.45us 15.52K -folly_sharedmutex_simple(64thread) 63.74% 55.37us 18.06K -folly_distributedmutex_simple(64thread) 170.45% 20.71us 48.30K -folly_distributedmutex_combining_simple(64threa 558.99% 6.31us 158.38K -folly_flatcombining_no_caching_simple(64thread) 311.86% 11.32us 88.36K -folly_flatcombining_caching_simple(64thread) 327.64% 10.77us 92.83K -atomics_fetch_add(64thread) 858.61% 4.11us 243.28K -atomic_fetch_xor(64thread) 738.35% 4.78us 209.20K -atomic_cas(64thread) 623.72% 5.66us 176.72K +std_mutex_simple(64thread) 34.33us 29.13K +google_spin_simple(64thread) 106.28% 32.30us 30.96K +folly_microspin_simple(64thread) 99.86% 34.37us 29.09K +folly_picospin_simple(64thread) 31.37% 109.42us 9.14K +folly_microlock_simple(64thread) 53.46% 64.21us 15.57K +folly_sharedmutex_simple(64thread) 62.94% 54.54us 18.33K +folly_distributedmutex_simple(64thread) 161.26% 21.29us 46.98K +folly_distributedmutex_combining_simple(64threa 603.87% 5.68us 175.91K +folly_flatcombining_no_caching_simple(64thread) 247.00% 13.90us 71.95K +folly_flatcombining_caching_simple(64thread) 310.66% 11.05us 90.50K +atomics_fetch_add(64thread) 839.49% 4.09us 244.55K +atomic_fetch_xor(64thread) 756.48% 4.54us 220.37K +atomic_cas(64thread) 606.85% 5.66us 176.78K ---------------------------------------------------------------------------- -std_mutex_simple(128thread) 69.21us 14.45K -google_spin_simple(128thread) 107.42% 64.43us 15.52K -folly_microspin_simple(128thread) 96.36% 71.82us 13.92K -folly_picospin_simple(128thread) 31.07% 222.75us 4.49K -folly_microlock_simple(128thread) 53.97% 128.25us 7.80K -folly_sharedmutex_simple(128thread) 60.56% 114.29us 8.75K -folly_distributedmutex_simple(128thread) 165.16% 41.91us 23.86K -folly_distributedmutex_combining_simple(128thre 542.63% 12.75us 78.40K -folly_flatcombining_no_caching_simple(128thread 246.16% 28.12us 35.57K -folly_flatcombining_caching_simple(128thread) 232.56% 29.76us 33.60K -atomics_fetch_add(128thread) 839.43% 8.24us 121.29K -atomic_fetch_xor(128thread) 761.39% 9.09us 110.01K -atomic_cas(128thread) 598.53% 11.56us 86.48K +std_mutex_simple(128thread) 67.35us 14.85K +google_spin_simple(128thread) 106.30% 63.36us 15.78K +folly_microspin_simple(128thread) 92.58% 72.75us 13.75K +folly_picospin_simple(128thread) 29.87% 225.47us 4.44K +folly_microlock_simple(128thread) 52.52% 128.25us 7.80K +folly_sharedmutex_simple(128thread) 59.79% 112.64us 8.88K +folly_distributedmutex_simple(128thread) 151.27% 44.52us 22.46K +folly_distributedmutex_combining_simple(128thre 580.11% 11.61us 86.13K +folly_flatcombining_no_caching_simple(128thread 219.20% 30.73us 32.55K +folly_flatcombining_caching_simple(128thread) 225.39% 29.88us 33.46K +atomics_fetch_add(128thread) 813.36% 8.28us 120.76K +atomic_fetch_xor(128thread) 740.02% 9.10us 109.88K +atomic_cas(128thread) 586.66% 11.48us 87.11K ============================================================================ ./small_locks_benchmark --bm_min_iters=100000 @@ -1153,204 +1165,204 @@ DistributedMutexUncontendedBenchmark 51.47ns 19.43M AtomicFetchAddUncontendedBenchmark 10.67ns 93.73M ---------------------------------------------------------------------------- ---------------------------------------------------------------------------- -std_mutex(1thread) 1.36us 737.48K -google_spin(1thread) 94.81% 1.43us 699.17K -folly_microspin(1thread) 100.17% 1.35us 738.74K -folly_picospin(1thread) 100.40% 1.35us 740.41K -folly_microlock(1thread) 82.90% 1.64us 611.34K -folly_sharedmutex(1thread) 101.07% 1.34us 745.36K -folly_distributedmutex(1thread) 101.50% 1.34us 748.54K -folly_distributedmutex_combining(1thread) 99.09% 1.37us 730.79K -folly_flatcombining_no_caching(1thread) 91.37% 1.48us 673.80K -folly_flatcombining_caching(1thread) 99.19% 1.37us 731.48K +std_mutex(1thread) 1.37us 730.43K +google_spin(1thread) 104.25% 1.31us 761.46K +folly_microspin(1thread) 102.06% 1.34us 745.45K +folly_picospin(1thread) 100.68% 1.36us 735.43K +folly_microlock(1thread) 104.27% 1.31us 761.64K +folly_sharedmutex(1thread) 101.95% 1.34us 744.65K +folly_distributedmutex(1thread) 98.63% 1.39us 720.41K +folly_distributedmutex_combining(1thread) 103.78% 1.32us 758.05K +folly_flatcombining_no_caching(1thread) 95.44% 1.43us 697.15K +folly_flatcombining_caching(1thread) 99.11% 1.38us 723.94K ---------------------------------------------------------------------------- -std_mutex(2thread) 1.65us 605.33K -google_spin(2thread) 113.28% 1.46us 685.74K -folly_microspin(2thread) 117.23% 1.41us 709.63K -folly_picospin(2thread) 113.56% 1.45us 687.40K -folly_microlock(2thread) 106.92% 1.55us 647.22K -folly_sharedmutex(2thread) 107.24% 1.54us 649.15K -folly_distributedmutex(2thread) 114.89% 1.44us 695.47K -folly_distributedmutex_combining(2thread) 83.44% 1.98us 505.10K -folly_flatcombining_no_caching(2thread) 75.89% 2.18us 459.42K -folly_flatcombining_caching(2thread) 76.96% 2.15us 465.86K +std_mutex(2thread) 1.65us 605.36K +google_spin(2thread) 111.27% 1.48us 673.61K +folly_microspin(2thread) 119.82% 1.38us 725.35K +folly_picospin(2thread) 112.46% 1.47us 680.81K +folly_microlock(2thread) 106.47% 1.55us 644.54K +folly_sharedmutex(2thread) 107.12% 1.54us 648.45K +folly_distributedmutex(2thread) 110.80% 1.49us 670.76K +folly_distributedmutex_combining(2thread) 97.09% 1.70us 587.77K +folly_flatcombining_no_caching(2thread) 83.37% 1.98us 504.68K +folly_flatcombining_caching(2thread) 108.62% 1.52us 657.54K ---------------------------------------------------------------------------- -std_mutex(4thread) 2.88us 347.43K -google_spin(4thread) 132.08% 2.18us 458.88K -folly_microspin(4thread) 160.15% 1.80us 556.43K -folly_picospin(4thread) 189.27% 1.52us 657.60K -folly_microlock(4thread) 155.13% 1.86us 538.97K -folly_sharedmutex(4thread) 148.96% 1.93us 517.55K -folly_distributedmutex(4thread) 106.64% 2.70us 370.51K -folly_distributedmutex_combining(4thread) 138.83% 2.07us 482.33K -folly_flatcombining_no_caching(4thread) 87.67% 3.28us 304.59K -folly_flatcombining_caching(4thread) 93.32% 3.08us 324.23K +std_mutex(4thread) 2.92us 341.96K +google_spin(4thread) 137.67% 2.12us 470.78K +folly_microspin(4thread) 165.47% 1.77us 565.85K +folly_picospin(4thread) 181.92% 1.61us 622.09K +folly_microlock(4thread) 149.83% 1.95us 512.35K +folly_sharedmutex(4thread) 158.69% 1.84us 542.66K +folly_distributedmutex(4thread) 107.42% 2.72us 367.35K +folly_distributedmutex_combining(4thread) 144.34% 2.03us 493.59K +folly_flatcombining_no_caching(4thread) 88.43% 3.31us 302.40K +folly_flatcombining_caching(4thread) 94.20% 3.10us 322.11K ---------------------------------------------------------------------------- -std_mutex(8thread) 7.01us 142.65K -google_spin(8thread) 127.58% 5.49us 182.00K -folly_microspin(8thread) 137.50% 5.10us 196.14K -folly_picospin(8thread) 114.66% 6.11us 163.56K -folly_microlock(8thread) 107.90% 6.50us 153.92K -folly_sharedmutex(8thread) 114.21% 6.14us 162.93K -folly_distributedmutex(8thread) 129.43% 5.42us 184.63K -folly_distributedmutex_combining(8thread) 271.46% 2.58us 387.23K -folly_flatcombining_no_caching(8thread) 148.27% 4.73us 211.50K -folly_flatcombining_caching(8thread) 170.26% 4.12us 242.88K +std_mutex(8thread) 7.04us 142.02K +google_spin(8thread) 128.50% 5.48us 182.49K +folly_microspin(8thread) 134.72% 5.23us 191.32K +folly_picospin(8thread) 112.37% 6.27us 159.58K +folly_microlock(8thread) 109.65% 6.42us 155.71K +folly_sharedmutex(8thread) 105.92% 6.65us 150.42K +folly_distributedmutex(8thread) 127.22% 5.53us 180.67K +folly_distributedmutex_combining(8thread) 275.50% 2.56us 391.26K +folly_flatcombining_no_caching(8thread) 144.99% 4.86us 205.92K +folly_flatcombining_caching(8thread) 156.31% 4.50us 221.99K ---------------------------------------------------------------------------- -std_mutex(16thread) 13.11us 76.30K -google_spin(16thread) 122.81% 10.67us 93.71K -folly_microspin(16thread) 91.61% 14.31us 69.90K -folly_picospin(16thread) 62.60% 20.94us 47.76K -folly_microlock(16thread) 73.44% 17.85us 56.04K -folly_sharedmutex(16thread) 74.68% 17.55us 56.98K -folly_distributedmutex(16thread) 142.42% 9.20us 108.67K -folly_distributedmutex_combining(16thread) 332.10% 3.95us 253.39K -folly_flatcombining_no_caching(16thread) 177.20% 7.40us 135.21K -folly_flatcombining_caching(16thread) 186.60% 7.02us 142.37K +std_mutex(16thread) 13.08us 76.44K +google_spin(16thread) 121.76% 10.74us 93.07K +folly_microspin(16thread) 91.47% 14.30us 69.92K +folly_picospin(16thread) 67.95% 19.25us 51.94K +folly_microlock(16thread) 73.57% 17.78us 56.24K +folly_sharedmutex(16thread) 70.59% 18.53us 53.96K +folly_distributedmutex(16thread) 139.74% 9.36us 106.82K +folly_distributedmutex_combining(16thread) 338.38% 3.87us 258.67K +folly_flatcombining_no_caching(16thread) 194.08% 6.74us 148.36K +folly_flatcombining_caching(16thread) 195.03% 6.71us 149.09K ---------------------------------------------------------------------------- -std_mutex(32thread) 25.45us 39.30K -google_spin(32thread) 122.57% 20.76us 48.17K -folly_microspin(32thread) 73.58% 34.58us 28.92K -folly_picospin(32thread) 50.29% 50.60us 19.76K -folly_microlock(32thread) 58.33% 43.63us 22.92K -folly_sharedmutex(32thread) 55.89% 45.53us 21.96K -folly_distributedmutex(32thread) 142.80% 17.82us 56.12K -folly_distributedmutex_combining(32thread) 352.23% 7.22us 138.42K -folly_flatcombining_no_caching(32thread) 237.42% 10.72us 93.30K -folly_flatcombining_caching(32thread) 251.05% 10.14us 98.66K +std_mutex(32thread) 25.35us 39.45K +google_spin(32thread) 122.73% 20.66us 48.41K +folly_microspin(32thread) 73.81% 34.35us 29.11K +folly_picospin(32thread) 50.66% 50.04us 19.98K +folly_microlock(32thread) 58.40% 43.41us 23.03K +folly_sharedmutex(32thread) 55.14% 45.98us 21.75K +folly_distributedmutex(32thread) 141.36% 17.93us 55.76K +folly_distributedmutex_combining(32thread) 358.52% 7.07us 141.42K +folly_flatcombining_no_caching(32thread) 257.78% 9.83us 101.68K +folly_flatcombining_caching(32thread) 285.82% 8.87us 112.74K ---------------------------------------------------------------------------- -std_mutex(64thread) 43.02us 23.25K -google_spin(64thread) 120.68% 35.65us 28.05K -folly_microspin(64thread) 70.09% 61.38us 16.29K -folly_picospin(64thread) 42.05% 102.31us 9.77K -folly_microlock(64thread) 54.50% 78.94us 12.67K -folly_sharedmutex(64thread) 50.37% 85.40us 11.71K -folly_distributedmutex(64thread) 135.17% 31.83us 31.42K -folly_distributedmutex_combining(64thread) 319.01% 13.49us 74.15K -folly_flatcombining_no_caching(64thread) 218.18% 19.72us 50.72K -folly_flatcombining_caching(64thread) 211.05% 20.38us 49.06K +std_mutex(64thread) 45.03us 22.21K +google_spin(64thread) 124.58% 36.15us 27.66K +folly_microspin(64thread) 75.05% 60.00us 16.67K +folly_picospin(64thread) 44.98% 100.12us 9.99K +folly_microlock(64thread) 56.99% 79.01us 12.66K +folly_sharedmutex(64thread) 52.67% 85.49us 11.70K +folly_distributedmutex(64thread) 139.71% 32.23us 31.02K +folly_distributedmutex_combining(64thread) 343.76% 13.10us 76.34K +folly_flatcombining_no_caching(64thread) 211.67% 21.27us 47.01K +folly_flatcombining_caching(64thread) 222.51% 20.24us 49.41K ---------------------------------------------------------------------------- -std_mutex(128thread) 84.62us 11.82K -google_spin(128thread) 120.25% 70.37us 14.21K -folly_microspin(128thread) 66.54% 127.16us 7.86K -folly_picospin(128thread) 33.40% 253.38us 3.95K -folly_microlock(128thread) 51.91% 163.03us 6.13K -folly_sharedmutex(128thread) 49.51% 170.90us 5.85K -folly_distributedmutex(128thread) 131.90% 64.15us 15.59K -folly_distributedmutex_combining(128thread) 273.55% 30.93us 32.33K -folly_flatcombining_no_caching(128thread) 183.86% 46.02us 21.73K -folly_flatcombining_caching(128thread) 180.95% 46.76us 21.38K +std_mutex(128thread) 88.78us 11.26K +google_spin(128thread) 125.10% 70.96us 14.09K +folly_microspin(128thread) 71.00% 125.03us 8.00K +folly_picospin(128thread) 30.97% 286.63us 3.49K +folly_microlock(128thread) 54.37% 163.28us 6.12K +folly_sharedmutex(128thread) 51.69% 171.76us 5.82K +folly_distributedmutex(128thread) 137.37% 64.63us 15.47K +folly_distributedmutex_combining(128thread) 281.23% 31.57us 31.68K +folly_flatcombining_no_caching(128thread) 136.61% 64.99us 15.39K +folly_flatcombining_caching(128thread) 152.32% 58.29us 17.16K ---------------------------------------------------------------------------- -std_mutex_simple(1thread) 1.19us 839.60K -google_spin_simple(1thread) 100.96% 1.18us 847.68K -folly_microspin_simple(1thread) 101.35% 1.18us 850.96K -folly_picospin_simple(1thread) 101.04% 1.18us 848.31K -folly_microlock_simple(1thread) 100.58% 1.18us 844.50K -folly_sharedmutex_simple(1thread) 100.75% 1.18us 845.88K -folly_distributedmutex_simple(1thread) 98.62% 1.21us 828.05K -folly_distributedmutex_combining_simple(1thread 99.58% 1.20us 836.07K -folly_flatcombining_no_caching_simple(1thread) 95.63% 1.25us 802.87K -folly_flatcombining_caching_simple(1thread) 99.37% 1.20us 834.27K -atomics_fetch_add(1thread) 101.98% 1.17us 856.25K -atomic_fetch_xor(1thread) 101.29% 1.18us 850.43K -atomic_cas(1thread) 101.73% 1.17us 854.11K +std_mutex_simple(1thread) 1.63us 611.75K +google_spin_simple(1thread) 105.70% 1.55us 646.61K +folly_microspin_simple(1thread) 103.24% 1.58us 631.57K +folly_picospin_simple(1thread) 109.17% 1.50us 667.87K +folly_microlock_simple(1thread) 111.22% 1.47us 680.41K +folly_sharedmutex_simple(1thread) 136.79% 1.19us 836.83K +folly_distributedmutex_simple(1thread) 107.21% 1.52us 655.88K +folly_distributedmutex_combining_simple(1thread 134.79% 1.21us 824.61K +folly_flatcombining_no_caching_simple(1thread) 127.99% 1.28us 782.99K +folly_flatcombining_caching_simple(1thread) 133.87% 1.22us 818.93K +atomics_fetch_add(1thread) 138.24% 1.18us 845.70K +atomic_fetch_xor(1thread) 106.94% 1.53us 654.23K +atomic_cas(1thread) 124.81% 1.31us 763.52K ---------------------------------------------------------------------------- -std_mutex_simple(2thread) 1.60us 623.66K -google_spin_simple(2thread) 113.06% 1.42us 705.12K -folly_microspin_simple(2thread) 114.38% 1.40us 713.32K -folly_picospin_simple(2thread) 112.84% 1.42us 703.74K -folly_microlock_simple(2thread) 97.27% 1.65us 606.66K -folly_sharedmutex_simple(2thread) 111.31% 1.44us 694.20K -folly_distributedmutex_simple(2thread) 109.21% 1.47us 681.11K -folly_distributedmutex_combining_simple(2thread 107.91% 1.49us 672.98K -folly_flatcombining_no_caching_simple(2thread) 89.48% 1.79us 558.04K -folly_flatcombining_caching_simple(2thread) 98.95% 1.62us 617.14K -atomics_fetch_add(2thread) 106.88% 1.50us 666.58K -atomic_fetch_xor(2thread) 126.82% 1.26us 790.91K -atomic_cas(2thread) 130.34% 1.23us 812.86K +std_mutex_simple(2thread) 1.60us 626.60K +google_spin_simple(2thread) 96.04% 1.66us 601.80K +folly_microspin_simple(2thread) 111.88% 1.43us 701.02K +folly_picospin_simple(2thread) 106.11% 1.50us 664.91K +folly_microlock_simple(2thread) 88.90% 1.80us 557.04K +folly_sharedmutex_simple(2thread) 90.93% 1.76us 569.79K +folly_distributedmutex_simple(2thread) 93.93% 1.70us 588.57K +folly_distributedmutex_combining_simple(2thread 106.86% 1.49us 669.61K +folly_flatcombining_no_caching_simple(2thread) 85.92% 1.86us 538.37K +folly_flatcombining_caching_simple(2thread) 98.82% 1.61us 619.24K +atomics_fetch_add(2thread) 104.61% 1.53us 655.46K +atomic_fetch_xor(2thread) 126.46% 1.26us 792.40K +atomic_cas(2thread) 125.92% 1.27us 788.99K ---------------------------------------------------------------------------- -std_mutex_simple(4thread) 2.74us 364.72K -google_spin_simple(4thread) 123.43% 2.22us 450.16K -folly_microspin_simple(4thread) 153.56% 1.79us 560.07K -folly_picospin_simple(4thread) 146.03% 1.88us 532.59K -folly_microlock_simple(4thread) 116.28% 2.36us 424.10K -folly_sharedmutex_simple(4thread) 142.39% 1.93us 519.33K -folly_distributedmutex_simple(4thread) 111.84% 2.45us 407.89K -folly_distributedmutex_combining_simple(4thread 140.61% 1.95us 512.83K -folly_flatcombining_no_caching_simple(4thread) 101.22% 2.71us 369.17K -folly_flatcombining_caching_simple(4thread) 105.38% 2.60us 384.35K -atomics_fetch_add(4thread) 150.95% 1.82us 550.52K -atomic_fetch_xor(4thread) 223.43% 1.23us 814.87K -atomic_cas(4thread) 217.57% 1.26us 793.52K +std_mutex_simple(4thread) 2.71us 368.45K +google_spin_simple(4thread) 124.52% 2.18us 458.79K +folly_microspin_simple(4thread) 146.48% 1.85us 539.69K +folly_picospin_simple(4thread) 163.54% 1.66us 602.57K +folly_microlock_simple(4thread) 113.17% 2.40us 416.99K +folly_sharedmutex_simple(4thread) 142.36% 1.91us 524.52K +folly_distributedmutex_simple(4thread) 108.22% 2.51us 398.74K +folly_distributedmutex_combining_simple(4thread 141.49% 1.92us 521.30K +folly_flatcombining_no_caching_simple(4thread) 97.27% 2.79us 358.38K +folly_flatcombining_caching_simple(4thread) 106.12% 2.56us 390.99K +atomics_fetch_add(4thread) 151.10% 1.80us 556.73K +atomic_fetch_xor(4thread) 213.14% 1.27us 785.32K +atomic_cas(4thread) 218.93% 1.24us 806.65K ---------------------------------------------------------------------------- -std_mutex_simple(8thread) 6.99us 142.98K -google_spin_simple(8thread) 128.58% 5.44us 183.84K -folly_microspin_simple(8thread) 131.98% 5.30us 188.69K -folly_picospin_simple(8thread) 121.81% 5.74us 174.16K -folly_microlock_simple(8thread) 100.06% 6.99us 143.06K -folly_sharedmutex_simple(8thread) 115.88% 6.04us 165.69K -folly_distributedmutex_simple(8thread) 123.11% 5.68us 176.02K -folly_distributedmutex_combining_simple(8thread 307.74% 2.27us 439.99K -folly_flatcombining_no_caching_simple(8thread) 136.00% 5.14us 194.45K -folly_flatcombining_caching_simple(8thread) 148.43% 4.71us 212.22K -atomics_fetch_add(8thread) 358.67% 1.95us 512.81K -atomic_fetch_xor(8thread) 466.73% 1.50us 667.32K -atomic_cas(8thread) 371.61% 1.88us 531.31K +std_mutex_simple(8thread) 7.02us 142.50K +google_spin_simple(8thread) 127.47% 5.51us 181.64K +folly_microspin_simple(8thread) 137.77% 5.09us 196.33K +folly_picospin_simple(8thread) 119.78% 5.86us 170.69K +folly_microlock_simple(8thread) 108.08% 6.49us 154.02K +folly_sharedmutex_simple(8thread) 114.77% 6.11us 163.55K +folly_distributedmutex_simple(8thread) 120.24% 5.84us 171.35K +folly_distributedmutex_combining_simple(8thread 316.54% 2.22us 451.07K +folly_flatcombining_no_caching_simple(8thread) 136.43% 5.14us 194.42K +folly_flatcombining_caching_simple(8thread) 145.04% 4.84us 206.68K +atomics_fetch_add(8thread) 358.98% 1.95us 511.55K +atomic_fetch_xor(8thread) 505.27% 1.39us 720.02K +atomic_cas(8thread) 389.32% 1.80us 554.79K ---------------------------------------------------------------------------- -std_mutex_simple(16thread) 12.83us 77.96K -google_spin_simple(16thread) 122.19% 10.50us 95.26K -folly_microspin_simple(16thread) 99.14% 12.94us 77.30K -folly_picospin_simple(16thread) 62.74% 20.44us 48.91K -folly_microlock_simple(16thread) 75.01% 17.10us 58.48K -folly_sharedmutex_simple(16thread) 79.92% 16.05us 62.31K -folly_distributedmutex_simple(16thread) 118.18% 10.85us 92.14K -folly_distributedmutex_combining_simple(16threa 482.27% 2.66us 376.00K -folly_flatcombining_no_caching_simple(16thread) 191.45% 6.70us 149.26K -folly_flatcombining_caching_simple(16thread) 227.12% 5.65us 177.07K -atomics_fetch_add(16thread) 612.80% 2.09us 477.77K -atomic_fetch_xor(16thread) 551.00% 2.33us 429.58K -atomic_cas(16thread) 282.79% 4.54us 220.47K +std_mutex_simple(16thread) 12.78us 78.24K +google_spin_simple(16thread) 122.66% 10.42us 95.96K +folly_microspin_simple(16thread) 98.10% 13.03us 76.75K +folly_picospin_simple(16thread) 72.52% 17.62us 56.74K +folly_microlock_simple(16thread) 70.12% 18.23us 54.86K +folly_sharedmutex_simple(16thread) 76.81% 16.64us 60.09K +folly_distributedmutex_simple(16thread) 113.84% 11.23us 89.06K +folly_distributedmutex_combining_simple(16threa 498.99% 2.56us 390.39K +folly_flatcombining_no_caching_simple(16thread) 193.05% 6.62us 151.04K +folly_flatcombining_caching_simple(16thread) 220.47% 5.80us 172.49K +atomics_fetch_add(16thread) 611.70% 2.09us 478.58K +atomic_fetch_xor(16thread) 515.51% 2.48us 403.32K +atomic_cas(16thread) 239.86% 5.33us 187.66K ---------------------------------------------------------------------------- -std_mutex_simple(32thread) 23.09us 43.30K -google_spin_simple(32thread) 125.07% 18.46us 54.16K -folly_microspin_simple(32thread) 76.39% 30.23us 33.08K -folly_picospin_simple(32thread) 46.54% 49.62us 20.16K -folly_microlock_simple(32thread) 52.84% 43.71us 22.88K -folly_sharedmutex_simple(32thread) 53.06% 43.52us 22.98K -folly_distributedmutex_simple(32thread) 107.10% 21.56us 46.38K -folly_distributedmutex_combining_simple(32threa 596.57% 3.87us 258.33K -folly_flatcombining_no_caching_simple(32thread) 274.44% 8.41us 118.84K -folly_flatcombining_caching_simple(32thread) 312.83% 7.38us 135.46K -atomics_fetch_add(32thread) 1082.13% 2.13us 468.59K -atomic_fetch_xor(32thread) 552.82% 4.18us 239.39K -atomic_cas(32thread) 203.03% 11.37us 87.92K +std_mutex_simple(32thread) 23.80us 42.02K +google_spin_simple(32thread) 125.41% 18.98us 52.69K +folly_microspin_simple(32thread) 76.32% 31.18us 32.07K +folly_picospin_simple(32thread) 48.82% 48.75us 20.51K +folly_microlock_simple(32thread) 52.99% 44.92us 22.26K +folly_sharedmutex_simple(32thread) 54.03% 44.05us 22.70K +folly_distributedmutex_simple(32thread) 108.28% 21.98us 45.49K +folly_distributedmutex_combining_simple(32threa 697.71% 3.41us 293.15K +folly_flatcombining_no_caching_simple(32thread) 291.70% 8.16us 122.56K +folly_flatcombining_caching_simple(32thread) 412.51% 5.77us 173.32K +atomics_fetch_add(32thread) 1074.64% 2.21us 451.52K +atomic_fetch_xor(32thread) 577.90% 4.12us 242.81K +atomic_cas(32thread) 193.87% 12.28us 81.46K ---------------------------------------------------------------------------- -std_mutex_simple(64thread) 39.95us 25.03K -google_spin_simple(64thread) 124.75% 32.02us 31.23K -folly_microspin_simple(64thread) 73.49% 54.36us 18.40K -folly_picospin_simple(64thread) 39.80% 100.37us 9.96K -folly_microlock_simple(64thread) 50.07% 79.78us 12.53K -folly_sharedmutex_simple(64thread) 49.52% 80.66us 12.40K -folly_distributedmutex_simple(64thread) 104.56% 38.20us 26.18K -folly_distributedmutex_combining_simple(64threa 532.34% 7.50us 133.26K -folly_flatcombining_no_caching_simple(64thread) 279.23% 14.31us 69.90K -folly_flatcombining_caching_simple(64thread) 325.10% 12.29us 81.39K -atomics_fetch_add(64thread) 1031.51% 3.87us 258.23K -atomic_fetch_xor(64thread) 525.68% 7.60us 131.60K -atomic_cas(64thread) 187.67% 21.28us 46.98K +std_mutex_simple(64thread) 41.40us 24.16K +google_spin_simple(64thread) 125.42% 33.01us 30.30K +folly_microspin_simple(64thread) 75.30% 54.98us 18.19K +folly_picospin_simple(64thread) 42.87% 96.57us 10.35K +folly_microlock_simple(64thread) 50.88% 81.37us 12.29K +folly_sharedmutex_simple(64thread) 50.08% 82.67us 12.10K +folly_distributedmutex_simple(64thread) 105.81% 39.12us 25.56K +folly_distributedmutex_combining_simple(64threa 604.86% 6.84us 146.11K +folly_flatcombining_no_caching_simple(64thread) 269.82% 15.34us 65.18K +folly_flatcombining_caching_simple(64thread) 334.78% 12.37us 80.87K +atomics_fetch_add(64thread) 1061.21% 3.90us 256.34K +atomic_fetch_xor(64thread) 551.00% 7.51us 133.10K +atomic_cas(64thread) 183.75% 22.53us 44.39K ---------------------------------------------------------------------------- -std_mutex_simple(128thread) 78.65us 12.71K -google_spin_simple(128thread) 124.05% 63.40us 15.77K -folly_microspin_simple(128thread) 70.00% 112.36us 8.90K -folly_picospin_simple(128thread) 29.72% 264.60us 3.78K -folly_microlock_simple(128thread) 47.74% 164.73us 6.07K -folly_sharedmutex_simple(128thread) 48.87% 160.93us 6.21K -folly_distributedmutex_simple(128thread) 104.04% 75.59us 13.23K -folly_distributedmutex_combining_simple(128thre 426.02% 18.46us 54.17K -folly_flatcombining_no_caching_simple(128thread 210.85% 37.30us 26.81K -folly_flatcombining_caching_simple(128thread) 241.48% 32.57us 30.70K -atomics_fetch_add(128thread) 992.30% 7.93us 126.17K -atomic_fetch_xor(128thread) 525.32% 14.97us 66.79K -atomic_cas(128thread) 181.89% 43.24us 23.13K +std_mutex_simple(128thread) 80.97us 12.35K +google_spin_simple(128thread) 124.75% 64.90us 15.41K +folly_microspin_simple(128thread) 70.93% 114.16us 8.76K +folly_picospin_simple(128thread) 32.81% 246.78us 4.05K +folly_microlock_simple(128thread) 48.00% 168.69us 5.93K +folly_sharedmutex_simple(128thread) 49.03% 165.15us 6.06K +folly_distributedmutex_simple(128thread) 103.96% 77.88us 12.84K +folly_distributedmutex_combining_simple(128thre 460.68% 17.58us 56.90K +folly_flatcombining_no_caching_simple(128thread 211.10% 38.35us 26.07K +folly_flatcombining_caching_simple(128thread) 220.02% 36.80us 27.17K +atomics_fetch_add(128thread) 1031.88% 7.85us 127.45K +atomic_fetch_xor(128thread) 543.67% 14.89us 67.15K +atomic_cas(128thread) 179.37% 45.14us 22.15K ============================================================================ */ -- 2.26.2