Commit cd39f451 authored by Dan Melnic's avatar Dan Melnic Committed by Facebook GitHub Bot

IoUringBackend rework

Summary:
IoUringBackend rework

(Note: this ignores all push blocking failures!)

Reviewed By: kevin-vigor

Differential Revision: D25287276

fbshipit-source-id: b69cc715e8bf5da2e493a742e71a62795a62f955
parent 8da5b759
...@@ -14,16 +14,105 @@ ...@@ -14,16 +14,105 @@
* limitations under the License. * limitations under the License.
*/ */
#include <folly/experimental/io/IoUringBackend.h> #include <signal.h>
#include <sys/timerfd.h>
#include <folly/FileUtil.h>
#include <folly/Likely.h> #include <folly/Likely.h>
#include <folly/SpinLock.h>
#include <folly/String.h> #include <folly/String.h>
#include <folly/container/F14Map.h> #include <folly/container/F14Map.h>
#include <folly/container/F14Set.h> #include <folly/container/F14Set.h>
#include <folly/experimental/io/IoUringBackend.h>
#include <folly/portability/GFlags.h> #include <folly/portability/GFlags.h>
#include <folly/portability/Sockets.h> #include <folly/portability/Sockets.h>
#include <folly/synchronization/CallOnce.h> #include <folly/synchronization/CallOnce.h>
#include <glog/logging.h> extern "C" FOLLY_ATTR_WEAK void eb_poll_loop_pre_hook(uint64_t* call_time);
extern "C" FOLLY_ATTR_WEAK void eb_poll_loop_post_hook(
uint64_t call_time,
int ret);
namespace {
struct SignalRegistry {
struct SigInfo {
struct sigaction sa_ {};
size_t refs_{0};
};
using SignalMap = std::map<int, SigInfo>;
constexpr SignalRegistry() {}
void notify(int sig);
void setNotifyFd(int sig, int fd);
// lock protecting the signal map
folly::MicroSpinLock mapLock_ = {0};
std::unique_ptr<SignalMap> map_;
std::atomic<int> notifyFd_{-1};
};
SignalRegistry& getSignalRegistry() {
static auto& sInstance = *new SignalRegistry();
return sInstance;
}
void evSigHandler(int sig) {
getSignalRegistry().notify(sig);
}
void SignalRegistry::notify(int sig) {
// use try_lock in case somebody already has the lock
std::unique_lock<folly::MicroSpinLock> lk(mapLock_, std::try_to_lock);
if (lk.owns_lock()) {
int fd = notifyFd_.load();
if (fd >= 0) {
uint8_t sigNum = static_cast<uint8_t>(sig);
::write(fd, &sigNum, 1);
}
}
}
void SignalRegistry::setNotifyFd(int sig, int fd) {
std::lock_guard<folly::MicroSpinLock> g(mapLock_);
if (fd >= 0) {
if (!map_) {
map_ = std::make_unique<SignalMap>();
}
// switch the fd
notifyFd_.store(fd);
auto iter = (*map_).find(sig);
if (iter != (*map_).end()) {
iter->second.refs_++;
} else {
auto& entry = (*map_)[sig];
entry.refs_ = 1;
struct sigaction sa = {};
sa.sa_handler = evSigHandler;
sa.sa_flags |= SA_RESTART;
::sigfillset(&sa.sa_mask);
if (::sigaction(sig, &sa, &entry.sa_) == -1) {
(*map_).erase(sig);
}
}
} else {
notifyFd_.store(fd);
if (map_) {
auto iter = (*map_).find(sig);
if ((iter != (*map_).end()) && (--iter->second.refs_ == 0)) {
auto entry = iter->second;
(*map_).erase(iter);
// just restore
::sigaction(sig, &entry.sa_, nullptr);
}
}
}
}
} // namespace
static constexpr int64_t kUnlimitedMlock = -1; static constexpr int64_t kUnlimitedMlock = -1;
DEFINE_int64( DEFINE_int64(
...@@ -183,6 +272,33 @@ static folly::Indestructible<SQGroupInfoRegistry> sSQGroupInfoRegistry; ...@@ -183,6 +272,33 @@ static folly::Indestructible<SQGroupInfoRegistry> sSQGroupInfoRegistry;
} // namespace } // namespace
namespace folly { namespace folly {
IoUringBackend::TimerEntry::TimerEntry(
Event* event,
const struct timeval& timeout)
: event_(event) {
setExpireTime(timeout, std::chrono::steady_clock::now());
}
IoUringBackend::SocketPair::SocketPair() {
if (::socketpair(AF_UNIX, SOCK_STREAM, 0, fds_.data())) {
throw std::runtime_error("socketpair error");
}
// set the sockets to non blocking mode
for (auto fd : fds_) {
auto flags = ::fcntl(fd, F_GETFL, 0);
::fcntl(fd, F_SETFL, flags | O_NONBLOCK);
}
}
IoUringBackend::SocketPair::~SocketPair() {
for (auto fd : fds_) {
if (fd >= 0) {
::close(fd);
}
}
}
IoUringBackend::FdRegistry::FdRegistry(struct io_uring& ioRing, size_t n) IoUringBackend::FdRegistry::FdRegistry(struct io_uring& ioRing, size_t n)
: ioRing_(ioRing), files_(n, -1), inUse_(n), records_(n) {} : ioRing_(ioRing), files_(n, -1), inUse_(n), records_(n) {}
...@@ -253,8 +369,14 @@ bool IoUringBackend::FdRegistry::free( ...@@ -253,8 +369,14 @@ bool IoUringBackend::FdRegistry::free(
} }
IoUringBackend::IoUringBackend(Options options) IoUringBackend::IoUringBackend(Options options)
: PollIoBackend(options), : options_(options),
numEntries_(options.capacity),
fdRegistry_(ioRing_, options.useRegisteredFds ? options.capacity : 0) { fdRegistry_(ioRing_, options.useRegisteredFds ? options.capacity : 0) {
// create the timer fd
timerFd_ = ::timerfd_create(CLOCK_MONOTONIC, TFD_NONBLOCK | TFD_CLOEXEC);
if (timerFd_ < 0) {
throw std::runtime_error("timerfd_create error");
}
FOLLY_MAYBE_UNUSED static bool sMlockInit = []() { FOLLY_MAYBE_UNUSED static bool sMlockInit = []() {
int ret = 0; int ret = 0;
if (FLAGS_io_uring_mlock_size) { if (FLAGS_io_uring_mlock_size) {
...@@ -311,11 +433,11 @@ IoUringBackend::IoUringBackend(Options options) ...@@ -311,11 +433,11 @@ IoUringBackend::IoUringBackend(Options options)
// timer entry // timer entry
timerEntry_ = std::make_unique<IoSqe>(this, false, true /*persist*/); timerEntry_ = std::make_unique<IoSqe>(this, false, true /*persist*/);
timerEntry_->backendCb_ = PollIoBackend::processTimerIoCb; timerEntry_->backendCb_ = IoUringBackend::processTimerIoSqe;
// signal entry // signal entry
signalReadEntry_ = std::make_unique<IoSqe>(this, false, true /*persist*/); signalReadEntry_ = std::make_unique<IoSqe>(this, false, true /*persist*/);
signalReadEntry_->backendCb_ = PollIoBackend::processSignalReadIoCb; signalReadEntry_->backendCb_ = IoUringBackend::processSignalReadIoSqe;
// we need to call the init before adding the timer fd // we need to call the init before adding the timer fd
// so we avoid a deadlock - waiting for the queue to be drained // so we avoid a deadlock - waiting for the queue to be drained
...@@ -337,31 +459,37 @@ IoUringBackend::~IoUringBackend() { ...@@ -337,31 +459,37 @@ IoUringBackend::~IoUringBackend() {
shuttingDown_ = true; shuttingDown_ = true;
cleanup(); cleanup();
CHECK(!timerEntry_);
CHECK(!signalReadEntry_);
CHECK(freeList_.empty());
::close(timerFd_);
} }
void IoUringBackend::cleanup() { void IoUringBackend::cleanup() {
if (ioRing_.ring_fd > 0) { if (ioRing_.ring_fd > 0) {
// release the nonsubmitted items from the submitList // release the nonsubmitted items from the submitList
while (!submitList_.empty()) { while (!submitList_.empty()) {
auto* ioCb = &submitList_.front(); auto* ioSqe = &submitList_.front();
submitList_.pop_front(); submitList_.pop_front();
releaseIoCb(ioCb); releaseIoSqe(ioSqe);
} }
// release the active events // release the active events
while (!activeEvents_.empty()) { while (!activeEvents_.empty()) {
auto* ioCb = &activeEvents_.front(); auto* ioSqe = &activeEvents_.front();
activeEvents_.pop_front(); activeEvents_.pop_front();
releaseIoCb(ioCb); releaseIoSqe(ioSqe);
} }
// wait for the outstanding events to finish // wait for the outstanding events to finish
while (numIoCbInUse()) { while (numIoSqeInUse()) {
struct io_uring_cqe* cqe = nullptr; struct io_uring_cqe* cqe = nullptr;
::io_uring_wait_cqe(&ioRing_, &cqe); ::io_uring_wait_cqe(&ioRing_, &cqe);
if (cqe) { if (cqe) {
IoSqe* sqe = reinterpret_cast<IoSqe*>(io_uring_cqe_get_data(cqe)); IoSqe* sqe = reinterpret_cast<IoSqe*>(io_uring_cqe_get_data(cqe));
releaseIoCb(sqe); releaseIoSqe(sqe);
::io_uring_cqe_seen(&ioRing_, cqe); ::io_uring_cqe_seen(&ioRing_, cqe);
} }
} }
...@@ -405,16 +533,499 @@ bool IoUringBackend::isAvailable() { ...@@ -405,16 +533,499 @@ bool IoUringBackend::isAvailable() {
return sAvailable; return sAvailable;
} }
void* IoUringBackend::allocSubmissionEntry() { bool IoUringBackend::addTimerFd() {
return get_sqe(); auto* entry = allocSubmissionEntry(); // this can be nullptr
timerEntry_->prepPollAdd(entry, timerFd_, POLLIN, true /*registerFd*/);
return (1 == submitOne());
}
bool IoUringBackend::addSignalFds() {
auto* entry = allocSubmissionEntry(); // this can be nullptr
signalReadEntry_->prepPollAdd(
entry, signalFds_.readFd(), POLLIN, false /*registerFd*/);
return (1 == submitOne());
}
void IoUringBackend::scheduleTimeout() {
if (!timerChanged_) {
return;
}
// reset
timerChanged_ = false;
if (!timers_.empty()) {
auto delta = timers_.begin()->second[0].getRemainingTime(
std::chrono::steady_clock::now());
if (delta.count() < 1000) {
delta = std::chrono::microseconds(1000);
}
scheduleTimeout(delta);
} else {
scheduleTimeout(std::chrono::microseconds(0)); // disable
}
// we do not call addTimerFd() here
// since it has to be added only once, after
// we process a poll callback
}
void IoUringBackend::scheduleTimeout(const std::chrono::microseconds& us) {
struct itimerspec val;
val.it_interval = {0, 0};
val.it_value.tv_sec =
std::chrono::duration_cast<std::chrono::seconds>(us).count();
val.it_value.tv_nsec =
std::chrono::duration_cast<std::chrono::nanoseconds>(us).count() %
1000000000LL;
CHECK_EQ(::timerfd_settime(timerFd_, 0, &val, nullptr), 0);
}
void IoUringBackend::addTimerEvent(
Event& event,
const struct timeval* timeout) {
// first try to remove if already existing
auto iter1 = eventToTimers_.find(&event);
if (iter1 != eventToTimers_.end()) {
// no neeed to remove it from eventToTimers_
auto expireTime = iter1->second;
auto iter2 = timers_.find(expireTime);
for (auto iter = iter2->second.begin(), last = iter2->second.end();
iter != last;
++iter) {
if (iter->event_ == &event) {
iter2->second.erase(iter);
break;
}
}
if (iter2->second.empty()) {
timers_.erase(iter2);
}
}
TimerEntry entry(&event, *timeout);
if (!timerChanged_) {
timerChanged_ =
timers_.empty() || (entry.expireTime_ < timers_.begin()->first);
}
timers_[entry.expireTime_].push_back(entry);
eventToTimers_[&event] = entry.expireTime_;
}
void IoUringBackend::removeTimerEvent(Event& event) {
auto iter1 = eventToTimers_.find(&event);
CHECK(iter1 != eventToTimers_.end());
auto expireTime = iter1->second;
eventToTimers_.erase(iter1);
auto iter2 = timers_.find(expireTime);
CHECK(iter2 != timers_.end());
for (auto iter = iter2->second.begin(), last = iter2->second.end();
iter != last;
++iter) {
if (iter->event_ == &event) {
iter2->second.erase(iter);
break;
}
}
if (iter2->second.empty()) {
if (!timerChanged_) {
timerChanged_ = (iter2 == timers_.begin());
}
timers_.erase(iter2);
}
}
size_t IoUringBackend::processTimers() {
size_t ret = 0;
uint64_t data = 0;
// this can fail with but it is OK since the fd
// will still be readable
folly::readNoInt(timerFd_, &data, sizeof(data));
auto now = std::chrono::steady_clock::now();
while (!timers_.empty() && (now >= timers_.begin()->first)) {
if (!timerChanged_) {
timerChanged_ = true;
}
auto vec = std::move(timers_.begin()->second);
timers_.erase(timers_.begin());
for (auto& entry : vec) {
ret++;
eventToTimers_.erase(entry.event_);
auto* ev = entry.event_->getEvent();
ev->ev_res = EV_TIMEOUT;
event_ref_flags(ev).get() = EVLIST_INIT;
(*event_ref_callback(ev))((int)ev->ev_fd, ev->ev_res, event_ref_arg(ev));
}
}
return ret;
}
void IoUringBackend::addSignalEvent(Event& event) {
auto* ev = event.getEvent();
signals_[ev->ev_fd].insert(&event);
// we pass the write fd for notifications
getSignalRegistry().setNotifyFd(ev->ev_fd, signalFds_.writeFd());
}
void IoUringBackend::removeSignalEvent(Event& event) {
auto* ev = event.getEvent();
auto iter = signals_.find(ev->ev_fd);
if (iter != signals_.end()) {
getSignalRegistry().setNotifyFd(ev->ev_fd, -1);
}
}
size_t IoUringBackend::processSignals() {
size_t ret = 0;
static constexpr auto kNumEntries = NSIG * 2;
static_assert(
NSIG < 256, "Use a different data type to cover all the signal values");
std::array<bool, NSIG> processed{};
std::array<uint8_t, kNumEntries> signals;
ssize_t num =
folly::readNoInt(signalFds_.readFd(), signals.data(), signals.size());
for (ssize_t i = 0; i < num; i++) {
int signum = static_cast<int>(signals[i]);
if ((signum >= 0) && (signum < static_cast<int>(processed.size())) &&
!processed[signum]) {
processed[signum] = true;
auto iter = signals_.find(signum);
if (iter != signals_.end()) {
auto& set = iter->second;
for (auto& event : set) {
auto* ev = event->getEvent();
ev->ev_res = 0;
event_ref_flags(ev) |= EVLIST_ACTIVE;
(*event_ref_callback(ev))(
(int)ev->ev_fd, ev->ev_res, event_ref_arg(ev));
event_ref_flags(ev) &= ~EVLIST_ACTIVE;
}
}
}
}
// add the signal fd(s) back
addSignalFds();
return ret;
}
IoUringBackend::IoSqe* IoUringBackend::allocIoSqe(const EventCallback& cb) {
// try to allocate from the pool first
if ((cb.type_ == EventCallback::Type::TYPE_NONE) && (!freeList_.empty())) {
auto* ret = &freeList_.front();
freeList_.pop_front();
numIoSqeInUse_++;
return ret;
}
// alloc a new IoSqe
auto* ret = allocNewIoSqe(cb);
if (FOLLY_LIKELY(!!ret)) {
numIoSqeInUse_++;
}
return ret;
}
void IoUringBackend::releaseIoSqe(IoUringBackend::IoSqe* aioIoSqe) {
CHECK_GT(numIoSqeInUse_, 0);
aioIoSqe->cbData_.releaseData();
// unregister the file descriptor record
if (aioIoSqe->fdRecord_) {
unregisterFd(aioIoSqe->fdRecord_);
aioIoSqe->fdRecord_ = nullptr;
}
if (FOLLY_LIKELY(aioIoSqe->poolAlloc_)) {
numIoSqeInUse_--;
aioIoSqe->event_ = nullptr;
freeList_.push_front(*aioIoSqe);
} else {
if (!aioIoSqe->persist_) {
numIoSqeInUse_--;
delete aioIoSqe;
}
}
}
void IoUringBackend::processPollIo(IoSqe* ioSqe, int64_t res) noexcept {
auto* ev = ioSqe->event_ ? (ioSqe->event_->getEvent()) : nullptr;
if (ev) {
if (~event_ref_flags(ev) & EVLIST_INTERNAL) {
// if this is not a persistent event
// remove the EVLIST_INSERTED flags
// and dec the numInsertedEvents_
if (~ev->ev_events & EV_PERSIST) {
DCHECK(numInsertedEvents_ > 0);
numInsertedEvents_--;
event_ref_flags(ev) &= ~EVLIST_INSERTED;
}
}
// add it to the active list
event_ref_flags(ev) |= EVLIST_ACTIVE;
ev->ev_res = res;
activeEvents_.push_back(*ioSqe);
} else {
releaseIoSqe(ioSqe);
}
}
size_t IoUringBackend::processActiveEvents() {
size_t ret = 0;
IoSqe* ioSqe;
while (!activeEvents_.empty() && !loopBreak_) {
bool release = true;
ioSqe = &activeEvents_.front();
activeEvents_.pop_front();
ret++;
auto* event = ioSqe->event_;
auto* ev = event ? event->getEvent() : nullptr;
if (ev) {
// remove it from the active list
event_ref_flags(ev) &= ~EVLIST_ACTIVE;
bool inserted = (event_ref_flags(ev) & EVLIST_INSERTED);
// prevent the callback from freeing the aioIoSqe
ioSqe->useCount_++;
if (!ioSqe->cbData_.processCb(ev->ev_res)) {
// adjust the ev_res for the poll case
ev->ev_res = getPollEvents(ev->ev_res, ev->ev_events);
// handle spurious poll events that return 0
// this can happen during high load on process startup
if (ev->ev_res) {
(*event_ref_callback(ev))(
(int)ev->ev_fd, ev->ev_res, event_ref_arg(ev));
}
}
// get the event again
event = ioSqe->event_;
ev = event ? event->getEvent() : nullptr;
if (ev && inserted && event_ref_flags(ev) & EVLIST_INSERTED &&
!shuttingDown_) {
release = false;
eb_event_modify_inserted(*event, ioSqe);
}
ioSqe->useCount_--;
} else {
ioSqe->processActive();
}
if (release) {
releaseIoSqe(ioSqe);
}
}
return ret;
}
int IoUringBackend::eb_event_base_loop(int flags) {
// schedule the timers
bool done = false;
auto waitForEvents = (flags & EVLOOP_NONBLOCK) ? WaitForEventsMode::DONT_WAIT
: WaitForEventsMode::WAIT;
while (!done) {
scheduleTimeout();
// check if we need to break here
if (loopBreak_) {
loopBreak_ = false;
break;
}
submitList(submitList_, waitForEvents);
if (!numInsertedEvents_ && timers_.empty() && signals_.empty()) {
return 1;
}
uint64_t call_time = 0;
if (eb_poll_loop_pre_hook) {
eb_poll_loop_pre_hook(&call_time);
}
// do not wait for events if EVLOOP_NONBLOCK is set
int ret = getActiveEvents(waitForEvents);
if (eb_poll_loop_post_hook) {
eb_poll_loop_post_hook(call_time, ret);
}
size_t numProcessedTimers = 0;
// save the processTimers_
// this means we've received a notification
// and we need to add the timer fd back
bool processTimersFlag = processTimers_;
if (processTimers_ && !loopBreak_) {
numProcessedTimers = processTimers();
processTimers_ = false;
}
size_t numProcessedSignals = 0;
if (processSignals_ && !loopBreak_) {
numProcessedSignals = processSignals();
processSignals_ = false;
}
if (!activeEvents_.empty() && !loopBreak_) {
processActiveEvents();
if (flags & EVLOOP_ONCE) {
done = true;
}
} else if (flags & EVLOOP_NONBLOCK) {
if (signals_.empty()) {
done = true;
}
}
if (!done && (numProcessedTimers || numProcessedSignals) &&
(flags & EVLOOP_ONCE)) {
done = true;
}
if (processTimersFlag) {
addTimerFd();
}
}
return 0;
}
int IoUringBackend::eb_event_base_loopbreak() {
loopBreak_ = true;
return 0;
}
int IoUringBackend::eb_event_add(Event& event, const struct timeval* timeout) {
auto* ev = event.getEvent();
CHECK(ev);
CHECK(!(event_ref_flags(ev) & ~EVLIST_ALL));
// we do not support read/write timeouts
if (timeout) {
event_ref_flags(ev) |= EVLIST_TIMEOUT;
addTimerEvent(event, timeout);
return 0;
}
if (ev->ev_events & EV_SIGNAL) {
event_ref_flags(ev) |= EVLIST_INSERTED;
addSignalEvent(event);
return 0;
}
if ((ev->ev_events & (EV_READ | EV_WRITE)) &&
!(event_ref_flags(ev) & (EVLIST_INSERTED | EVLIST_ACTIVE))) {
auto* ioSqe = allocIoSqe(event.getCallback());
CHECK(ioSqe);
ioSqe->event_ = &event;
// just append it
submitList_.push_back(*ioSqe);
if (~event_ref_flags(ev) & EVLIST_INTERNAL) {
numInsertedEvents_++;
}
event_ref_flags(ev) |= EVLIST_INSERTED;
event.setUserData(ioSqe);
}
return 0;
}
int IoUringBackend::eb_event_del(Event& event) {
if (!event.eb_ev_base()) {
return -1;
}
auto* ev = event.getEvent();
if (event_ref_flags(ev) & EVLIST_TIMEOUT) {
event_ref_flags(ev) &= ~EVLIST_TIMEOUT;
removeTimerEvent(event);
return 1;
}
if (!(event_ref_flags(ev) & (EVLIST_ACTIVE | EVLIST_INSERTED))) {
return -1;
}
if (ev->ev_events & EV_SIGNAL) {
event_ref_flags(ev) &= ~(EVLIST_INSERTED | EVLIST_ACTIVE);
removeSignalEvent(event);
return 0;
}
auto* ioSqe = reinterpret_cast<IoSqe*>(event.getUserData());
bool wasLinked = ioSqe->is_linked();
ioSqe->resetEvent();
// if the event is on the active list, we just clear the flags
// and reset the event_ ptr
if (event_ref_flags(ev) & EVLIST_ACTIVE) {
event_ref_flags(ev) &= ~EVLIST_ACTIVE;
}
if (event_ref_flags(ev) & EVLIST_INSERTED) {
event_ref_flags(ev) &= ~EVLIST_INSERTED;
// not in use - we can cancel it
if (!ioSqe->useCount_ && !wasLinked) {
// io_cancel will attempt to cancel the event. the result is
// EINVAL - usually the event has already been delivered
// EINPROGRESS - cancellation in progress
// EFAULT - bad ctx
// regardless, we want to dec the numInsertedEvents_
// since even if the events get delivered, the event ptr is nullptr
int ret = cancelOne(ioSqe);
if (ret < 0) {
// release the ioSqe
releaseIoSqe(ioSqe);
}
} else {
if (!ioSqe->useCount_) {
releaseIoSqe(ioSqe);
}
}
if (~event_ref_flags(ev) & EVLIST_INTERNAL) {
CHECK_GT(numInsertedEvents_, 0);
numInsertedEvents_--;
}
return 0;
} else {
// we can have an EVLIST_ACTIVE event
// which does not have the EVLIST_INSERTED flag set
// so we need to release it here
releaseIoSqe(ioSqe);
}
return -1;
}
int IoUringBackend::eb_event_modify_inserted(Event& event, IoSqe* ioSqe) {
// unlink and append
ioSqe->unlink();
submitList_.push_back(*ioSqe);
event.setUserData(ioSqe);
return 0;
} }
int IoUringBackend::submitOne(IoCb* /*unused*/) { int IoUringBackend::submitOne() {
return submitBusyCheck(1, WaitForEventsMode::DONT_WAIT); return submitBusyCheck(1, WaitForEventsMode::DONT_WAIT);
} }
int IoUringBackend::cancelOne(IoCb* ioCb) { int IoUringBackend::cancelOne(IoSqe* ioSqe) {
auto* rentry = static_cast<IoSqe*>(allocIoCb(EventCallback())); auto* rentry = static_cast<IoSqe*>(allocIoSqe(EventCallback()));
if (!rentry) { if (!rentry) {
return 0; return 0;
} }
...@@ -422,13 +1033,13 @@ int IoUringBackend::cancelOne(IoCb* ioCb) { ...@@ -422,13 +1033,13 @@ int IoUringBackend::cancelOne(IoCb* ioCb) {
auto* sqe = get_sqe(); auto* sqe = get_sqe();
CHECK(sqe); CHECK(sqe);
rentry->prepCancel(sqe, ioCb); // prev entry rentry->prepCancel(sqe, ioSqe); // prev entry
int ret = submitBusyCheck(1, WaitForEventsMode::DONT_WAIT); int ret = submitBusyCheck(1, WaitForEventsMode::DONT_WAIT);
if (ret < 0) { if (ret < 0) {
// release the sqe // release the sqe
releaseIoCb(rentry); releaseIoSqe(rentry);
} }
return ret; return ret;
...@@ -518,21 +1129,21 @@ int IoUringBackend::submitBusyCheck(int num, WaitForEventsMode waitForEvents) { ...@@ -518,21 +1129,21 @@ int IoUringBackend::submitBusyCheck(int num, WaitForEventsMode waitForEvents) {
} }
size_t IoUringBackend::submitList( size_t IoUringBackend::submitList(
IoCbList& ioCbs, IoSqeList& ioSqes,
WaitForEventsMode waitForEvents) { WaitForEventsMode waitForEvents) {
int i = 0; int i = 0;
size_t ret = 0; size_t ret = 0;
while (!ioCbs.empty()) { while (!ioSqes.empty()) {
auto* entry = &ioCbs.front(); auto* entry = &ioSqes.front();
ioCbs.pop_front(); ioSqes.pop_front();
auto* sqe = get_sqe(); auto* sqe = get_sqe();
CHECK(sqe); // this should not happen CHECK(sqe); // this should not happen
entry->processSubmit(sqe); entry->processSubmit(sqe);
i++; i++;
if (ioCbs.empty()) { if (ioSqes.empty()) {
int num = submitBusyCheck(i, waitForEvents); int num = submitBusyCheck(i, waitForEvents);
CHECK_EQ(num, i); CHECK_EQ(num, i);
ret += i; ret += i;
...@@ -558,11 +1169,11 @@ void IoUringBackend::queueRead( ...@@ -558,11 +1169,11 @@ void IoUringBackend::queueRead(
struct iovec iov { struct iovec iov {
buf, nbytes buf, nbytes
}; };
auto* iocb = new ReadIoSqe(this, fd, &iov, offset, std::move(cb)); auto* ioSqe = new ReadIoSqe(this, fd, &iov, offset, std::move(cb));
iocb->backendCb_ = processFileOpCB; ioSqe->backendCb_ = processFileOpCB;
incNumIoCbInUse(); incNumIoSqeInUse();
submitImmediateIoCb(*iocb); submitImmediateIoSqe(*ioSqe);
} }
void IoUringBackend::queueWrite( void IoUringBackend::queueWrite(
...@@ -574,11 +1185,11 @@ void IoUringBackend::queueWrite( ...@@ -574,11 +1185,11 @@ void IoUringBackend::queueWrite(
struct iovec iov { struct iovec iov {
const_cast<void*>(buf), nbytes const_cast<void*>(buf), nbytes
}; };
auto* iocb = new WriteIoSqe(this, fd, &iov, offset, std::move(cb)); auto* ioSqe = new WriteIoSqe(this, fd, &iov, offset, std::move(cb));
iocb->backendCb_ = processFileOpCB; ioSqe->backendCb_ = processFileOpCB;
incNumIoCbInUse(); incNumIoSqeInUse();
submitImmediateIoCb(*iocb); submitImmediateIoSqe(*ioSqe);
} }
void IoUringBackend::queueReadv( void IoUringBackend::queueReadv(
...@@ -586,11 +1197,11 @@ void IoUringBackend::queueReadv( ...@@ -586,11 +1197,11 @@ void IoUringBackend::queueReadv(
Range<const struct iovec*> iovecs, Range<const struct iovec*> iovecs,
off_t offset, off_t offset,
FileOpCallback&& cb) { FileOpCallback&& cb) {
auto* iocb = new ReadvIoSqe(this, fd, iovecs, offset, std::move(cb)); auto* ioSqe = new ReadvIoSqe(this, fd, iovecs, offset, std::move(cb));
iocb->backendCb_ = processFileOpCB; ioSqe->backendCb_ = processFileOpCB;
incNumIoCbInUse(); incNumIoSqeInUse();
submitImmediateIoCb(*iocb); submitImmediateIoSqe(*ioSqe);
} }
void IoUringBackend::queueWritev( void IoUringBackend::queueWritev(
...@@ -598,11 +1209,11 @@ void IoUringBackend::queueWritev( ...@@ -598,11 +1209,11 @@ void IoUringBackend::queueWritev(
Range<const struct iovec*> iovecs, Range<const struct iovec*> iovecs,
off_t offset, off_t offset,
FileOpCallback&& cb) { FileOpCallback&& cb) {
auto* iocb = new WritevIoSqe(this, fd, iovecs, offset, std::move(cb)); auto* ioSqe = new WritevIoSqe(this, fd, iovecs, offset, std::move(cb));
iocb->backendCb_ = processFileOpCB; ioSqe->backendCb_ = processFileOpCB;
incNumIoCbInUse(); incNumIoSqeInUse();
submitImmediateIoCb(*iocb); submitImmediateIoSqe(*ioSqe);
} }
void IoUringBackend::queueFsync(int fd, FileOpCallback&& cb) { void IoUringBackend::queueFsync(int fd, FileOpCallback&& cb) {
...@@ -614,18 +1225,18 @@ void IoUringBackend::queueFdatasync(int fd, FileOpCallback&& cb) { ...@@ -614,18 +1225,18 @@ void IoUringBackend::queueFdatasync(int fd, FileOpCallback&& cb) {
} }
void IoUringBackend::queueFsync(int fd, FSyncFlags flags, FileOpCallback&& cb) { void IoUringBackend::queueFsync(int fd, FSyncFlags flags, FileOpCallback&& cb) {
auto* iocb = new FSyncIoSqe(this, fd, flags, std::move(cb)); auto* ioSqe = new FSyncIoSqe(this, fd, flags, std::move(cb));
iocb->backendCb_ = processFileOpCB; ioSqe->backendCb_ = processFileOpCB;
incNumIoCbInUse(); incNumIoSqeInUse();
submitImmediateIoCb(*iocb); submitImmediateIoSqe(*ioSqe);
} }
void IoUringBackend::processFileOp(IoCb* ioCb, int64_t res) noexcept { void IoUringBackend::processFileOp(IoSqe* sqe, int64_t res) noexcept {
auto* ioSqe = reinterpret_cast<FileOpIoSqe*>(ioCb); auto* ioSqe = reinterpret_cast<FileOpIoSqe*>(sqe);
// save the res // save the res
ioSqe->res_ = res; ioSqe->res_ = res;
activeEvents_.push_back(*ioCb); activeEvents_.push_back(*ioSqe);
numInsertedEvents_--; numInsertedEvents_--;
} }
......
...@@ -17,31 +17,147 @@ ...@@ -17,31 +17,147 @@
#pragma once #pragma once
#include <liburing.h> #include <liburing.h>
#include <poll.h>
#include <sys/types.h>
#include <chrono>
#include <map>
#include <set>
#include <vector>
#include <boost/intrusive/list.hpp>
#include <boost/intrusive/slist.hpp>
#include <glog/logging.h>
#include <folly/CPortability.h>
#include <folly/CppAttributes.h>
#include <folly/Function.h> #include <folly/Function.h>
#include <folly/Range.h> #include <folly/Range.h>
#include <folly/experimental/io/PollIoBackend.h> #include <folly/io/async/EventBaseBackendBase.h>
#include <folly/portability/Asm.h> #include <folly/portability/Asm.h>
#include <folly/small_vector.h> #include <folly/small_vector.h>
#include <glog/logging.h>
namespace folly { namespace folly {
class IoUringBackend : public PollIoBackend { class IoUringBackend : public EventBaseBackendBase {
public: public:
class FOLLY_EXPORT NotAvailable : public std::runtime_error { class FOLLY_EXPORT NotAvailable : public std::runtime_error {
public: public:
using std::runtime_error::runtime_error; using std::runtime_error::runtime_error;
}; };
struct Options {
enum Flags {
POLL_SQ = 0x1,
POLL_CQ = 0x2,
POLL_SQ_IMMEDIATE_IO = 0x4, // do not enqueue I/O operations
};
Options() = default;
Options& setCapacity(size_t v) {
capacity = v;
return *this;
}
Options& setMaxSubmit(size_t v) {
maxSubmit = v;
return *this;
}
Options& setMaxGet(size_t v) {
maxGet = v;
return *this;
}
Options& setUseRegisteredFds(bool v) {
useRegisteredFds = v;
return *this;
}
Options& setFlags(uint32_t v) {
flags = v;
return *this;
}
Options& setSQIdle(std::chrono::milliseconds v) {
sqIdle = v;
return *this;
}
Options& setCQIdle(std::chrono::milliseconds v) {
cqIdle = v;
return *this;
}
Options& setSQCpu(uint32_t v) {
sqCpu = v;
return *this;
}
Options& setSQGroupName(const std::string& v) {
sqGroupName = v;
return *this;
}
Options& setSQGroupNumThreads(size_t v) {
sqGroupNumThreads = v;
return *this;
}
size_t capacity{0};
size_t maxSubmit{128};
size_t maxGet{std::numeric_limits<size_t>::max()};
bool useRegisteredFds{false};
uint32_t flags{0};
std::chrono::milliseconds sqIdle{0};
std::chrono::milliseconds cqIdle{0};
uint32_t sqCpu{0};
std::string sqGroupName;
size_t sqGroupNumThreads{1};
};
explicit IoUringBackend(Options options); explicit IoUringBackend(Options options);
~IoUringBackend() override; ~IoUringBackend() override;
// from EventBaseBackendBase
event_base* getEventBase() override { return nullptr; }
int eb_event_base_loop(int flags) override;
int eb_event_base_loopbreak() override;
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; }
// returns true if the current Linux kernel version // returns true if the current Linux kernel version
// supports the io_uring backend // supports the io_uring backend
static bool isAvailable(); static bool isAvailable();
struct FdRegistrationRecord : public boost::intrusive::slist_base_hook<
boost::intrusive::cache_last<false>> {
int count_{0};
int fd_{-1};
size_t idx_{0};
};
FdRegistrationRecord* registerFd(int fd) { return fdRegistry_.alloc(fd); }
bool unregisterFd(FdRegistrationRecord* rec) { return fdRegistry_.free(rec); }
// CQ poll mode loop callback // CQ poll mode loop callback
using CQPollLoopCallback = folly::Function<void()>; using CQPollLoopCallback = folly::Function<void()>;
...@@ -49,15 +165,6 @@ class IoUringBackend : public PollIoBackend { ...@@ -49,15 +165,6 @@ class IoUringBackend : public PollIoBackend {
cqPollLoopCallback_ = std::move(cb); cqPollLoopCallback_ = std::move(cb);
} }
// from PollIoBackend
FdRegistrationRecord* registerFd(int fd) override {
return fdRegistry_.alloc(fd);
}
bool unregisterFd(FdRegistrationRecord* rec) override {
return fdRegistry_.free(rec);
}
// read/write/fsync/fdatasync file operation callback // read/write/fsync/fdatasync file operation callback
// int param is the io_uring_cqe res field // int param is the io_uring_cqe res field
// i.e. the result of the file operation // i.e. the result of the file operation
...@@ -98,6 +205,153 @@ class IoUringBackend : public PollIoBackend { ...@@ -98,6 +205,153 @@ class IoUringBackend : public PollIoBackend {
void queueFdatasync(int fd, FileOpCallback&& cb); void queueFdatasync(int fd, FileOpCallback&& cb);
protected: protected:
enum class WaitForEventsMode { WAIT, DONT_WAIT };
struct TimerEntry {
explicit TimerEntry(Event* event) : event_(event) {}
TimerEntry(Event* event, const struct timeval& timeout);
Event* event_{nullptr};
std::chrono::time_point<std::chrono::steady_clock> expireTime_;
bool operator==(const TimerEntry& other) { return event_ == other.event_; }
std::chrono::microseconds getRemainingTime(
std::chrono::steady_clock::time_point now) const {
if (expireTime_ > now) {
return std::chrono::duration_cast<std::chrono::microseconds>(
expireTime_ - now);
}
return std::chrono::microseconds(0);
}
static bool isExpired(
const std::chrono::time_point<std::chrono::steady_clock>& timestamp,
std::chrono::steady_clock::time_point now) {
return (now >= timestamp);
}
void setExpireTime(
const struct timeval& timeout,
std::chrono::steady_clock::time_point now) {
uint64_t us = static_cast<uint64_t>(timeout.tv_sec) *
static_cast<uint64_t>(1000000) +
static_cast<uint64_t>(timeout.tv_usec);
expireTime_ = now + std::chrono::microseconds(us);
}
};
class SocketPair {
public:
SocketPair();
SocketPair(const SocketPair&) = delete;
SocketPair& operator=(const SocketPair&) = delete;
~SocketPair();
int readFd() const { return fds_[1]; }
int writeFd() const { return fds_[0]; }
private:
std::array<int, 2> fds_{{-1, -1}};
};
static uint32_t getPollFlags(short events) {
uint32_t ret = 0;
if (events & EV_READ) {
ret |= POLLIN;
}
if (events & EV_WRITE) {
ret |= POLLOUT;
}
return ret;
}
static short getPollEvents(uint32_t flags, short events) {
short ret = 0;
if (flags & POLLIN) {
ret |= EV_READ;
}
if (flags & POLLOUT) {
ret |= EV_WRITE;
}
if (flags & (POLLERR | POLLHUP)) {
ret |= (EV_READ | EV_WRITE);
}
ret &= events;
return ret;
}
// timer processing
bool addTimerFd();
void scheduleTimeout();
void scheduleTimeout(const std::chrono::microseconds& us);
void addTimerEvent(Event& event, const struct timeval* timeout);
void removeTimerEvent(Event& event);
size_t processTimers();
void setProcessTimers() { processTimers_ = true; }
size_t processActiveEvents();
struct IoSqe;
static void
processPollIoSqe(IoUringBackend* backend, IoSqe* ioSqe, int64_t res) {
backend->processPollIo(ioSqe, res);
}
static void
processTimerIoSqe(IoUringBackend* backend, IoSqe* /*sqe*/, int64_t /*res*/) {
backend->setProcessTimers();
}
// signal handling
void addSignalEvent(Event& event);
void removeSignalEvent(Event& event);
bool addSignalFds();
size_t processSignals();
FOLLY_ALWAYS_INLINE void setProcessSignals() { processSignals_ = true; }
static void processSignalReadIoSqe(
IoUringBackend* backend,
IoSqe* /*sqe*/,
int64_t /*res*/) {
backend->setProcessSignals();
}
void processPollIo(IoSqe* ioSqe, int64_t res) noexcept;
IoSqe* FOLLY_NULLABLE allocIoSqe(const EventCallback& cb);
void releaseIoSqe(IoSqe* aioIoSqe);
void incNumIoSqeInUse() { numIoSqeInUse_++; }
// submit immediate if POLL_SQ | POLL_SQ_IMMEDIATE_IO flags are set
void submitImmediateIoSqe(IoSqe& ioSqe) {
if (options_.flags &
(Options::Flags::POLL_SQ | Options::Flags::POLL_SQ_IMMEDIATE_IO)) {
IoSqeList s;
s.push_back(ioSqe);
numInsertedEvents_++;
submitList(s, WaitForEventsMode::DONT_WAIT);
} else {
submitList_.push_back(ioSqe);
numInsertedEvents_++;
}
}
int eb_event_modify_inserted(Event& event, IoSqe* ioSqe);
FOLLY_ALWAYS_INLINE size_t numIoSqeInUse() const { return numIoSqeInUse_; }
struct FdRegistry { struct FdRegistry {
FdRegistry() = delete; FdRegistry() = delete;
FdRegistry(struct io_uring& ioRing, size_t n); FdRegistry(struct io_uring& ioRing, size_t n);
...@@ -118,28 +372,41 @@ class IoUringBackend : public PollIoBackend { ...@@ -118,28 +372,41 @@ class IoUringBackend : public PollIoBackend {
free_; free_;
}; };
// from PollIoBackend struct io_uring_sqe* allocSubmissionEntry() {
void* allocSubmissionEntry() override; return get_sqe();
int getActiveEvents(WaitForEventsMode waitForEvents) override; }
size_t submitList(IoCbList& ioCbs, WaitForEventsMode waitForEvents) override;
int submitOne(IoCb* ioCb) override;
int cancelOne(IoCb* ioCb) override;
int submitBusyCheck(int num, WaitForEventsMode waitForEvents);
struct IoSqe : public PollIoBackend::IoCb { struct IoSqe
: public boost::intrusive::list_base_hook<
boost::intrusive::link_mode<boost::intrusive::auto_unlink>> {
using BackendCb = void(IoUringBackend*, IoSqe*, int64_t);
explicit IoSqe( explicit IoSqe(
PollIoBackend* backend = nullptr, IoUringBackend* backend = nullptr,
bool poolAlloc = false, bool poolAlloc = false,
bool persist = false) bool persist = false)
: PollIoBackend::IoCb(backend, poolAlloc, persist) {} : backend_(backend), poolAlloc_(poolAlloc), persist_(persist) {}
~IoSqe() override = default; virtual ~IoSqe() = default;
IoUringBackend* backend_;
BackendCb* backendCb_{nullptr};
const bool poolAlloc_;
const bool persist_;
Event* event_{nullptr};
FdRegistrationRecord* fdRecord_{nullptr};
size_t useCount_{0};
FOLLY_ALWAYS_INLINE void resetEvent() {
// remove it from the list
unlink();
if (event_) {
event_->setUserData(nullptr);
event_ = nullptr;
}
}
void processSubmit(void* entry) override { virtual void processSubmit(struct io_uring_sqe* sqe) {
auto* ev = event_->getEvent(); auto* ev = event_->getEvent();
if (ev) { if (ev) {
struct io_uring_sqe* sqe =
reinterpret_cast<struct io_uring_sqe*>(entry);
const auto& cb = event_->getCallback(); const auto& cb = event_->getCallback();
switch (cb.type_) { switch (cb.type_) {
case EventCallback::Type::TYPE_NONE: case EventCallback::Type::TYPE_NONE:
...@@ -177,10 +444,77 @@ class IoUringBackend : public PollIoBackend { ...@@ -177,10 +444,77 @@ class IoUringBackend : public PollIoBackend {
} }
} }
void prepPollAdd(void* entry, int fd, uint32_t events, bool registerFd) virtual void processActive() {}
override {
CHECK(entry); struct EventCallbackData {
struct io_uring_sqe* sqe = reinterpret_cast<struct io_uring_sqe*>(entry); EventCallback::Type type_{EventCallback::Type::TYPE_NONE};
union {
EventReadCallback::IoVec* ioVec_;
EventRecvmsgCallback::MsgHdr* msgHdr_;
};
void set(EventReadCallback::IoVec* ioVec) {
type_ = EventCallback::Type::TYPE_READ;
ioVec_ = ioVec;
}
void set(EventRecvmsgCallback::MsgHdr* msgHdr) {
type_ = EventCallback::Type::TYPE_RECVMSG;
msgHdr_ = msgHdr;
}
void reset() { type_ = EventCallback::Type::TYPE_NONE; }
bool processCb(int res) {
bool ret = false;
switch (type_) {
case EventCallback::Type::TYPE_READ: {
ret = true;
auto cbFunc = ioVec_->cbFunc_;
cbFunc(ioVec_, res);
break;
}
case EventCallback::Type::TYPE_RECVMSG: {
ret = true;
auto cbFunc = msgHdr_->cbFunc_;
cbFunc(msgHdr_, res);
break;
}
case EventCallback::Type::TYPE_NONE:
break;
}
type_ = EventCallback::Type::TYPE_NONE;
return ret;
}
void releaseData() {
switch (type_) {
case EventCallback::Type::TYPE_READ: {
auto freeFunc = ioVec_->freeFunc_;
freeFunc(ioVec_);
break;
}
case EventCallback::Type::TYPE_RECVMSG: {
auto freeFunc = msgHdr_->freeFunc_;
freeFunc(msgHdr_);
break;
}
case EventCallback::Type::TYPE_NONE:
break;
}
type_ = EventCallback::Type::TYPE_NONE;
}
};
EventCallbackData cbData_;
void prepPollAdd(
struct io_uring_sqe* sqe,
int fd,
uint32_t events,
bool registerFd) {
CHECK(sqe);
if (registerFd && !fdRecord_) { if (registerFd && !fdRecord_) {
fdRecord_ = backend_->registerFd(fd); fdRecord_ = backend_->registerFd(fd);
} }
...@@ -195,13 +529,12 @@ class IoUringBackend : public PollIoBackend { ...@@ -195,13 +529,12 @@ class IoUringBackend : public PollIoBackend {
} }
void prepRead( void prepRead(
void* entry, struct io_uring_sqe* sqe,
int fd, int fd,
const struct iovec* iov, const struct iovec* iov,
off_t offset, off_t offset,
bool registerFd) { bool registerFd) {
CHECK(entry); CHECK(sqe);
struct io_uring_sqe* sqe = reinterpret_cast<struct io_uring_sqe*>(entry);
if (registerFd && !fdRecord_) { if (registerFd && !fdRecord_) {
fdRecord_ = backend_->registerFd(fd); fdRecord_ = backend_->registerFd(fd);
} }
...@@ -217,13 +550,12 @@ class IoUringBackend : public PollIoBackend { ...@@ -217,13 +550,12 @@ class IoUringBackend : public PollIoBackend {
} }
void prepWrite( void prepWrite(
void* entry, struct io_uring_sqe* sqe,
int fd, int fd,
const struct iovec* iov, const struct iovec* iov,
off_t offset, off_t offset,
bool registerFd) { bool registerFd) {
CHECK(entry); CHECK(sqe);
struct io_uring_sqe* sqe = reinterpret_cast<struct io_uring_sqe*>(entry);
if (registerFd && !fdRecord_) { if (registerFd && !fdRecord_) {
fdRecord_ = backend_->registerFd(fd); fdRecord_ = backend_->registerFd(fd);
} }
...@@ -238,9 +570,12 @@ class IoUringBackend : public PollIoBackend { ...@@ -238,9 +570,12 @@ class IoUringBackend : public PollIoBackend {
::io_uring_sqe_set_data(sqe, this); ::io_uring_sqe_set_data(sqe, this);
} }
void prepRecvmsg(void* entry, int fd, struct msghdr* msg, bool registerFd) { void prepRecvmsg(
CHECK(entry); struct io_uring_sqe* sqe,
struct io_uring_sqe* sqe = reinterpret_cast<struct io_uring_sqe*>(entry); int fd,
struct msghdr* msg,
bool registerFd) {
CHECK(sqe);
if (registerFd && !fdRecord_) { if (registerFd && !fdRecord_) {
fdRecord_ = backend_->registerFd(fd); fdRecord_ = backend_->registerFd(fd);
} }
...@@ -263,8 +598,11 @@ class IoUringBackend : public PollIoBackend { ...@@ -263,8 +598,11 @@ class IoUringBackend : public PollIoBackend {
} }
}; };
using IoSqeList = boost::intrusive::
list<IoSqe, boost::intrusive::constant_time_size<false>>;
struct FileOpIoSqe : public IoSqe { struct FileOpIoSqe : public IoSqe {
FileOpIoSqe(PollIoBackend* backend, int fd, FileOpCallback&& cb) FileOpIoSqe(IoUringBackend* backend, int fd, FileOpCallback&& cb)
: IoSqe(backend, false), fd_(fd), cb_(std::move(cb)) {} : IoSqe(backend, false), fd_(fd), cb_(std::move(cb)) {}
~FileOpIoSqe() override = default; ~FileOpIoSqe() override = default;
...@@ -279,7 +617,7 @@ class IoUringBackend : public PollIoBackend { ...@@ -279,7 +617,7 @@ class IoUringBackend : public PollIoBackend {
struct ReadWriteIoSqe : public FileOpIoSqe { struct ReadWriteIoSqe : public FileOpIoSqe {
ReadWriteIoSqe( ReadWriteIoSqe(
PollIoBackend* backend, IoUringBackend* backend,
int fd, int fd,
const struct iovec* iov, const struct iovec* iov,
off_t offset, off_t offset,
...@@ -289,7 +627,7 @@ class IoUringBackend : public PollIoBackend { ...@@ -289,7 +627,7 @@ class IoUringBackend : public PollIoBackend {
offset_(offset) {} offset_(offset) {}
ReadWriteIoSqe( ReadWriteIoSqe(
PollIoBackend* backend, IoUringBackend* backend,
int fd, int fd,
Range<const struct iovec*> iov, Range<const struct iovec*> iov,
off_t offset, off_t offset,
...@@ -310,8 +648,8 @@ class IoUringBackend : public PollIoBackend { ...@@ -310,8 +648,8 @@ class IoUringBackend : public PollIoBackend {
~ReadIoSqe() override = default; ~ReadIoSqe() override = default;
void processSubmit(void* entry) override { void processSubmit(struct io_uring_sqe* sqe) override {
prepRead(entry, fd_, iov_.data(), offset_, false); prepRead(sqe, fd_, iov_.data(), offset_, false);
} }
}; };
...@@ -319,8 +657,8 @@ class IoUringBackend : public PollIoBackend { ...@@ -319,8 +657,8 @@ class IoUringBackend : public PollIoBackend {
using ReadWriteIoSqe::ReadWriteIoSqe; using ReadWriteIoSqe::ReadWriteIoSqe;
~WriteIoSqe() override = default; ~WriteIoSqe() override = default;
void processSubmit(void* entry) override { void processSubmit(struct io_uring_sqe* sqe) override {
prepWrite(entry, fd_, iov_.data(), offset_, false); prepWrite(sqe, fd_, iov_.data(), offset_, false);
} }
}; };
...@@ -329,8 +667,7 @@ class IoUringBackend : public PollIoBackend { ...@@ -329,8 +667,7 @@ class IoUringBackend : public PollIoBackend {
~ReadvIoSqe() override = default; ~ReadvIoSqe() override = default;
void processSubmit(void* entry) override { void processSubmit(struct io_uring_sqe* sqe) override {
struct io_uring_sqe* sqe = reinterpret_cast<struct io_uring_sqe*>(entry);
::io_uring_prep_readv(sqe, fd_, iov_.data(), iov_.size(), offset_); ::io_uring_prep_readv(sqe, fd_, iov_.data(), iov_.size(), offset_);
::io_uring_sqe_set_data(sqe, this); ::io_uring_sqe_set_data(sqe, this);
} }
...@@ -340,8 +677,7 @@ class IoUringBackend : public PollIoBackend { ...@@ -340,8 +677,7 @@ class IoUringBackend : public PollIoBackend {
using ReadWriteIoSqe::ReadWriteIoSqe; using ReadWriteIoSqe::ReadWriteIoSqe;
~WritevIoSqe() override = default; ~WritevIoSqe() override = default;
void processSubmit(void* entry) override { void processSubmit(struct io_uring_sqe* sqe) override {
struct io_uring_sqe* sqe = reinterpret_cast<struct io_uring_sqe*>(entry);
::io_uring_prep_writev(sqe, fd_, iov_.data(), iov_.size(), offset_); ::io_uring_prep_writev(sqe, fd_, iov_.data(), iov_.size(), offset_);
::io_uring_sqe_set_data(sqe, this); ::io_uring_sqe_set_data(sqe, this);
} }
...@@ -354,7 +690,7 @@ class IoUringBackend : public PollIoBackend { ...@@ -354,7 +690,7 @@ class IoUringBackend : public PollIoBackend {
struct FSyncIoSqe : public FileOpIoSqe { struct FSyncIoSqe : public FileOpIoSqe {
FSyncIoSqe( FSyncIoSqe(
PollIoBackend* backend, IoUringBackend* backend,
int fd, int fd,
FSyncFlags flags, FSyncFlags flags,
FileOpCallback&& cb) FileOpCallback&& cb)
...@@ -362,9 +698,7 @@ class IoUringBackend : public PollIoBackend { ...@@ -362,9 +698,7 @@ class IoUringBackend : public PollIoBackend {
~FSyncIoSqe() override = default; ~FSyncIoSqe() override = default;
void processSubmit(void* entry) override { void processSubmit(struct io_uring_sqe* sqe) override {
struct io_uring_sqe* sqe = reinterpret_cast<struct io_uring_sqe*>(entry);
unsigned int fsyncFlags = 0; unsigned int fsyncFlags = 0;
switch (flags_) { switch (flags_) {
case FSyncFlags::FLAGS_FSYNC: case FSyncFlags::FLAGS_FSYNC:
...@@ -382,18 +716,26 @@ class IoUringBackend : public PollIoBackend { ...@@ -382,18 +716,26 @@ class IoUringBackend : public PollIoBackend {
FSyncFlags flags_; FSyncFlags flags_;
}; };
int getActiveEvents(WaitForEventsMode waitForEvents);
size_t submitList(IoSqeList& ioSqes, WaitForEventsMode waitForEvents);
int submitOne();
int cancelOne(IoSqe* ioSqe);
int submitBusyCheck(int num, WaitForEventsMode waitForEvents);
void queueFsync(int fd, FSyncFlags flags, FileOpCallback&& cb); void queueFsync(int fd, FSyncFlags flags, FileOpCallback&& cb);
void processFileOp(IoCb* ioCb, int64_t res) noexcept; void processFileOp(IoSqe* ioSqe, int64_t res) noexcept;
static void processFileOpCB(PollIoBackend* backend, IoCb* ioCb, int64_t res) { static void
static_cast<IoUringBackend*>(backend)->processFileOp(ioCb, res); processFileOpCB(IoUringBackend* backend, IoSqe* ioSqe, int64_t res) {
static_cast<IoUringBackend*>(backend)->processFileOp(ioSqe, res);
} }
PollIoBackend::IoCb* allocNewIoCb(const EventCallback& /*cb*/) override { IoUringBackend::IoSqe* allocNewIoSqe(const EventCallback& /*cb*/) {
// allow pool alloc if numIoCbInUse_ < numEntries_ // allow pool alloc if numIoSqeInUse_ < numEntries_
auto* ret = new IoSqe(this, numIoCbInUse_ < numEntries_); auto* ret = new IoSqe(this, numIoSqeInUse_ < numEntries_);
ret->backendCb_ = PollIoBackend::processPollIoCb; ret->backendCb_ = IoUringBackend::processPollIoSqe;
return ret; return ret;
} }
...@@ -415,6 +757,36 @@ class IoUringBackend : public PollIoBackend { ...@@ -415,6 +757,36 @@ class IoUringBackend : public PollIoBackend {
size_t submit_internal(); size_t submit_internal();
Options options_;
size_t numEntries_;
std::unique_ptr<IoSqe> timerEntry_;
std::unique_ptr<IoSqe> signalReadEntry_;
IoSqeList freeList_;
// timer related
int timerFd_{-1};
bool timerChanged_{false};
std::map<std::chrono::steady_clock::time_point, std::vector<TimerEntry>>
timers_;
std::map<Event*, std::chrono::steady_clock::time_point> eventToTimers_;
// signal related
SocketPair signalFds_;
std::map<int, std::set<Event*>> signals_;
// submit
IoSqeList submitList_;
// loop related
bool loopBreak_{false};
bool shuttingDown_{false};
bool processTimers_{false};
bool processSignals_{false};
size_t numInsertedEvents_{0};
IoSqeList activeEvents_;
// number of IoSqe instances in use
size_t numIoSqeInUse_{0};
// io_uring related // io_uring related
struct io_uring_params params_; struct io_uring_params params_;
struct io_uring ioRing_; struct io_uring ioRing_;
...@@ -425,4 +797,6 @@ class IoUringBackend : public PollIoBackend { ...@@ -425,4 +797,6 @@ class IoUringBackend : public PollIoBackend {
// every time we poll for a CQE // every time we poll for a CQE
CQPollLoopCallback cqPollLoopCallback_; CQPollLoopCallback cqPollLoopCallback_;
}; };
using PollIoBackend = IoUringBackend;
} // namespace folly } // namespace folly
/*
* 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.
*/
#include <signal.h>
#include <sys/timerfd.h>
#include <atomic>
#include <folly/FileUtil.h>
#include <folly/Likely.h>
#include <folly/SpinLock.h>
#include <folly/experimental/io/PollIoBackend.h>
#include <folly/portability/Sockets.h>
#include <folly/synchronization/CallOnce.h>
#include <glog/logging.h>
extern "C" FOLLY_ATTR_WEAK void eb_poll_loop_pre_hook(uint64_t* call_time);
extern "C" FOLLY_ATTR_WEAK void eb_poll_loop_post_hook(
uint64_t call_time,
int ret);
namespace {
struct SignalRegistry {
struct SigInfo {
struct sigaction sa_ {};
size_t count_{0};
};
using SignalMap = std::map<int, SigInfo>;
constexpr SignalRegistry() {}
~SignalRegistry() {}
void notify(int sig);
void setNotifyFd(int sig, int fd);
// lock protecting the signal map
folly::MicroSpinLock mapLock_ = {0};
std::unique_ptr<SignalMap> map_;
std::atomic<int> notifyFd_{-1};
};
SignalRegistry sSignalRegistry;
void evSigHandler(int sig) {
sSignalRegistry.notify(sig);
}
void SignalRegistry::notify(int sig) {
// use try_lock in case somebody already has the lock
if (mapLock_.try_lock()) {
int fd = notifyFd_.load();
if (fd >= 0) {
uint8_t sigNum = static_cast<uint8_t>(sig);
::write(fd, &sigNum, 1);
}
mapLock_.unlock();
}
}
void SignalRegistry::setNotifyFd(int sig, int fd) {
folly::MSLGuard g(mapLock_);
if (fd >= 0) {
if (!map_) {
map_ = std::make_unique<SignalMap>();
}
// switch the fd
notifyFd_.store(fd);
auto iter = (*map_).find(sig);
if (iter != (*map_).end()) {
iter->second.count_++;
} else {
auto& entry = (*map_)[sig];
entry.count_ = 1;
struct sigaction sa = {};
sa.sa_handler = evSigHandler;
sa.sa_flags |= SA_RESTART;
::sigfillset(&sa.sa_mask);
if (::sigaction(sig, &sa, &entry.sa_) == -1) {
(*map_).erase(sig);
}
}
} else {
notifyFd_.store(fd);
if (map_) {
auto iter = (*map_).find(sig);
if ((iter != (*map_).end()) && (--iter->second.count_ == 0)) {
auto entry = iter->second;
(*map_).erase(iter);
// just restore
::sigaction(sig, &entry.sa_, nullptr);
}
}
}
}
} // namespace
namespace folly {
PollIoBackend::TimerEntry::TimerEntry(
Event* event,
const struct timeval& timeout)
: event_(event) {
setExpireTime(timeout);
}
PollIoBackend::SocketPair::SocketPair() {
if (::socketpair(AF_UNIX, SOCK_STREAM, 0, fds_.data())) {
throw std::runtime_error("socketpair error");
}
// set the sockets to non blocking mode
for (auto fd : fds_) {
auto flags = ::fcntl(fd, F_GETFL, 0);
::fcntl(fd, F_SETFL, flags | O_NONBLOCK);
}
}
PollIoBackend::SocketPair::~SocketPair() {
for (auto fd : fds_) {
if (fd >= 0) {
::close(fd);
}
}
}
PollIoBackend::PollIoBackend(Options options)
: options_(options), numEntries_(options.capacity) {
// create the timer fd
timerFd_ = ::timerfd_create(CLOCK_MONOTONIC, TFD_NONBLOCK | TFD_CLOEXEC);
if (timerFd_ < 0) {
throw std::runtime_error("timerfd_create error");
}
}
PollIoBackend::~PollIoBackend() {
CHECK(!timerEntry_);
CHECK(!signalReadEntry_);
CHECK(freeList_.empty());
::close(timerFd_);
}
bool PollIoBackend::addTimerFd() {
auto* entry = allocSubmissionEntry(); // this can be nullptr
timerEntry_->prepPollAdd(entry, timerFd_, POLLIN, true /*registerFd*/);
return (1 == submitOne(timerEntry_.get()));
}
bool PollIoBackend::addSignalFds() {
auto* entry = allocSubmissionEntry(); // this can be nullptr
signalReadEntry_->prepPollAdd(
entry, signalFds_.readFd(), POLLIN, false /*registerFd*/);
return (1 == submitOne(signalReadEntry_.get()));
}
void PollIoBackend::scheduleTimeout() {
if (!timerChanged_) {
return;
}
// reset
timerChanged_ = false;
if (!timers_.empty()) {
auto delta = timers_.begin()->second[0].getRemainingTime();
if (delta.count() < 1000) {
delta = std::chrono::microseconds(1000);
}
scheduleTimeout(delta);
} else {
scheduleTimeout(std::chrono::microseconds(0)); // disable
}
// we do not call addTimerFd() here
// since it has to be added only once, after
// we process a poll callback
}
void PollIoBackend::scheduleTimeout(const std::chrono::microseconds& us) {
struct itimerspec val;
val.it_interval = {0, 0};
val.it_value.tv_sec =
std::chrono::duration_cast<std::chrono::seconds>(us).count();
val.it_value.tv_nsec =
std::chrono::duration_cast<std::chrono::nanoseconds>(us).count() %
1000000000LL;
CHECK_EQ(::timerfd_settime(timerFd_, 0, &val, nullptr), 0);
}
void PollIoBackend::addTimerEvent(Event& event, const struct timeval* timeout) {
// first try to remove if already existing
auto iter1 = eventToTimers_.find(&event);
if (iter1 != eventToTimers_.end()) {
// no neeed to remove it from eventToTimers_
auto expireTime = iter1->second;
auto iter2 = timers_.find(expireTime);
for (auto iter = iter2->second.begin(), last = iter2->second.end();
iter != last;
++iter) {
if (iter->event_ == &event) {
iter2->second.erase(iter);
break;
}
}
if (iter2->second.empty()) {
timers_.erase(iter2);
}
}
TimerEntry entry(&event, *timeout);
if (!timerChanged_) {
timerChanged_ =
timers_.empty() || (entry.expireTime_ < timers_.begin()->first);
}
timers_[entry.expireTime_].push_back(entry);
eventToTimers_[&event] = entry.expireTime_;
}
void PollIoBackend::removeTimerEvent(Event& event) {
auto iter1 = eventToTimers_.find(&event);
CHECK(iter1 != eventToTimers_.end());
auto expireTime = iter1->second;
eventToTimers_.erase(iter1);
auto iter2 = timers_.find(expireTime);
CHECK(iter2 != timers_.end());
for (auto iter = iter2->second.begin(), last = iter2->second.end();
iter != last;
++iter) {
if (iter->event_ == &event) {
iter2->second.erase(iter);
break;
}
}
if (iter2->second.empty()) {
if (!timerChanged_) {
timerChanged_ = (iter2 == timers_.begin());
}
timers_.erase(iter2);
}
}
size_t PollIoBackend::processTimers() {
size_t ret = 0;
uint64_t data = 0;
// this can fail with but it is OK since the fd
// will still be readable
folly::readNoInt(timerFd_, &data, sizeof(data));
auto now = std::chrono::steady_clock::now();
while (!timers_.empty() && (now >= timers_.begin()->first)) {
if (!timerChanged_) {
timerChanged_ = true;
}
auto vec = std::move(timers_.begin()->second);
timers_.erase(timers_.begin());
for (auto& entry : vec) {
ret++;
eventToTimers_.erase(entry.event_);
auto* ev = entry.event_->getEvent();
ev->ev_res = EV_TIMEOUT;
event_ref_flags(ev).get() = EVLIST_INIT;
(*event_ref_callback(ev))((int)ev->ev_fd, ev->ev_res, event_ref_arg(ev));
}
}
return ret;
}
void PollIoBackend::addSignalEvent(Event& event) {
auto* ev = event.getEvent();
signals_[ev->ev_fd].insert(&event);
// we pass the write fd for notifications
sSignalRegistry.setNotifyFd(ev->ev_fd, signalFds_.writeFd());
}
void PollIoBackend::removeSignalEvent(Event& event) {
auto* ev = event.getEvent();
auto iter = signals_.find(ev->ev_fd);
if (iter != signals_.end()) {
sSignalRegistry.setNotifyFd(ev->ev_fd, -1);
}
}
size_t PollIoBackend::processSignals() {
size_t ret = 0;
static constexpr auto kNumEntries = NSIG * 2;
static_assert(
NSIG < 256, "Use a different data type to cover all the signal values");
std::array<bool, NSIG> processed{};
std::array<uint8_t, kNumEntries> signals;
ssize_t num =
folly::readNoInt(signalFds_.readFd(), signals.data(), signals.size());
for (ssize_t i = 0; i < num; i++) {
int signum = static_cast<int>(signals[i]);
if ((signum >= 0) && (signum < static_cast<int>(processed.size())) &&
!processed[signum]) {
processed[signum] = true;
auto iter = signals_.find(signum);
if (iter != signals_.end()) {
auto& set = iter->second;
for (auto& event : set) {
auto* ev = event->getEvent();
ev->ev_res = 0;
event_ref_flags(ev) |= EVLIST_ACTIVE;
(*event_ref_callback(ev))(
(int)ev->ev_fd, ev->ev_res, event_ref_arg(ev));
event_ref_flags(ev) &= ~EVLIST_ACTIVE;
}
}
}
}
// add the signal fd(s) back
addSignalFds();
return ret;
}
PollIoBackend::IoCb* PollIoBackend::allocIoCb(const EventCallback& cb) {
// try to allocate from the pool first
if ((cb.type_ == EventCallback::Type::TYPE_NONE) && (!freeList_.empty())) {
auto* ret = &freeList_.front();
freeList_.pop_front();
numIoCbInUse_++;
return ret;
}
// alloc a new IoCb
auto* ret = allocNewIoCb(cb);
if (FOLLY_LIKELY(!!ret)) {
numIoCbInUse_++;
}
return ret;
}
void PollIoBackend::releaseIoCb(PollIoBackend::IoCb* aioIoCb) {
CHECK_GT(numIoCbInUse_, 0);
aioIoCb->cbData_.releaseData();
// unregister the file descriptor record
if (aioIoCb->fdRecord_) {
unregisterFd(aioIoCb->fdRecord_);
aioIoCb->fdRecord_ = nullptr;
}
if (FOLLY_LIKELY(aioIoCb->poolAlloc_)) {
numIoCbInUse_--;
aioIoCb->event_ = nullptr;
freeList_.push_front(*aioIoCb);
} else {
if (!aioIoCb->persist_) {
numIoCbInUse_--;
delete aioIoCb;
}
}
}
void PollIoBackend::processPollIo(IoCb* ioCb, int64_t res) noexcept {
auto* ev = ioCb->event_ ? (ioCb->event_->getEvent()) : nullptr;
if (ev) {
if (~event_ref_flags(ev) & EVLIST_INTERNAL) {
// if this is not a persistent event
// remove the EVLIST_INSERTED flags
// and dec the numInsertedEvents_
if (~ev->ev_events & EV_PERSIST) {
DCHECK(numInsertedEvents_ > 0);
numInsertedEvents_--;
event_ref_flags(ev) &= ~EVLIST_INSERTED;
}
}
// add it to the active list
event_ref_flags(ev) |= EVLIST_ACTIVE;
ev->ev_res = res;
activeEvents_.push_back(*ioCb);
} else {
releaseIoCb(ioCb);
}
}
size_t PollIoBackend::processActiveEvents() {
size_t ret = 0;
IoCb* ioCb;
while (!activeEvents_.empty() && !loopBreak_) {
bool release = true;
ioCb = &activeEvents_.front();
activeEvents_.pop_front();
ret++;
auto* event = ioCb->event_;
auto* ev = event ? event->getEvent() : nullptr;
if (ev) {
// remove it from the active list
event_ref_flags(ev) &= ~EVLIST_ACTIVE;
bool inserted = (event_ref_flags(ev) & EVLIST_INSERTED);
// prevent the callback from freeing the aioIoCb
ioCb->useCount_++;
if (!ioCb->cbData_.processCb(ev->ev_res)) {
// adjust the ev_res for the poll case
ev->ev_res = getPollEvents(ev->ev_res, ev->ev_events);
// handle spurious poll events that return 0
// this can happen during high load on process startup
if (ev->ev_res) {
(*event_ref_callback(ev))(
(int)ev->ev_fd, ev->ev_res, event_ref_arg(ev));
}
}
// get the event again
event = ioCb->event_;
ev = event ? event->getEvent() : nullptr;
if (ev && inserted && event_ref_flags(ev) & EVLIST_INSERTED &&
!shuttingDown_) {
release = false;
eb_event_modify_inserted(*event, ioCb);
}
ioCb->useCount_--;
} else {
ioCb->processActive();
}
if (release) {
releaseIoCb(ioCb);
}
}
return ret;
}
int PollIoBackend::eb_event_base_loop(int flags) {
// schedule the timers
bool done = false;
auto waitForEvents = (flags & EVLOOP_NONBLOCK) ? WaitForEventsMode::DONT_WAIT
: WaitForEventsMode::WAIT;
while (!done) {
scheduleTimeout();
// check if we need to break here
if (loopBreak_) {
loopBreak_ = false;
break;
}
submitList(submitList_, waitForEvents);
if (!numInsertedEvents_ && timers_.empty() && signals_.empty()) {
return 1;
}
uint64_t call_time = 0;
if (eb_poll_loop_pre_hook) {
eb_poll_loop_pre_hook(&call_time);
}
// do not wait for events if EVLOOP_NONBLOCK is set
int ret = getActiveEvents(waitForEvents);
if (eb_poll_loop_post_hook) {
eb_poll_loop_post_hook(call_time, ret);
}
size_t numProcessedTimers = 0;
// save the processTimers_
// this means we've received a notification
// and we need to add the timer fd back
bool processTimersFlag = processTimers_;
if (processTimers_ && !loopBreak_) {
numProcessedTimers = processTimers();
processTimers_ = false;
}
size_t numProcessedSignals = 0;
if (processSignals_ && !loopBreak_) {
numProcessedSignals = processSignals();
processSignals_ = false;
}
if (!activeEvents_.empty() && !loopBreak_) {
processActiveEvents();
if (flags & EVLOOP_ONCE) {
done = true;
}
} else if (flags & EVLOOP_NONBLOCK) {
if (signals_.empty()) {
done = true;
}
}
if (!done && (numProcessedTimers || numProcessedSignals) &&
(flags & EVLOOP_ONCE)) {
done = true;
}
if (processTimersFlag) {
addTimerFd();
}
}
return 0;
}
int PollIoBackend::eb_event_base_loopbreak() {
loopBreak_ = true;
return 0;
}
int PollIoBackend::eb_event_add(Event& event, const struct timeval* timeout) {
auto* ev = event.getEvent();
CHECK(ev);
CHECK(!(event_ref_flags(ev) & ~EVLIST_ALL));
// we do not support read/write timeouts
if (timeout) {
event_ref_flags(ev) |= EVLIST_TIMEOUT;
addTimerEvent(event, timeout);
return 0;
}
if (ev->ev_events & EV_SIGNAL) {
event_ref_flags(ev) |= EVLIST_INSERTED;
addSignalEvent(event);
return 0;
}
if ((ev->ev_events & (EV_READ | EV_WRITE)) &&
!(event_ref_flags(ev) & (EVLIST_INSERTED | EVLIST_ACTIVE))) {
auto* iocb = allocIoCb(event.getCallback());
CHECK(iocb);
iocb->event_ = &event;
// just append it
submitList_.push_back(*iocb);
if (~event_ref_flags(ev) & EVLIST_INTERNAL) {
numInsertedEvents_++;
}
event_ref_flags(ev) |= EVLIST_INSERTED;
event.setUserData(iocb);
}
return 0;
}
int PollIoBackend::eb_event_del(Event& event) {
if (!event.eb_ev_base()) {
return -1;
}
auto* ev = event.getEvent();
if (event_ref_flags(ev) & EVLIST_TIMEOUT) {
event_ref_flags(ev) &= ~EVLIST_TIMEOUT;
removeTimerEvent(event);
return 1;
}
if (!(event_ref_flags(ev) & (EVLIST_ACTIVE | EVLIST_INSERTED))) {
return -1;
}
if (ev->ev_events & EV_SIGNAL) {
event_ref_flags(ev) &= ~(EVLIST_INSERTED | EVLIST_ACTIVE);
removeSignalEvent(event);
return 0;
}
auto* iocb = reinterpret_cast<IoCb*>(event.getUserData());
bool wasLinked = iocb->is_linked();
iocb->resetEvent();
// if the event is on the active list, we just clear the flags
// and reset the event_ ptr
if (event_ref_flags(ev) & EVLIST_ACTIVE) {
event_ref_flags(ev) &= ~EVLIST_ACTIVE;
}
if (event_ref_flags(ev) & EVLIST_INSERTED) {
event_ref_flags(ev) &= ~EVLIST_INSERTED;
// not in use - we can cancel it
if (!iocb->useCount_ && !wasLinked) {
// io_cancel will attempt to cancel the event. the result is
// EINVAL - usually the event has already been delivered
// EINPROGRESS - cancellation in progress
// EFAULT - bad ctx
// regardless, we want to dec the numInsertedEvents_
// since even if the events get delivered, the event ptr is nullptr
int ret = cancelOne(iocb);
if (ret < 0) {
// release the iocb
releaseIoCb(iocb);
}
} else {
if (!iocb->useCount_) {
releaseIoCb(iocb);
}
}
if (~event_ref_flags(ev) & EVLIST_INTERNAL) {
CHECK_GT(numInsertedEvents_, 0);
numInsertedEvents_--;
}
return 0;
} else {
// we can have an EVLIST_ACTIVE event
// which does not have the EVLIST_INSERTED flag set
// so we need to release it here
releaseIoCb(iocb);
}
return -1;
}
int PollIoBackend::eb_event_modify_inserted(Event& event, IoCb* ioCb) {
// unlink and append
ioCb->unlink();
submitList_.push_back(*ioCb);
event.setUserData(ioCb);
return 0;
}
} // namespace folly
/*
* 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 <poll.h>
#include <sys/types.h>
#include <chrono>
#include <map>
#include <set>
#include <vector>
#include <boost/intrusive/list.hpp>
#include <boost/intrusive/slist.hpp>
#include <folly/CPortability.h>
#include <folly/CppAttributes.h>
#include <folly/Function.h>
#include <folly/io/async/EventBaseBackendBase.h>
namespace folly {
class PollIoBackend : public EventBaseBackendBase {
public:
struct Options {
enum Flags {
POLL_SQ = 0x1,
POLL_CQ = 0x2,
POLL_SQ_IMMEDIATE_IO = 0x4, // do not enqueue I/O operations
};
Options() = default;
Options& setCapacity(size_t v) {
capacity = v;
return *this;
}
Options& setMaxSubmit(size_t v) {
maxSubmit = v;
return *this;
}
Options& setMaxGet(size_t v) {
maxGet = v;
return *this;
}
Options& setUseRegisteredFds(bool v) {
useRegisteredFds = v;
return *this;
}
Options& setFlags(uint32_t v) {
flags = v;
return *this;
}
Options& setSQIdle(std::chrono::milliseconds v) {
sqIdle = v;
return *this;
}
Options& setCQIdle(std::chrono::milliseconds v) {
cqIdle = v;
return *this;
}
Options& setSQCpu(uint32_t v) {
sqCpu = v;
return *this;
}
Options& setSQGroupName(const std::string& v) {
sqGroupName = v;
return *this;
}
Options& setSQGroupNumThreads(size_t v) {
sqGroupNumThreads = v;
return *this;
}
size_t capacity{0};
size_t maxSubmit{128};
size_t maxGet{static_cast<size_t>(-1)};
bool useRegisteredFds{false};
uint32_t flags{0};
std::chrono::milliseconds sqIdle{0};
std::chrono::milliseconds cqIdle{0};
uint32_t sqCpu{0};
std::string sqGroupName;
size_t sqGroupNumThreads{1};
};
explicit PollIoBackend(Options options);
~PollIoBackend() override;
// from EventBaseBackendBase
event_base* getEventBase() override { return nullptr; }
int eb_event_base_loop(int flags) override;
int eb_event_base_loopbreak() override;
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};
int fd_{-1};
size_t idx_{0};
};
virtual FdRegistrationRecord* registerFd(int /*fd*/) { return nullptr; }
virtual bool unregisterFd(FdRegistrationRecord* /*rec*/) { return false; }
protected:
enum class WaitForEventsMode { WAIT, DONT_WAIT };
struct IoCb;
struct IoCb
: public boost::intrusive::list_base_hook<
boost::intrusive::link_mode<boost::intrusive::auto_unlink>> {
using BackendCb = void(PollIoBackend*, IoCb*, int64_t);
// persist are entries that do not go through the normal delete path if
// not allocated from a pool
explicit IoCb(
PollIoBackend* backend,
bool poolAlloc = true,
bool persist = false)
: backend_(backend), poolAlloc_(poolAlloc), persist_(persist) {}
virtual ~IoCb() = default;
virtual void processSubmit(void* entry) = 0;
virtual void processActive() {}
PollIoBackend* backend_;
BackendCb* backendCb_{nullptr};
const bool poolAlloc_;
const bool persist_;
Event* event_{nullptr};
FdRegistrationRecord* fdRecord_{nullptr};
size_t useCount_{0};
FOLLY_ALWAYS_INLINE void resetEvent() {
// remove it from the list
unlink();
if (event_) {
event_->setUserData(nullptr);
event_ = nullptr;
}
}
virtual void
prepPollAdd(void* entry, int fd, uint32_t events, bool registerFd) = 0;
struct EventCallbackData {
EventCallback::Type type_{EventCallback::Type::TYPE_NONE};
union {
EventReadCallback::IoVec* ioVec_;
EventRecvmsgCallback::MsgHdr* msgHdr_;
};
void set(EventReadCallback::IoVec* ioVec) {
type_ = EventCallback::Type::TYPE_READ;
ioVec_ = ioVec;
}
void set(EventRecvmsgCallback::MsgHdr* msgHdr) {
type_ = EventCallback::Type::TYPE_RECVMSG;
msgHdr_ = msgHdr;
}
void reset() { type_ = EventCallback::Type::TYPE_NONE; }
bool processCb(int res) {
bool ret = false;
switch (type_) {
case EventCallback::Type::TYPE_READ: {
ret = true;
auto cbFunc = ioVec_->cbFunc_;
cbFunc(ioVec_, res);
break;
}
case EventCallback::Type::TYPE_RECVMSG: {
ret = true;
auto cbFunc = msgHdr_->cbFunc_;
cbFunc(msgHdr_, res);
break;
}
case EventCallback::Type::TYPE_NONE:
break;
}
type_ = EventCallback::Type::TYPE_NONE;
return ret;
}
void releaseData() {
switch (type_) {
case EventCallback::Type::TYPE_READ: {
auto freeFunc = ioVec_->freeFunc_;
freeFunc(ioVec_);
break;
}
case EventCallback::Type::TYPE_RECVMSG: {
auto freeFunc = msgHdr_->freeFunc_;
freeFunc(msgHdr_);
break;
}
case EventCallback::Type::TYPE_NONE:
break;
}
type_ = EventCallback::Type::TYPE_NONE;
}
};
EventCallbackData cbData_;
};
using IoCbList =
boost::intrusive::list<IoCb, boost::intrusive::constant_time_size<false>>;
struct TimerEntry {
explicit TimerEntry(Event* event) : event_(event) {}
TimerEntry(Event* event, const struct timeval& timeout);
Event* event_{nullptr};
std::chrono::time_point<std::chrono::steady_clock> expireTime_;
bool operator==(const TimerEntry& other) { return event_ == other.event_; }
std::chrono::microseconds getRemainingTime() const {
auto now = std::chrono::steady_clock::now();
if (expireTime_ > now) {
return std::chrono::duration_cast<std::chrono::microseconds>(
expireTime_ - now);
}
return std::chrono::microseconds(0);
}
static bool isExpired(
const std::chrono::time_point<std::chrono::steady_clock>& timestamp) {
return (std::chrono::steady_clock::now() >= timestamp);
}
void setExpireTime(const struct timeval& timeout) {
uint64_t us = static_cast<uint64_t>(timeout.tv_sec) *
static_cast<uint64_t>(1000000) +
static_cast<uint64_t>(timeout.tv_usec);
expireTime_ =
std::chrono::steady_clock::now() + std::chrono::microseconds(us);
}
};
class SocketPair {
public:
SocketPair();
SocketPair(const SocketPair&) = delete;
SocketPair& operator=(const SocketPair&) = delete;
~SocketPair();
int readFd() const { return fds_[1]; }
int writeFd() const { return fds_[0]; }
private:
std::array<int, 2> fds_{{-1, -1}};
};
static FOLLY_ALWAYS_INLINE uint32_t getPollFlags(short events) {
uint32_t ret = 0;
if (events & EV_READ) {
ret |= POLLIN;
}
if (events & EV_WRITE) {
ret |= POLLOUT;
}
return ret;
}
static FOLLY_ALWAYS_INLINE short getPollEvents(uint32_t flags, short events) {
short ret = 0;
if (flags & POLLIN) {
ret |= EV_READ;
}
if (flags & POLLOUT) {
ret |= EV_WRITE;
}
if (flags & (POLLERR | POLLHUP)) {
ret |= (EV_READ | EV_WRITE);
}
ret &= events;
return ret;
}
// timer processing
bool addTimerFd();
void scheduleTimeout();
void scheduleTimeout(const std::chrono::microseconds& us);
void addTimerEvent(Event& event, const struct timeval* timeout);
void removeTimerEvent(Event& event);
size_t processTimers();
FOLLY_ALWAYS_INLINE void setProcessTimers() { processTimers_ = true; }
size_t processActiveEvents();
static void processPollIoCb(PollIoBackend* backend, IoCb* ioCb, int64_t res) {
backend->processPollIo(ioCb, res);
}
static void processTimerIoCb(
PollIoBackend* backend,
IoCb* /*unused*/,
int64_t /*unused*/) {
backend->setProcessTimers();
}
// signal handling
void addSignalEvent(Event& event);
void removeSignalEvent(Event& event);
bool addSignalFds();
size_t processSignals();
FOLLY_ALWAYS_INLINE void setProcessSignals() { processSignals_ = true; }
static void processSignalReadIoCb(
PollIoBackend* backend,
IoCb* /*unused*/,
int64_t /*unused*/) {
backend->setProcessSignals();
}
void processPollIo(IoCb* ioCb, int64_t res) noexcept;
IoCb* FOLLY_NULLABLE allocIoCb(const EventCallback& cb);
void releaseIoCb(IoCb* aioIoCb);
void incNumIoCbInUse() { numIoCbInUse_++; }
virtual IoCb* allocNewIoCb(const EventCallback& cb) = 0;
virtual void* allocSubmissionEntry() = 0;
virtual int getActiveEvents(WaitForEventsMode waitForEvents) = 0;
virtual size_t submitList(
IoCbList& ioCbs,
WaitForEventsMode waitForEvents) = 0;
// submit immediate if POLL_SQ | POLL_SQ_IMMEDIATE_IO flags are set
void submitImmediateIoCb(IoCb& iocb) {
if (options_.flags &
(Options::Flags::POLL_SQ | Options::Flags::POLL_SQ_IMMEDIATE_IO)) {
IoCbList s;
s.push_back(iocb);
numInsertedEvents_++;
submitList(s, WaitForEventsMode::DONT_WAIT);
} else {
submitList_.push_back(iocb);
numInsertedEvents_++;
}
}
virtual int submitOne(IoCb* ioCb) = 0;
virtual int cancelOne(IoCb* ioCb) = 0;
int eb_event_modify_inserted(Event& event, IoCb* ioCb);
FOLLY_ALWAYS_INLINE size_t numIoCbInUse() const { return numIoCbInUse_; }
Options options_;
size_t numEntries_;
std::unique_ptr<IoCb> timerEntry_;
std::unique_ptr<IoCb> signalReadEntry_;
IoCbList freeList_;
// timer related
int timerFd_{-1};
bool timerChanged_{false};
std::map<
std::chrono::time_point<std::chrono::steady_clock>,
std::vector<TimerEntry>>
timers_;
std::map<Event*, std::chrono::time_point<std::chrono::steady_clock>>
eventToTimers_;
// signal related
SocketPair signalFds_;
std::map<int, std::set<Event*>> signals_;
// submit
IoCbList submitList_;
// loop related
bool loopBreak_{false};
bool shuttingDown_{false};
bool processTimers_{false};
bool processSignals_{false};
size_t numInsertedEvents_{0};
IoCbList activeEvents_;
// number of IoCb instances in use
size_t numIoCbInUse_{0};
};
} // namespace folly
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