Commit 8b260639 authored by Nathan Bronson's avatar Nathan Bronson Committed by Sara Golemon

Multi-producer multi-consumer queue with optional blocking

Summary:
MPMCQueue<T> is a high-performance bounded concurrent queue that
supports multiple producers, multiple consumers, and optional blocking.
The queue has a fixed capacity, for which all memory will be allocated
up front.  The bulk of the work of enqueuing and dequeuing can be
performed in parallel.

To make an MPMCQueue<T>, T must satisfy either of two conditions:
- it has been tagged FOLLY_ASSUME_FBVECTOR_COMPATIBLE; or
- both the constructor used during enqueue and the move operator are
marked noexcept.

This diff extracts the generic component from tao/queues/ConcurrentQueue
and renames identifiers to match those of existing folly queues.
It also includes an extraction of Futex, which wraps the futex syscall,
and DeterministicScheduler, which allows for deterministic exploration
of thread interleavings for components built from std::atomic and Futex.

Test Plan: new unit tests

Reviewed By: tudorb@fb.com

FB internal diff: D866566
parent 78f3142f
This diff is collapsed.
/*
* Copyright 2013 Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include <atomic>
#include <limits>
#include <assert.h>
#include <errno.h>
#include <linux/futex.h>
#include <sys/syscall.h>
#include <unistd.h>
#include <boost/noncopyable.hpp>
namespace folly { namespace detail {
/**
* Futex is an atomic 32 bit unsigned integer that provides access to the
* futex() syscall on that value. It is templated in such a way that it
* can interact properly with DeterministicSchedule testing.
*
* If you don't know how to use futex(), you probably shouldn't be using
* this class. Even if you do know how, you should have a good reason
* (and benchmarks to back you up).
*/
template <template <typename> class Atom = std::atomic>
struct Futex : Atom<uint32_t>, boost::noncopyable {
explicit Futex(uint32_t init = 0) : Atom<uint32_t>(init) {}
/** Puts the thread to sleep if this->load() == expected. Returns true when
* it is returning because it has consumed a wake() event, false for any
* other return (signal, this->load() != expected, or spurious wakeup). */
bool futexWait(uint32_t expected, uint32_t waitMask = -1);
/** Wakens up to count waiters where (waitMask & wakeMask) != 0,
* returning the number of awoken threads. */
int futexWake(int count = std::numeric_limits<int>::max(),
uint32_t wakeMask = -1);
};
template <>
inline bool Futex<std::atomic>::futexWait(uint32_t expected,
uint32_t waitMask) {
assert(sizeof(*this) == sizeof(int));
int rv = syscall(SYS_futex,
this, /* addr1 */
FUTEX_WAIT_BITSET | FUTEX_PRIVATE_FLAG, /* op */
expected, /* val */
nullptr, /* timeout */
nullptr, /* addr2 */
waitMask); /* val3 */
assert(rv == 0 || (errno == EWOULDBLOCK || errno == EINTR));
return rv == 0;
}
template <>
inline int Futex<std::atomic>::futexWake(int count, uint32_t wakeMask) {
assert(sizeof(*this) == sizeof(int));
int rv = syscall(SYS_futex,
this, /* addr1 */
FUTEX_WAKE_BITSET | FUTEX_PRIVATE_FLAG, /* op */
count, /* val */
nullptr, /* timeout */
nullptr, /* addr2 */
wakeMask); /* val3 */
assert(rv >= 0);
return rv;
}
}}
/*
* Copyright 2013 Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "DeterministicSchedule.h"
#include <algorithm>
#include <list>
#include <mutex>
#include <random>
#include <utility>
#include <unordered_map>
#include <assert.h>
namespace folly { namespace test {
__thread sem_t* DeterministicSchedule::tls_sem;
__thread DeterministicSchedule* DeterministicSchedule::tls_sched;
// access is protected by futexLock
static std::unordered_map<detail::Futex<DeterministicAtomic>*,
std::list<std::pair<uint32_t,bool*>>> futexQueues;
static std::mutex futexLock;
DeterministicSchedule::DeterministicSchedule(
const std::function<int(int)>& scheduler)
: scheduler_(scheduler)
{
assert(tls_sem == nullptr);
assert(tls_sched == nullptr);
tls_sem = new sem_t;
sem_init(tls_sem, 0, 1);
sems_.push_back(tls_sem);
tls_sched = this;
}
DeterministicSchedule::~DeterministicSchedule() {
assert(tls_sched == this);
assert(sems_.size() == 1);
assert(sems_[0] == tls_sem);
beforeThreadExit();
}
std::function<int(int)>
DeterministicSchedule::uniform(long seed) {
auto rand = std::make_shared<std::ranlux48>(seed);
return [rand](int numActive) {
auto dist = std::uniform_int_distribution<int>(0, numActive - 1);
return dist(*rand);
};
}
struct UniformSubset {
UniformSubset(long seed, int subsetSize, int stepsBetweenSelect)
: uniform_(DeterministicSchedule::uniform(seed))
, subsetSize_(subsetSize)
, stepsBetweenSelect_(stepsBetweenSelect)
, stepsLeft_(0)
{
}
int operator()(int numActive) {
adjustPermSize(numActive);
if (stepsLeft_-- == 0) {
stepsLeft_ = stepsBetweenSelect_ - 1;
shufflePrefix();
}
return perm_[uniform_(std::min(numActive, subsetSize_))];
}
private:
std::function<int(int)> uniform_;
const int subsetSize_;
const int stepsBetweenSelect_;
int stepsLeft_;
// only the first subsetSize_ is properly randomized
std::vector<int> perm_;
void adjustPermSize(int numActive) {
if (perm_.size() > numActive) {
perm_.erase(std::remove_if(perm_.begin(), perm_.end(),
[=](int x){ return x >= numActive; }), perm_.end());
} else {
while (perm_.size() < numActive) {
perm_.push_back(perm_.size());
}
}
assert(perm_.size() == numActive);
}
void shufflePrefix() {
for (int i = 0; i < std::min(int(perm_.size() - 1), subsetSize_); ++i) {
int j = uniform_(perm_.size() - i) + i;
std::swap(perm_[i], perm_[j]);
}
}
};
std::function<int(int)>
DeterministicSchedule::uniformSubset(long seed, int n, int m) {
auto gen = std::make_shared<UniformSubset>(seed, n, m);
return [=](int numActive) { return (*gen)(numActive); };
}
void
DeterministicSchedule::beforeSharedAccess() {
if (tls_sem) {
sem_wait(tls_sem);
}
}
void
DeterministicSchedule::afterSharedAccess() {
auto sched = tls_sched;
if (!sched) {
return;
}
sem_post(sched->sems_[sched->scheduler_(sched->sems_.size())]);
}
sem_t*
DeterministicSchedule::beforeThreadCreate() {
sem_t* s = new sem_t;
sem_init(s, 0, 0);
beforeSharedAccess();
sems_.push_back(s);
afterSharedAccess();
return s;
}
void
DeterministicSchedule::afterThreadCreate(sem_t* sem) {
assert(tls_sem == nullptr);
assert(tls_sched == nullptr);
tls_sem = sem;
tls_sched = this;
bool started = false;
while (!started) {
beforeSharedAccess();
if (active_.count(std::this_thread::get_id()) == 1) {
started = true;
}
afterSharedAccess();
}
}
void
DeterministicSchedule::beforeThreadExit() {
assert(tls_sched == this);
beforeSharedAccess();
sems_.erase(std::find(sems_.begin(), sems_.end(), tls_sem));
active_.erase(std::this_thread::get_id());
if (sems_.size() > 0) {
afterSharedAccess();
}
sem_destroy(tls_sem);
delete tls_sem;
tls_sem = nullptr;
tls_sched = nullptr;
}
void
DeterministicSchedule::join(std::thread& child) {
auto sched = tls_sched;
if (sched) {
bool done = false;
while (!done) {
beforeSharedAccess();
done = !sched->active_.count(child.get_id());
afterSharedAccess();
}
}
child.join();
}
void
DeterministicSchedule::post(sem_t* sem) {
beforeSharedAccess();
sem_post(sem);
afterSharedAccess();
}
bool
DeterministicSchedule::tryWait(sem_t* sem) {
beforeSharedAccess();
int rv = sem_trywait(sem);
afterSharedAccess();
if (rv == 0) {
return true;
} else {
assert(errno == EAGAIN);
return false;
}
}
void
DeterministicSchedule::wait(sem_t* sem) {
while (!tryWait(sem)) {
// we're not busy waiting because this is a deterministic schedule
}
}
}}
namespace folly { namespace detail {
using namespace test;
template<>
bool Futex<DeterministicAtomic>::futexWait(uint32_t expected,
uint32_t waitMask) {
bool rv;
DeterministicSchedule::beforeSharedAccess();
futexLock.lock();
if (data != expected) {
rv = false;
} else {
auto& queue = futexQueues[this];
bool done = false;
queue.push_back(std::make_pair(waitMask, &done));
while (!done) {
futexLock.unlock();
DeterministicSchedule::afterSharedAccess();
DeterministicSchedule::beforeSharedAccess();
futexLock.lock();
}
rv = true;
}
futexLock.unlock();
DeterministicSchedule::afterSharedAccess();
return rv;
}
template<>
int Futex<DeterministicAtomic>::futexWake(int count, uint32_t wakeMask) {
int rv = 0;
DeterministicSchedule::beforeSharedAccess();
futexLock.lock();
if (futexQueues.count(this) > 0) {
auto& queue = futexQueues[this];
auto iter = queue.begin();
while (iter != queue.end() && rv < count) {
auto cur = iter++;
if ((cur->first & wakeMask) != 0) {
*(cur->second) = true;
rv++;
queue.erase(cur);
}
}
if (queue.empty()) {
futexQueues.erase(this);
}
}
futexLock.unlock();
DeterministicSchedule::afterSharedAccess();
return rv;
}
}}
/*
* Copyright 2013 Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include <atomic>
#include <functional>
#include <thread>
#include <unordered_set>
#include <vector>
#include <boost/noncopyable.hpp>
#include <semaphore.h>
#include <errno.h>
#include <assert.h>
#include <folly/ScopeGuard.h>
#include <folly/detail/Futex.h>
namespace folly { namespace test {
/**
* DeterministicSchedule coordinates the inter-thread communication of a
* set of threads under test, so that despite concurrency the execution is
* the same every time. It works by stashing a reference to the schedule
* in a thread-local variable, then blocking all but one thread at a time.
*
* In order for DeterministicSchedule to work, it needs to intercept
* all inter-thread communication. To do this you should use
* DeterministicAtomic<T> instead of std::atomic<T>, create threads
* using DeterministicSchedule::thread() instead of the std::thread
* constructor, DeterministicSchedule::join(thr) instead of thr.join(),
* and access semaphores via the helper functions in DeterministicSchedule.
* Locks are not yet supported, although they would be easy to add with
* the same strategy as the mapping of sem_wait.
*
* The actual schedule is defined by a function from n -> [0,n). At
* each step, the function will be given the number of active threads
* (n), and it returns the index of the thread that should be run next.
* Invocations of the scheduler function will be serialized, but will
* occur from multiple threads. A good starting schedule is uniform(0).
*/
class DeterministicSchedule : boost::noncopyable {
public:
/**
* Arranges for the current thread (and all threads created by
* DeterministicSchedule::thread on a thread participating in this
* schedule) to participate in a deterministic schedule.
*/
explicit DeterministicSchedule(const std::function<int(int)>& scheduler);
/** Completes the schedule. */
~DeterministicSchedule();
/**
* Returns a scheduling function that randomly chooses one of the
* runnable threads at each step, with no history. This implements
* a schedule that is equivalent to one in which the steps between
* inter-thread communication are random variables following a poisson
* distribution.
*/
static std::function<int(int)> uniform(long seed);
/**
* Returns a scheduling function that chooses a subset of the active
* threads and randomly chooses a member of the subset as the next
* runnable thread. The subset is chosen with size n, and the choice
* is made every m steps.
*/
static std::function<int(int)> uniformSubset(long seed, int n = 2,
int m = 64);
/** Obtains permission for the current thread to perform inter-thread
* communication. */
static void beforeSharedAccess();
/** Releases permission for the current thread to perform inter-thread
* communication. */
static void afterSharedAccess();
/** Launches a thread that will participate in the same deterministic
* schedule as the current thread. */
template <typename Func, typename... Args>
static inline std::thread thread(Func&& func, Args&&... args) {
// TODO: maybe future versions of gcc will allow forwarding to thread
auto sched = tls_sched;
auto sem = sched ? sched->beforeThreadCreate() : nullptr;
auto child = std::thread([=](Args... a) {
if (sched) sched->afterThreadCreate(sem);
SCOPE_EXIT { if (sched) sched->beforeThreadExit(); };
func(a...);
}, args...);
if (sched) {
beforeSharedAccess();
sched->active_.insert(child.get_id());
afterSharedAccess();
}
return child;
}
/** Calls child.join() as part of a deterministic schedule. */
static void join(std::thread& child);
/** Calls sem_post(sem) as part of a deterministic schedule. */
static void post(sem_t* sem);
/** Calls sem_trywait(sem) as part of a deterministic schedule, returning
* true on success and false on transient failure. */
static bool tryWait(sem_t* sem);
/** Calls sem_wait(sem) as part of a deterministic schedule. */
static void wait(sem_t* sem);
private:
static __thread sem_t* tls_sem;
static __thread DeterministicSchedule* tls_sched;
std::function<int(int)> scheduler_;
std::vector<sem_t*> sems_;
std::unordered_set<std::thread::id> active_;
sem_t* beforeThreadCreate();
void afterThreadCreate(sem_t*);
void beforeThreadExit();
};
/**
* DeterministicAtomic<T> is a drop-in replacement std::atomic<T> that
* cooperates with DeterministicSchedule.
*/
template <typename T>
struct DeterministicAtomic {
std::atomic<T> data;
DeterministicAtomic() = default;
~DeterministicAtomic() = default;
DeterministicAtomic(DeterministicAtomic<T> const &) = delete;
DeterministicAtomic<T>& operator= (DeterministicAtomic<T> const &) = delete;
constexpr /* implicit */ DeterministicAtomic(T v) noexcept : data(v) {}
bool is_lock_free() const noexcept {
return data.is_lock_free();
}
bool compare_exchange_strong(
T& v0, T v1,
std::memory_order mo = std::memory_order_seq_cst) noexcept {
DeterministicSchedule::beforeSharedAccess();
bool rv = data.compare_exchange_strong(v0, v1, mo);
DeterministicSchedule::afterSharedAccess();
return rv;
}
bool compare_exchange_weak(
T& v0, T v1,
std::memory_order mo = std::memory_order_seq_cst) noexcept {
DeterministicSchedule::beforeSharedAccess();
bool rv = data.compare_exchange_weak(v0, v1, mo);
DeterministicSchedule::afterSharedAccess();
return rv;
}
T exchange(T v, std::memory_order mo = std::memory_order_seq_cst) noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = data.exchange(v, mo);
DeterministicSchedule::afterSharedAccess();
return rv;
}
/* implicit */ operator T () const noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = data;
DeterministicSchedule::afterSharedAccess();
return rv;
}
T load(std::memory_order mo = std::memory_order_seq_cst) const noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = data.load(mo);
DeterministicSchedule::afterSharedAccess();
return rv;
}
T operator= (T v) noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = (data = v);
DeterministicSchedule::afterSharedAccess();
return rv;
}
void store(T v, std::memory_order mo = std::memory_order_seq_cst) noexcept {
DeterministicSchedule::beforeSharedAccess();
data.store(v, mo);
DeterministicSchedule::afterSharedAccess();
}
T operator++ () noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = ++data;
DeterministicSchedule::afterSharedAccess();
return rv;
}
T operator++ (int postDummy) noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = data++;
DeterministicSchedule::afterSharedAccess();
return rv;
}
T operator-- () noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = --data;
DeterministicSchedule::afterSharedAccess();
return rv;
}
T operator-- (int postDummy) noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = data--;
DeterministicSchedule::afterSharedAccess();
return rv;
}
T operator+= (T v) noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = (data += v);
DeterministicSchedule::afterSharedAccess();
return rv;
}
T operator-= (T v) noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = (data -= v);
DeterministicSchedule::afterSharedAccess();
return rv;
}
T operator&= (T v) noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = (data &= v);
DeterministicSchedule::afterSharedAccess();
return rv;
}
T operator|= (T v) noexcept {
DeterministicSchedule::beforeSharedAccess();
T rv = (data |= v);
DeterministicSchedule::afterSharedAccess();
return rv;
}
};
}}
namespace folly { namespace detail {
template<>
bool Futex<test::DeterministicAtomic>::futexWait(uint32_t expected,
uint32_t waitMask);
template<>
int Futex<test::DeterministicAtomic>::futexWake(int count, uint32_t wakeMask);
}}
/*
* Copyright 2013 Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "DeterministicSchedule.h"
#include <gflags/gflags.h>
#include <gtest/gtest.h>
using namespace folly::test;
TEST(DeterministicSchedule, uniform) {
auto p = DeterministicSchedule::uniform(0);
int buckets[10] = {};
for (int i = 0; i < 100000; ++i) {
buckets[p(10)]++;
}
for (int i = 0; i < 10; ++i) {
EXPECT_TRUE(buckets[i] > 9000);
}
}
TEST(DeterministicSchedule, uniformSubset) {
auto ps = DeterministicSchedule::uniformSubset(0, 3, 100);
int buckets[10] = {};
std::set<int> seen;
for (int i = 0; i < 100000; ++i) {
if (i > 0 && (i % 100) == 0) {
EXPECT_EQ(seen.size(), 3);
seen.clear();
}
int x = ps(10);
seen.insert(x);
EXPECT_TRUE(seen.size() <= 3);
buckets[x]++;
}
for (int i = 0; i < 10; ++i) {
EXPECT_TRUE(buckets[i] > 9000);
}
}
int main(int argc, char** argv) {
testing::InitGoogleTest(&argc, argv);
google::ParseCommandLineFlags(&argc, &argv, true);
return RUN_ALL_TESTS();
}
/*
* Copyright 2013 Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include "folly/detail/Futex.h"
#include "folly/test/DeterministicSchedule.h"
#include <thread>
#include <gflags/gflags.h>
#include <gtest/gtest.h>
using namespace folly::detail;
using namespace folly::test;
typedef DeterministicSchedule DSched;
template <template<typename> class Atom>
void run_basic_tests() {
Futex<Atom> f(0);
EXPECT_FALSE(f.futexWait(1));
EXPECT_EQ(f.futexWake(), 0);
auto thr = DSched::thread([&]{
EXPECT_TRUE(f.futexWait(0));
});
while (f.futexWake() != 1) {
std::this_thread::yield();
}
DSched::join(thr);
}
TEST(Futex, basic_live) {
run_basic_tests<std::atomic>();
}
TEST(Futex, basic_deterministic) {
DSched sched(DSched::uniform(0));
run_basic_tests<DeterministicAtomic>();
}
int main(int argc, char ** argv) {
testing::InitGoogleTest(&argc, argv);
google::ParseCommandLineFlags(&argc, &argv, true);
return RUN_ALL_TESTS();
}
This diff is collapsed.
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment