Commit 50e9d1bb authored by Andrii Grynenko's avatar Andrii Grynenko Committed by Facebook GitHub Bot

AtomicNotificationQueue

Summary:
This is a lock-free replacement for NotificationQueue. Unlike notification queue it supports only one consumer, which allows simplifying the internal structure a lot and avoiding any use of spin locks.
Another improvement comparing to NotificationQueue is the use of epoll edge-triggered events. This allows completely eliminating read syscals, since we don't need to drain eventfd to stop it from waking up the epoll loop.

Differential Revision: D24297158

fbshipit-source-id: 1f418d861210f6dcad52b4d663f49eb2e8f3ae8c
parent eaee7b4b
......@@ -130,6 +130,8 @@ class PollIoBackend : public EventBaseBackendBase {
int eb_event_add(Event& event, const struct timeval* timeout) override;
int eb_event_del(Event& event) override;
bool eb_event_active(Event&, int) override { return false; }
struct FdRegistrationRecord : public boost::intrusive::slist_base_hook<
boost::intrusive::cache_last<false>> {
int count_{0};
......
This diff is collapsed.
/*
* Copyright (c) Facebook, Inc. and its affiliates.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include <folly/io/async/EventBase.h>
#include <folly/io/async/EventHandler.h>
#include <folly/portability/Fcntl.h>
#include <folly/portability/Sockets.h>
#include <folly/portability/Unistd.h>
#if defined(__linux__) && !defined(__ANDROID__)
#define FOLLY_HAVE_EVENTFD
#include <folly/io/async/EventFDWrapper.h>
#endif
namespace folly {
/**
* A producer-consumer queue for passing tasks to EventBase thread.
*
* Tasks can be added to the queue from any thread. A single EventBase
* thread can be listening to the queue. Tasks are processed in the FIFO order.
*/
class AtomicNotificationQueue : private EventBase::LoopCallback,
private EventHandler {
using Task = std::pair<Func, std::shared_ptr<RequestContext>>;
class AtomicQueue;
class Queue {
public:
Queue() {}
Queue(Queue&& other) noexcept;
Queue& operator=(Queue&& other) noexcept;
~Queue();
bool empty() const;
ssize_t size() const;
Task& front();
void pop();
void clear();
struct Node {
private:
friend class AtomicNotificationQueue::AtomicQueue;
friend class Queue;
explicit Node(Task&& t) : value(std::move(t)) {}
Task value;
Node* next{};
};
private:
friend class AtomicNotificationQueue::AtomicQueue;
Queue(Node* head, ssize_t size);
static Queue fromReversed(Node* tail);
Node* head_{nullptr};
ssize_t size_{0};
};
/**
* Lock-free queue implementation.
* The queue can be in 3 states:
* 1) Empty
* 2) Armed
* 3) Non-empty (1 or more tasks in it)
*
* This diagram shows possible state transitions:
*
* +---------+ successful arm +-------------+
* | | +---------- arm() ----------> | |
* | Empty | | Armed | +-+
* | | <------- getTasks() --------+ | | |
* +-+--+----+ consumer disarm +-------------+ |
* | ^ |
* | | |
* | | consumer pull armed push v
* | | |
* | | +-------------------+ |
* v +- getTasks() -+ | | |
* | | | Non-empty | <---- push()----+
* | ^---- arm() ---+ | |
* | +-+--+------------+-+
* | ^ ^ |
* | | | |
* +------- push() -------^ ^-- push() --+
* disarmed push
*
* push() can be called in any state. It always transitions the queue into
* Non-empty:
* When Armed - push() returns true
* When Empty/Non-empty - push() returns false
*
* getTasks() can be called in any state. It always transitions the queue into
* Empty.
*
* arm() can be can't be called if the queue is already in Armed state:
* When Empty - arm() returns an empty queue and transitions into Armed
* When Non-Empty: equivalent to getTasks()
*
*/
class AtomicQueue {
public:
AtomicQueue() {}
~AtomicQueue();
AtomicQueue(const AtomicQueue&) = delete;
AtomicQueue& operator=(const AtomicQueue&) = delete;
/*
* Pushes a task into the queue. Returns true iff the queue was armed.
* Can be called from any thread.
*/
bool push(Task&& value);
/*
* Returns true if the queue has tasks.
* Can be called from any thread.
*/
bool hasTasks() const;
/*
* Returns all tasks currently in the queue (in FIFO order). Queue becomes
* empty.
* Can be called from consumer thread only.
*/
Queue getTasks();
/*
* Tries to arm the queue.
* 1) If the queue was empty: the queue becomes armed and an empty queue is
* returned.
* 2) If the queue wasn't empty: acts as getTasks().
* Can be called from consumer thread only.
*/
Queue arm();
/*
* Returns how many armed push happened.
* Can be called from consumer thread only. And only when queue state is
* Empty.
*/
ssize_t getArmedPushCount() const {
DCHECK(!head_) << "AtomicQueue state has to be Empty";
DCHECK(successfulArmCount_ >= consumerDisarmCount_);
return successfulArmCount_ - consumerDisarmCount_;
}
private:
std::atomic<Queue::Node*> head_{};
ssize_t successfulArmCount_{0};
ssize_t consumerDisarmCount_{0};
static constexpr intptr_t kQueueArmedTag = 1;
};
public:
AtomicNotificationQueue();
~AtomicNotificationQueue() override;
/*
* Set the maximum number of tasks processed in a single round.
* Can be called from consumer thread only.
*/
void setMaxReadAtOnce(uint32_t maxAtOnce);
/*
* Returns the number of tasks in the queue.
* Can be called from any thread.
*/
int32_t size() const;
/*
* Checks if the queue is empty.
* Can be called from consumer thread only.
*/
bool empty() const;
/*
* Adds a task into the queue.
* Can be called from any thread.
*/
void putMessage(Func&& func);
/*
* Detaches the queue from an EventBase.
* Can be called from consumer thread only.
*/
void stopConsuming();
/*
* Attaches the queue to an EventBase.
* Can be called from consumer thread only.
*/
void startConsuming(EventBase* evb);
/*
* Attaches the queue to an EventBase.
* Can be called from consumer thread only.
*
* Unlike startConsuming, startConsumingInternal registers this queue as
* an internal event. This means that this event may be skipped if
* EventBase doesn't have any other registered events. This generally should
* only be used for queues managed by an EventBase itself.
*/
void startConsumingInternal(EventBase* evb);
/*
* Executes all tasks until the queue is empty.
* Can be called from consumer thread only.
*/
void drain();
/*
* Executes one round of tasks. Re-activates the event if more tasks are
* available.
* Can be called from consumer thread only.
*/
void execute();
private:
/*
* Write into the signal fd to wake up the consumer thread.
*/
void notifyFd();
/*
* Read all messages from the signal fd.
*/
void drainFd();
/*
* Executes one round of tasks. Returns true iff tasks were run.
* Can be called from consumer thread only.
*/
bool drive();
/*
* Either arm the queue or reactivate the EventBase event.
* This has to be a loop callback because the event can't be activated from
* within the event callback. It also allows delayed re-arming the queue.
*/
void runLoopCallback() noexcept override;
void startConsumingImpl(EventBase* evb, bool internal);
void handlerReady(uint16_t) noexcept override;
void activateEvent();
/**
* Check that the AtomicNotificationQueue is being used from the correct
* process.
*
* If you create a AtomicNotificationQueue in one process, then fork, and try
* to send messages to the queue from the child process, you're going to have
* a bad time. Unfortunately users have (accidentally) run into this.
*
* Because we use an eventfd/pipe, the child process can actually signal the
* parent process that an event is ready. However, it can't put anything on
* the parent's queue, so the parent wakes up and finds an empty queue. This
* check ensures that we catch the problem in the misbehaving child process
* code, and crash before signalling the parent process.
*/
void checkPid() const;
[[noreturn]] FOLLY_NOINLINE void checkPidFail() const;
AtomicQueue atomicQueue_;
Queue queue_;
std::atomic<int32_t> queueSize_{0};
int32_t maxReadAtOnce_{10};
int eventfd_{-1};
int pipeFds_[2]{-1, -1}; // to fallback to on older/non-linux systems
/*
* If event is registered with the EventBase, this describes whether
* edge-triggered flag was set for it. For edge-triggered events we don't
* need to drain the fd to deactivate them.
*/
bool edgeTriggeredSet_{false};
EventBase* evb_{nullptr};
ssize_t writesObserved_{0};
ssize_t writesLocal_{0};
const pid_t pid_;
};
} // namespace folly
......@@ -52,6 +52,8 @@ class EventBaseBackend : public folly::EventBaseBackendBase {
int eb_event_add(Event& event, const struct timeval* timeout) override;
int eb_event_del(EventBaseBackendBase::Event& event) override;
bool eb_event_active(Event& event, int res) override;
private:
event_base* evb_;
};
......@@ -116,6 +118,11 @@ int EventBaseBackend::eb_event_del(EventBaseBackendBase::Event& event) {
return event_del(event.getEvent());
}
bool EventBaseBackend::eb_event_active(Event& event, int res) {
event_active(event.getEvent(), res, 1);
return true;
}
EventBaseBackend::~EventBaseBackend() {
std::lock_guard<std::mutex> lock(libevent_mutex_);
event_base_free(evb_);
......@@ -551,10 +558,13 @@ void EventBase::terminateLoopSoon() {
}
}
void EventBase::runInLoop(LoopCallback* callback, bool thisIteration) {
void EventBase::runInLoop(
LoopCallback* callback,
bool thisIteration,
std::shared_ptr<RequestContext> rctx) {
dcheckIsInEventBaseThread();
callback->cancelLoopCallback();
callback->context_ = RequestContext::saveContext();
callback->context_ = std::move(rctx);
if (runOnceCallbacks_ != nullptr && thisIteration) {
runOnceCallbacks_->push_back(*callback);
} else {
......
......@@ -54,6 +54,7 @@ namespace folly {
class EventBaseBackendBase;
using Cob = Func; // defined in folly/Executor.h
template <typename MessageT>
class NotificationQueue;
......@@ -469,7 +470,10 @@ class EventBase : public TimeoutManager,
* Ideally we would not need thisIteration, and instead just use
* runInLoop with loop() (instead of terminateLoopSoon).
*/
void runInLoop(LoopCallback* callback, bool thisIteration = false);
void runInLoop(
LoopCallback* callback,
bool thisIteration = false,
std::shared_ptr<RequestContext> rctx = RequestContext::saveContext());
/**
* Convenience function to call runInLoop() with a folly::Function.
......
......@@ -17,6 +17,24 @@
#include <folly/io/async/EventBaseBackendBase.h>
#include <folly/io/async/EventBase.h>
#if defined(__linux__) && !FOLLY_MOBILE
#define FOLLY_USE_EPOLLET
#include <sys/epoll.h>
struct event_base {
void* evsel;
void* evbase;
};
struct epollop {
void* fds;
void* events;
int nevents;
int epfd;
};
#endif
namespace folly {
void EventBaseEvent::eb_ev_base(EventBase* evb) {
evb_ = evb;
......@@ -50,4 +68,50 @@ int EventBaseEvent::eb_event_del() {
return -1;
}
bool EventBaseEvent::eb_event_active(int res) {
auto* backend = evb_ ? (evb_->getBackend()) : nullptr;
if (backend) {
return backend->eb_event_active(*this, res);
}
return false;
}
bool EventBaseEvent::setEdgeTriggered() {
#ifdef FOLLY_USE_EPOLLET
// Until v2 libevent doesn't expose API to set edge-triggered flag for events.
// If epoll backend is used by libevent, we can enable it though epoll_ctl
// directly.
// Note that this code depends on internal event_base and epollop layout, so
// we have to validate libevent version.
static const bool supportedVersion =
!strcmp(event_get_version(), "1.4.14b-stable");
if (!supportedVersion) {
return false;
}
auto* base = evb_ ? (evb_->getLibeventBase()) : nullptr;
if (!base || strcmp(event_base_get_method(base), "epoll")) {
return false;
}
auto epfd = static_cast<epollop*>(base->evbase)->epfd;
epoll_event epev = {0, {0}};
epev.data.fd = eb_ev_fd();
epev.events = EPOLLET;
if (eb_ev_events() & EV_READ) {
epev.events |= EPOLLIN;
}
if (eb_ev_events() & EV_WRITE) {
epev.events |= EPOLLOUT;
}
if (::epoll_ctl(epfd, EPOLL_CTL_MOD, eb_ev_fd(), &epev) == -1) {
LOG(DFATAL) << "epoll_ctl failed: " << errno;
return false;
}
return true;
#else
return false;
#endif
}
} // namespace folly
......@@ -158,6 +158,10 @@ class EventBaseEvent {
int eb_event_del();
bool eb_event_active(int res);
bool setEdgeTriggered();
protected:
struct event event_;
EventBase* evb_{nullptr};
......@@ -184,6 +188,8 @@ class EventBaseBackendBase {
virtual int eb_event_add(Event& event, const struct timeval* timeout) = 0;
virtual int eb_event_del(Event& event) = 0;
virtual bool eb_event_active(Event& event, int res) = 0;
};
} // namespace folly
......@@ -181,6 +181,22 @@ class EventHandler {
void resetEventCallback() { event_.resetCallback(); }
/*
* If supported by the backend updates the event to be edge-triggered.
* Returns true iff the update was successful.
*
* This should only be used for already registered events (e.g. after
* registerHandler/registerInternalHandler calls). Calling any other method
* on the EventHandler may reset this flag.
* This can be useful to avoid read calls with eventfds.
*/
bool setEdgeTriggered() { return event_.setEdgeTriggered(); }
/*
* Make an event active.
*/
bool activateEvent(int res) { return event_.eb_event_active(res); }
private:
bool registerImpl(uint16_t events, bool internal);
void ensureNotRegistered(const char* fn);
......
......@@ -18,6 +18,7 @@
#include <thread>
#include <folly/Benchmark.h>
#include <folly/io/async/AtomicNotificationQueue.h>
#include <folly/io/async/EventBase.h>
#include <folly/io/async/NotificationQueue.h>
#include <folly/synchronization/Baton.h>
......@@ -28,23 +29,43 @@ static size_t constexpr kMaxRead = 20;
static size_t constexpr kProducerWarmup = 1000;
static size_t constexpr kBusyLoopSize = 0;
class MockConsumer : public NotificationQueue<Func>::Consumer {
using Task = std::pair<Func, std::shared_ptr<RequestContext>>;
class MockConsumer : public NotificationQueue<Task>::Consumer {
public:
void messageAvailable(Func&& message) noexcept override { message(); }
void messageAvailable(Task&& message) noexcept override {
RequestContextScopeGuard rctx(std::move(message.second));
message.first();
}
};
void putMessageHelper(NotificationQueue<Task>& q, Func f) {
q.putMessage(Task(std::move(f), RequestContext::saveContext()));
}
struct AtomicNotificationQueueConsumerAdaptor {
void startConsuming(EventBase* evb, AtomicNotificationQueue* queue) {
queue->startConsuming(evb);
}
};
void putMessageHelper(AtomicNotificationQueue& q, Func f) {
q.putMessage(std::move(f));
}
static void burn(size_t n) {
for (size_t i = 0; i < n; ++i) {
folly::doNotOptimizeAway(i);
}
}
template <typename Queue, typename Consumer>
void multiProducerMultiConsumer(
int iters,
size_t numProducers,
size_t numConsumers) {
BenchmarkSuspender susp;
NotificationQueue<Func> queue;
Queue queue;
std::vector<std::unique_ptr<EventBase>> consumerEventBases;
std::vector<std::thread> consumerThreads;
// Initialize consumers
......@@ -54,7 +75,7 @@ void multiProducerMultiConsumer(
EventBase& base = *consumerEventBases.back();
consumerThreads.emplace_back([&base, &queue]() mutable {
base.setMaxReadAtOnce(kMaxRead);
MockConsumer consumer;
Consumer consumer;
consumer.startConsuming(&base, &queue);
base.loopForever();
});
......@@ -86,7 +107,7 @@ void multiProducerMultiConsumer(
producersWarmedUp.fetch_add(1, std::memory_order_relaxed) + 1) {
warmUpBaton.post();
}
queue.putMessage([&itemsToProcess, &finishedBaton]() {
putMessageHelper(queue, [&itemsToProcess, &finishedBaton]() {
burn(kBusyLoopSize);
if (itemsToProcess.fetch_sub(1, std::memory_order_relaxed) == 0) {
finishedBaton.post();
......@@ -116,6 +137,25 @@ void multiProducerMultiConsumer(
}
}
void multiProducerMultiConsumerNQ(
int iters,
size_t numProducers,
size_t numConsumers) {
multiProducerMultiConsumer<NotificationQueue<Task>, MockConsumer>(
iters, numProducers, numConsumers);
}
void multiProducerMultiConsumerANQ(
int iters,
size_t numProducers,
size_t numConsumers) {
CHECK(numConsumers == 1);
multiProducerMultiConsumer<
AtomicNotificationQueue,
AtomicNotificationQueueConsumerAdaptor>(
iters, numProducers, numConsumers);
}
BENCHMARK(EnqueueBenchmark, n) {
BenchmarkSuspender suspender;
NotificationQueue<Func> queue;
......@@ -128,13 +168,13 @@ BENCHMARK(EnqueueBenchmark, n) {
BENCHMARK(DequeueBenchmark, n) {
BenchmarkSuspender suspender;
NotificationQueue<Func> queue;
NotificationQueue<Task> queue;
EventBase base;
MockConsumer consumer;
consumer.setMaxReadAtOnce(kMaxRead);
consumer.startConsumingInternal(&base, &queue);
for (unsigned int i = 0; i < n; ++i) {
queue.putMessage([]() {});
putMessageHelper(queue, []() {});
}
suspender.dismiss();
for (unsigned int i = 0; i <= (n / kMaxRead); ++i) {
......@@ -148,47 +188,54 @@ BENCHMARK(DequeueBenchmark, n) {
}
BENCHMARK_DRAW_LINE();
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _1p__1c, 1, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _2p__1c, 2, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _4p__1c, 4, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _8p__1c, 8, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 16p__1c, 16, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 32p__1c, 32, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerANQ, _1p__1c, 1, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerANQ, _2p__1c, 2, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerANQ, _4p__1c, 4, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerANQ, _8p__1c, 8, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerANQ, 16p__1c, 16, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerANQ, 32p__1c, 32, 1)
BENCHMARK_DRAW_LINE();
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _1p__1c, 1, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _2p__1c, 2, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _4p__1c, 4, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _8p__1c, 8, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 16p__1c, 16, 1)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 32p__1c, 32, 1)
BENCHMARK_DRAW_LINE();
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _1p__2c, 1, 2)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _2p__2c, 2, 2)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _4p__2c, 4, 2)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _8p__2c, 8, 2)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 16p__2c, 16, 2)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 32p__2c, 32, 2)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _1p__2c, 1, 2)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _2p__2c, 2, 2)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _4p__2c, 4, 2)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _8p__2c, 8, 2)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 16p__2c, 16, 2)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 32p__2c, 32, 2)
BENCHMARK_DRAW_LINE();
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _1p__4c, 1, 4)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _2p__4c, 2, 4)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _4p__4c, 4, 4)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _8p__4c, 8, 4)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 16p__4c, 16, 4)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 32p__4c, 32, 4)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _1p__4c, 1, 4)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _2p__4c, 2, 4)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _4p__4c, 4, 4)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _8p__4c, 8, 4)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 16p__4c, 16, 4)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 32p__4c, 32, 4)
BENCHMARK_DRAW_LINE();
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _1p__8c, 1, 8)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _2p__8c, 2, 8)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _4p__8c, 4, 8)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _8p__8c, 8, 8)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 16p__8c, 16, 8)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 32p__8c, 32, 8)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _1p__8c, 1, 8)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _2p__8c, 2, 8)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _4p__8c, 4, 8)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _8p__8c, 8, 8)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 16p__8c, 16, 8)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 32p__8c, 32, 8)
BENCHMARK_DRAW_LINE();
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _1p_16c, 1, 16)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _2p_16c, 2, 16)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _4p_16c, 4, 16)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _8p_16c, 8, 16)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 16p_16c, 16, 16)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 32p_16c, 32, 16)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _1p_16c, 1, 16)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _2p_16c, 2, 16)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _4p_16c, 4, 16)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _8p_16c, 8, 16)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 16p_16c, 16, 16)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 32p_16c, 32, 16)
BENCHMARK_DRAW_LINE();
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _1p_32c, 1, 32)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _2p_32c, 2, 32)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _4p_32c, 4, 32)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, _8p_32c, 8, 32)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 16p_32c, 16, 32)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumer, 32p_32c, 32, 32)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _1p_32c, 1, 32)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _2p_32c, 2, 32)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _4p_32c, 4, 32)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, _8p_32c, 8, 32)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 16p_32c, 16, 32)
BENCHMARK_NAMED_PARAM(multiProducerMultiConsumerNQ, 32p_32c, 32, 32)
BENCHMARK_DRAW_LINE();
int main(int argc, char* argv[]) {
......
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