Added second implementation of liburing as discussed in #1761. (#1804)

Motivation:

As outlined in #1761, io_uring is a new async I/O facility on Linux.

This commit includes a second stab at adding this to SwiftNIO.

Modifications:

Added Uring Selector implementation.

Added liburing support shims.

Disabled one assert that trips during normal usage likely due to async nature of poll updates, for discussion

Added shared kernel sqpoll ring support (can be run with normal user privs in 5.13)

Support for both single shot polls (should work all the way back to 5.1 kernels, needs testing) and multishot streaming polls and modifications for polls (scheduled due in 5.13) for slightly better performance (and better impedance match to SwiftNIO usage)

Added extensive debug logs which can be enabled with -D compiler flags (should likely be removed when bringup and testing is complete)

Adjusted tests.

Added documentation.

Result:

Basic liburing support is in place.

Co-authored-by: Johannes Weiss <johannesweiss@apple.com>
This commit is contained in:
Joakim Hassila 2021-04-29 11:40:27 +02:00 committed by GitHub
parent c6fc49a84a
commit 8ea0afb4c4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 1060 additions and 11 deletions

View File

@ -24,6 +24,7 @@
#include <errno.h>
#include <pthread.h>
#include <netinet/ip.h>
#include "liburing_nio.h"
// Some explanation is required here.
//

View File

@ -0,0 +1,38 @@
//===----------------------------------------------------------------------===//
//
// This source file is part of the SwiftNIO open source project
//
// Copyright (c) 2021 Apple Inc. and the SwiftNIO project authors
// Licensed under Apache License v2.0
//
// See LICENSE.txt for license information
// See CONTRIBUTORS.txt for the list of SwiftNIO project authors
//
// SPDX-License-Identifier: Apache-2.0
//
//===----------------------------------------------------------------------===//
#ifndef LIBURING_NIO_H
#define LIBURING_NIO_H
#ifdef __linux__
#ifdef SWIFTNIO_USE_IO_URING
#if __has_include(<liburing.h>)
#include <liburing.h>
#else
#error "SWIFTNIO_USE_IO_URING specified but liburing.h not available. You need to install https://github.com/axboe/liburing."
#endif
// OR in the IOSQE_IO_LINK flag, couldn't access the define from Swift
void CNIOLinux_io_uring_set_link_flag(struct io_uring_sqe *sqe);
// No way I managed to get this even when defining _GNU_SOURCE properly. Argh.
unsigned int CNIOLinux_POLLRDHUP();
#endif
#endif /* __linux__ */
#endif /* LIBURING_NIO_H */

View File

@ -0,0 +1,44 @@
//===----------------------------------------------------------------------===//
//
// This source file is part of the SwiftNIO open source project
//
// Copyright (c) 2021 Apple Inc. and the SwiftNIO project authors
// Licensed under Apache License v2.0
//
// See LICENSE.txt for license information
// See CONTRIBUTORS.txt for the list of SwiftNIO project authors
//
// SPDX-License-Identifier: Apache-2.0
//
//===----------------------------------------------------------------------===//
// Support functions for liburing
// Check if this is needed, copied from shim.c to avoid possible problems due to:
// Xcode's Archive builds with Xcode's Package support struggle with empty .c files
// (https://bugs.swift.org/browse/SR-12939).
void CNIOLinux_i_do_nothing_just_working_around_a_darwin_toolchain_bug2(void) {}
#ifdef __linux__
#ifdef SWIFTNIO_USE_IO_URING
#define _GNU_SOURCE
#include <CNIOLinux.h>
#include <signal.h>
#include <sys/poll.h>
void CNIOLinux_io_uring_set_link_flag(struct io_uring_sqe *sqe)
{
sqe->flags |= IOSQE_IO_LINK;
return;
}
unsigned int CNIOLinux_POLLRDHUP()
{
return POLLRDHUP;
}
#endif
#endif

View File

@ -1095,7 +1095,24 @@ class BaseSocketChannel<SocketType: BaseSocketProtocol>: SelectableChannel, Chan
return readStreamState
}
// This assert needs to be disabled for io_uring, as the io_uring backend does not have the implicit synchronisation between
// modifications to the poll mask and the actual returned events on the completion queue that kqueue and epoll has.
// For kqueue and epoll, there is an implicit synchronisation point such that after a modification of the poll mask has been
// issued, the next call to reap events will be sure to not include events which does not match the new poll mask.
// Specifically for this assert, it means that we will be guaranteed to never receive a POLLIN notification unless there are
// bytes available to read.
// For a fully asynchronous backend like io_uring, there are no such implicit synchronisation point, so after we have
// submitted the asynchronous event to change the poll mask, we may still reap pending asynchronous replies for the old
// poll mask, and thus receive a POLLIN even though we have modified the mask visavi the kernel.
// Which would trigger the assert.
// The only way to avoid that race, would be to use heavy handed synchronisation primitives like IOSQE_IO_DRAIN (basically
// flushing all pending requests and wait for a fake event result to sync up) which would be awful for performance,
// so better skip the assert() for io_uring instead.
#if !SWIFTNIO_USE_IO_URING
assert(readResult == .some)
#endif
if self.lifecycleManager.isActive {
self.pipeline.fireChannelReadComplete0()
}

View File

@ -0,0 +1,534 @@
//===----------------------------------------------------------------------===//
//
// This source file is part of the SwiftNIO open source project
//
// Copyright (c) 2021 Apple Inc. and the SwiftNIO project authors
// Licensed under Apache License v2.0
//
// See LICENSE.txt for license information
// See CONTRIBUTORS.txt for the list of SwiftNIO project authors
//
// SPDX-License-Identifier: Apache-2.0
//
//===----------------------------------------------------------------------===//
#if SWIFTNIO_USE_IO_URING
#if os(Linux)
import CNIOLinux
@usableFromInline
enum CQEEventType: UInt8 {
case poll = 1, pollModify, pollDelete // start with 1 to not get zero bit patterns for stdin
}
internal enum URingError: Error {
case loadFailure
case uringSetupFailure
case uringWaitCqeFailure
}
internal extension TimeAmount {
func kernelTimespec() -> __kernel_timespec {
var ts = __kernel_timespec()
ts.tv_sec = self.nanoseconds / 1_000_000_000
ts.tv_nsec = self.nanoseconds % 1_000_000_000
return ts
}
}
// URingUserData supports (un)packing into an `UInt64` as io_uring has a user_data 64-bit payload which is set in the SQE
// and returned in the CQE. We're using 56 of those 64 bits, 32 for the file descriptor, 16 for a "registration ID" and 8
// for the type of event issued (poll/modify/delete).
@usableFromInline struct URingUserData {
@usableFromInline var fileDescriptor: CInt
@usableFromInline var registrationID: UInt16 // SelectorRegistrationID truncated, only have room for bottom 16 bits (could be expanded to 24 if required)
@usableFromInline var eventType: CQEEventType
@usableFromInline var padding: Int8 // reserved for future use
@inlinable init(registrationID: SelectorRegistrationID, fileDescriptor: CInt, eventType: CQEEventType) {
assert(MemoryLayout<UInt64>.size == MemoryLayout<URingUserData>.size)
self.registrationID = UInt16(truncatingIfNeeded: registrationID.rawValue)
self.fileDescriptor = fileDescriptor
self.eventType = eventType
self.padding = 0
}
@inlinable init(rawValue: UInt64) {
let unpacked = IntegerBitPacking.unpackUInt32UInt16UInt8(rawValue)
self = .init(registrationID: SelectorRegistrationID(rawValue: UInt32(unpacked.1)),
fileDescriptor: CInt(unpacked.0),
eventType: CQEEventType(rawValue:unpacked.2)!)
}
}
extension UInt64 {
init(_ uringUserData: URingUserData) {
let fd = uringUserData.fileDescriptor
let eventType = uringUserData.eventType.rawValue
assert(fd >= 0, "\(fd) is not a valid file descriptor")
assert(eventType >= 0, "\(eventType) is not a valid eventType")
self = IntegerBitPacking.packUInt32UInt16UInt8(UInt32(truncatingIfNeeded: fd),
uringUserData.registrationID,
eventType)
}
}
// These are the events returned up to the selector
internal struct URingEvent {
var fd: CInt
var pollMask: UInt32
var registrationID: UInt16 // we just have the truncated lower 16 bits of the registrationID
var pollCancelled: Bool
init () {
self.fd = -1
self.pollMask = 0
self.registrationID = 0
self.pollCancelled = false
}
}
// This is the key we use for merging events in our internal hashtable
struct FDEventKey: Hashable {
var fileDescriptor: CInt
var registrationID: UInt16 // we just have the truncated lower 16 bits of the registrationID
init(_ f: CInt, _ s: UInt16) {
self.fileDescriptor = f
self.registrationID = s
}
}
final internal class URing {
internal static let POLLIN: CUnsignedInt = numericCast(CNIOLinux.POLLIN)
internal static let POLLOUT: CUnsignedInt = numericCast(CNIOLinux.POLLOUT)
internal static let POLLERR: CUnsignedInt = numericCast(CNIOLinux.POLLERR)
internal static let POLLRDHUP: CUnsignedInt = CNIOLinux_POLLRDHUP() // numericCast(CNIOLinux.POLLRDHUP)
internal static let POLLHUP: CUnsignedInt = numericCast(CNIOLinux.POLLHUP)
internal static let POLLCANCEL: CUnsignedInt = 0xF0000000 // Poll cancelled, need to reregister for singleshot polls
private var ring = io_uring()
private let ringEntries: CUnsignedInt = 8192
private let cqeMaxCount: UInt32 = 8192 // this is the max chunk of CQE we take.
var cqes: UnsafeMutablePointer<UnsafeMutablePointer<io_uring_cqe>?>
var fdEvents = [FDEventKey : UInt32]() // fd, sequence_identifier : merged event_poll_return
var emptyCqe = io_uring_cqe()
var fd: CInt {
return ring.ring_fd
}
static var io_uring_use_multishot_poll: Bool {
#if SWIFTNIO_IO_URING_MULTISHOT
return true
#else
return false
#endif
}
func _dumpCqes(_ header:String, count: Int = 1) {
#if SWIFTNIO_IO_URING_DEBUG_DUMP_CQE
func _debugPrintCQE(_ s: String) {
print("Q [\(NIOThread.current)] " + s)
}
if count < 0 {
return
}
_debugPrintCQE(header + " CQE:s [\(cqes)] - ring flags are [\(ring.flags)]")
for i in 0..<count {
let c = cqes[i]!.pointee
let bitPattern = UInt(bitPattern:io_uring_cqe_get_data(cqes[i]))
let uringUserData = URingUserData(rawValue: UInt64(bitPattern))
_debugPrintCQE("\(i) = fd[\(uringUserData.fileDescriptor)] eventType[\(String(describing:uringUserData.eventType))] registrationID[\(uringUserData.registrationID)] res [\(c.res)] flags [\(c.flags)]")
}
#endif
}
init() {
cqes = UnsafeMutablePointer<UnsafeMutablePointer<io_uring_cqe>?>.allocate(capacity: Int(cqeMaxCount))
cqes.initialize(repeating:&emptyCqe, count:Int(cqeMaxCount))
}
deinit {
cqes.deallocate()
}
internal func io_uring_queue_init() throws -> () {
if (CNIOLinux.io_uring_queue_init(ringEntries, &ring, 0 ) != 0)
{
throw URingError.uringSetupFailure
}
_debugPrint("io_uring_queue_init \(self.ring.ring_fd)")
}
internal func io_uring_queue_exit() {
_debugPrint("io_uring_queue_exit \(self.ring.ring_fd)")
CNIOLinux.io_uring_queue_exit(&ring)
}
// Adopting some retry code from queue.c from liburing with slight
// modifications - we never want to have to handle retries of
// SQE allocation in all places it could possibly occur.
// If the SQ ring is full, we may need to submit IO first
func withSQE<R>(_ body: (UnsafeMutablePointer<io_uring_sqe>?) throws -> R) rethrows -> R
{
// io_uring_submit can fail here due to backpressure from kernel for not reaping CQE:s.
//
// I think we should consider handling that as a fatalError, as fundamentally the ring size is too small
// compared to the amount of events the user tries to push through in a single eventloop tick.
//
// This is mostly a problem for synthetic tests that e.g. do a huge amount of registration modifications.
//
// This is a slight design issue with SwiftNIO in general that should be discussed.
//
while true {
if let sqe = CNIOLinux.io_uring_get_sqe(&ring) {
return try body(sqe)
}
self.io_uring_flush()
}
}
// Ok, this was a bummer - turns out that flushing multiple SQE:s
// can fail midflight and this will actually happen for real when e.g. a socket
// has gone down and we are re-registering polls this means we will silently lose any
// entries after the failed fd. Ouch. Proper approach is to use io_uring_sq_ready() in a loop.
// See: https://github.com/axboe/liburing/issues/309
internal func io_uring_flush() { // When using SQPOLL this is basically a NOP
var waitingSubmissions: UInt32 = 0
var submissionCount = 0
var retval: CInt
waitingSubmissions = CNIOLinux.io_uring_sq_ready(&ring)
loop: while (waitingSubmissions > 0)
{
retval = CNIOLinux.io_uring_submit(&ring)
submissionCount += 1
switch retval {
// We can get -EAGAIN if the CQE queue is full and we get back pressure from
// the kernel to start processing CQE:s. If we break here with unsubmitted
// SQE:s, they will stay pending on the user-level side and be flushed
// to the kernel after we had the opportunity to reap more CQE:s
// In practice it will be at the end of whenReady the next
// time around. Given the async nature, this is fine, we will not
// lose any submissions. We could possibly still get stuck
// trying to get new SQE if the actual SQE queue is full, but
// that would be due to user error in usage IMHO and we should fatalError there.
case -EAGAIN, -EBUSY:
_debugPrint("io_uring_flush io_uring_submit -EBUSY/-EAGAIN waitingSubmissions[\(waitingSubmissions)] submissionCount[\(submissionCount)]. Breaking out and resubmitting later (whenReady() end).")
break loop
// -ENOMEM when there is not enough memory to do internal allocations on the kernel side.
// Right nog we just loop with a sleep trying to buy time, but could also possibly fatalError here.
// See: https://github.com/axboe/liburing/issues/309
case -ENOMEM:
usleep(10_000) // let's not busy loop to give the kernel some time to recover if possible
_debugPrint("io_uring_flush io_uring_submit -ENOMEM \(submissionCount)")
case 0:
_debugPrint("io_uring_flush io_uring_submit submitted 0, so far needed submissionCount[\(submissionCount)] waitingSubmissions[\(waitingSubmissions)] submitted [\(retval)] SQE:s this iteration")
break
case 1...:
_debugPrint("io_uring_flush io_uring_submit needed [\(submissionCount)] submission(s), submitted [\(retval)] SQE:s out of [\(waitingSubmissions)] possible")
break
default: // other errors
fatalError("Unexpected error [\(retval)] from io_uring_submit ")
}
waitingSubmissions = CNIOLinux.io_uring_sq_ready(&ring)
}
}
// we stuff event type into the upper byte, the next 3 bytes gives us the sequence number (16M before wrap) and final 4 bytes are fd.
internal func io_uring_prep_poll_add(fileDescriptor: CInt, pollMask: UInt32, registrationID: SelectorRegistrationID, submitNow: Bool = true, multishot: Bool = true) -> () {
let bitPattern = UInt64(URingUserData(registrationID: registrationID, fileDescriptor: fileDescriptor, eventType:CQEEventType.poll))
let bitpatternAsPointer = UnsafeMutableRawPointer.init(bitPattern: UInt(bitPattern))
_debugPrint("io_uring_prep_poll_add fileDescriptor[\(fileDescriptor)] pollMask[\(pollMask)] bitpatternAsPointer[\(String(describing:bitpatternAsPointer))] submitNow[\(submitNow)] multishot[\(multishot)]")
self.withSQE { sqe in
CNIOLinux.io_uring_prep_poll_add(sqe, fileDescriptor, pollMask)
CNIOLinux.io_uring_sqe_set_data(sqe, bitpatternAsPointer) // must be done after prep_poll_add, otherwise zeroed out.
if multishot {
sqe!.pointee.len |= IORING_POLL_ADD_MULTI; // turn on multishots, set through environment variable
}
}
if submitNow {
self.io_uring_flush()
}
}
internal func io_uring_prep_poll_remove(fileDescriptor: CInt, pollMask: UInt32, registrationID: SelectorRegistrationID, submitNow: Bool = true, link: Bool = false) -> () {
let bitPattern = UInt64(URingUserData(registrationID: registrationID,
fileDescriptor: fileDescriptor,
eventType:CQEEventType.poll))
let userbitPattern = UInt64(URingUserData(registrationID: registrationID,
fileDescriptor: fileDescriptor,
eventType:CQEEventType.pollDelete))
let bitpatternAsPointer = UnsafeMutableRawPointer.init(bitPattern: UInt(bitPattern))
let userBitpatternAsPointer = UnsafeMutableRawPointer.init(bitPattern: UInt(userbitPattern))
_debugPrint("io_uring_prep_poll_remove fileDescriptor[\(fileDescriptor)] pollMask[\(pollMask)] bitpatternAsPointer[\(String(describing:bitpatternAsPointer))] userBitpatternAsPointer[\(String(describing:userBitpatternAsPointer))] submitNow[\(submitNow)] link[\(link)]")
self.withSQE { sqe in
CNIOLinux.io_uring_prep_poll_remove(sqe, bitpatternAsPointer)
CNIOLinux.io_uring_sqe_set_data(sqe, userBitpatternAsPointer) // must be done after prep_poll_add, otherwise zeroed out.
if link {
CNIOLinux_io_uring_set_link_flag(sqe)
}
}
if submitNow {
self.io_uring_flush()
}
}
// the update/multishot polls are
internal func io_uring_poll_update(fileDescriptor: CInt, newPollmask: UInt32, oldPollmask: UInt32, registrationID: SelectorRegistrationID, submitNow: Bool = true, multishot: Bool = true) -> () {
let bitpattern = UInt64(URingUserData(registrationID: registrationID,
fileDescriptor: fileDescriptor,
eventType:CQEEventType.poll))
let userbitPattern = UInt64(URingUserData(registrationID: registrationID,
fileDescriptor: fileDescriptor,
eventType:CQEEventType.pollModify))
let bitpatternAsPointer = UnsafeMutableRawPointer.init(bitPattern: UInt(bitpattern))
let userBitpatternAsPointer = UnsafeMutableRawPointer.init(bitPattern: UInt(userbitPattern))
_debugPrint("io_uring_poll_update fileDescriptor[\(fileDescriptor)] oldPollmask[\(oldPollmask)] newPollmask[\(newPollmask)] userBitpatternAsPointer[\(String(describing:userBitpatternAsPointer))]")
self.withSQE { sqe in
// "Documentation" for multishot polls and updates here:
// https://git.kernel.dk/cgit/linux-block/commit/?h=poll-multiple&id=33021a19e324fb747c2038416753e63fd7cd9266
var flags = IORING_POLL_UPDATE_EVENTS | IORING_POLL_UPDATE_USER_DATA
if multishot {
flags |= IORING_POLL_ADD_MULTI // ask for multiple updates
}
CNIOLinux.io_uring_prep_poll_update(sqe, bitpatternAsPointer, bitpatternAsPointer, newPollmask, flags)
CNIOLinux.io_uring_sqe_set_data(sqe, userBitpatternAsPointer)
}
if submitNow {
self.io_uring_flush()
}
}
internal func _debugPrint(_ s: @autoclosure () -> String)
{
#if SWIFTNIO_IO_URING_DEBUG_URING
print("L [\(NIOThread.current)] " + s())
#endif
}
// We merge results into fdEvents on (fd, registrationID) for the given CQE
// this minimizes amount of events propagating up and allows Selector to discard
// events with an old sequence identifier.
internal func _process_cqe(events: UnsafeMutablePointer<URingEvent>, cqeIndex: Int, multishot: Bool) {
let bitPattern = UInt(bitPattern:io_uring_cqe_get_data(cqes[cqeIndex]))
let uringUserData = URingUserData(rawValue: UInt64(bitPattern))
let result = cqes[cqeIndex]!.pointee.res
switch uringUserData.eventType {
case .poll:
switch result {
case -ECANCELED:
var pollError: UInt32 = 0
assert(uringUserData.fileDescriptor >= 0, "fd must be zero or greater")
if multishot { // -ECANCELED for streaming polls, should signal error
pollError = URing.POLLERR | URing.POLLHUP
} else { // this just signals that Selector just should resubmit a new fresh poll
pollError = URing.POLLCANCEL
}
if let current = fdEvents[FDEventKey(uringUserData.fileDescriptor, uringUserData.registrationID)] {
fdEvents[FDEventKey(uringUserData.fileDescriptor, uringUserData.registrationID)] = current | pollError
} else {
fdEvents[FDEventKey(uringUserData.fileDescriptor, uringUserData.registrationID)] = pollError
}
break
// We can validly receive an EBADF as a close() can race vis-a-vis pending SQE:s
// with polls / pollModifications - in that case, we should just discard the result.
// This is similar to the assert in BaseSocketChannel and is due to the lack
// of implicit synchronization with regard to registration changes for io_uring
// - we simply can't know when the kernel will process our SQE without
// heavy-handed synchronization which would dump performance.
// Discussion here:
// https://github.com/apple/swift-nio/pull/1804#discussion_r621304055
// including clarifications from @isilence (one of the io_uring developers)
case -EBADF:
_debugPrint("Failed poll with -EBADF for cqeIndex[\(cqeIndex)]")
break
case ..<0: // other errors
fatalError("Failed poll with unexpected error (\(result) for cqeIndex[\(cqeIndex)]")
break
case 0: // successfull chained add for singleshots, not an event
break
default: // positive success
assert(uringUserData.fileDescriptor >= 0, "fd must be zero or greater")
let uresult = UInt32(result)
if let current = fdEvents[FDEventKey(uringUserData.fileDescriptor, uringUserData.registrationID)] {
fdEvents[FDEventKey(uringUserData.fileDescriptor, uringUserData.registrationID)] = current | uresult
} else {
fdEvents[FDEventKey(uringUserData.fileDescriptor, uringUserData.registrationID)] = uresult
}
}
case .pollModify: // we only get this for multishot modifications
switch result {
case -ECANCELED: // -ECANCELED for streaming polls, should signal error
assert(uringUserData.fileDescriptor >= 0, "fd must be zero or greater")
let pollError = URing.POLLERR // URing.POLLERR // (URing.POLLHUP | URing.POLLERR)
if let current = fdEvents[FDEventKey(uringUserData.fileDescriptor, uringUserData.registrationID)] {
fdEvents[FDEventKey(uringUserData.fileDescriptor, uringUserData.registrationID)] = current | pollError
} else {
fdEvents[FDEventKey(uringUserData.fileDescriptor, uringUserData.registrationID)] = pollError
}
break
case -EALREADY:
_debugPrint("Failed pollModify with -EALREADY for cqeIndex[\(cqeIndex)]")
break
case -ENOENT:
_debugPrint("Failed pollModify with -ENOENT for cqeIndex [\(cqeIndex)]")
break
// See the description for EBADF handling above in the poll case for rationale of allowing EBADF.
case -EBADF:
_debugPrint("Failed pollModify with -EBADF for cqeIndex[\(cqeIndex)]")
break
case ..<0: // other errors
fatalError("Failed pollModify with unexpected error (\(result) for cqeIndex[\(cqeIndex)]")
break
case 0: // successfull chained add, not an event
break
default: // positive success
fatalError("pollModify returned > 0")
}
break
case .pollDelete:
break
}
}
internal func io_uring_peek_batch_cqe(events: UnsafeMutablePointer<URingEvent>, maxevents: UInt32, multishot: Bool = true) -> Int {
var eventCount = 0
var currentCqeCount = CNIOLinux.io_uring_peek_batch_cqe(&ring, cqes, cqeMaxCount)
if currentCqeCount == 0 {
_debugPrint("io_uring_peek_batch_cqe found zero events, breaking out")
return 0
}
_debugPrint("io_uring_peek_batch_cqe found [\(currentCqeCount)] events")
self._dumpCqes("io_uring_peek_batch_cqe", count: Int(currentCqeCount))
assert(currentCqeCount >= 0, "currentCqeCount should never be negative")
assert(maxevents > 0, "maxevents should be a positive number")
for cqeIndex in 0 ..< currentCqeCount
{
self._process_cqe(events: events, cqeIndex: Int(cqeIndex), multishot:multishot)
if (fdEvents.count == maxevents) // ensure we don't generate more events than maxevents
{
_debugPrint("io_uring_peek_batch_cqe breaking loop early, currentCqeCount [\(currentCqeCount)] maxevents [\(maxevents)]")
currentCqeCount = maxevents // to make sure we only cq_advance the correct amount
break
}
}
io_uring_cq_advance(&ring, currentCqeCount) // bulk variant of io_uring_cqe_seen(&ring, dataPointer)
// we just return single event per fd, sequencenumber pair
eventCount = 0
for (eventKey, pollMask) in fdEvents {
assert(eventCount < maxevents)
assert(eventKey.fileDescriptor >= 0)
events[eventCount].fd = eventKey.fileDescriptor
events[eventCount].pollMask = pollMask
events[eventCount].registrationID = eventKey.registrationID
if (pollMask & URing.POLLCANCEL) != 0 {
events[eventCount].pollMask &= ~URing.POLLCANCEL
events[eventCount].pollCancelled = true
}
eventCount += 1
}
fdEvents.removeAll(keepingCapacity: true) // reused for next batch
_debugPrint("io_uring_peek_batch_cqe returning [\(eventCount)] events, fdEvents.count [\(fdEvents.count)]")
return eventCount
}
internal func _io_uring_wait_cqe_shared(events: UnsafeMutablePointer<URingEvent>, error: Int32, multishot: Bool) throws -> Int {
var eventCount = 0
switch error {
case 0:
break
case -CNIOLinux.EINTR:
_debugPrint("_io_uring_wait_cqe_shared got CNIOLinux.EINTR")
return eventCount
case -CNIOLinux.ETIME:
_debugPrint("_io_uring_wait_cqe_shared timed out with -CNIOLinux.ETIME")
CNIOLinux.io_uring_cqe_seen(&ring, cqes[0])
return eventCount
default:
_debugPrint("URingError.uringWaitCqeFailure \(error)")
throw URingError.uringWaitCqeFailure
}
self._dumpCqes("_io_uring_wait_cqe_shared")
self._process_cqe(events: events, cqeIndex: 0, multishot:multishot)
CNIOLinux.io_uring_cqe_seen(&ring, cqes[0])
if let firstEvent = fdEvents.first {
events[0].fd = firstEvent.key.fileDescriptor
events[0].pollMask = firstEvent.value
events[0].registrationID = firstEvent.key.registrationID
eventCount = 1
} else {
_debugPrint("_io_uring_wait_cqe_shared if let firstEvent = fdEvents.first failed")
}
fdEvents.removeAll(keepingCapacity: true) // reused for next batch
return eventCount
}
internal func io_uring_wait_cqe(events: UnsafeMutablePointer<URingEvent>, maxevents: UInt32, multishot: Bool = true) throws -> Int {
_debugPrint("io_uring_wait_cqe")
let error = CNIOLinux.io_uring_wait_cqe(&ring, cqes)
return try self._io_uring_wait_cqe_shared(events: events, error: error, multishot:multishot)
}
internal func io_uring_wait_cqe_timeout(events: UnsafeMutablePointer<URingEvent>, maxevents: UInt32, timeout: TimeAmount, multishot: Bool = true) throws -> Int {
var ts = timeout.kernelTimespec()
_debugPrint("io_uring_wait_cqe_timeout.ETIME milliseconds \(ts)")
let error = CNIOLinux.io_uring_wait_cqe_timeout(&ring, cqes, &ts)
return try self._io_uring_wait_cqe_shared(events: events, error: error, multishot:multishot)
}
}
#endif
#endif

View File

@ -12,6 +12,8 @@
//
//===----------------------------------------------------------------------===//
#if !SWIFTNIO_USE_IO_URING
#if os(Linux) || os(Android)
/// Represents the `epoll` filters/events we might use:
@ -288,3 +290,5 @@ extension Selector: _SelectorBackendProtocol {
}
#endif
#endif

View File

@ -135,11 +135,20 @@ internal class Selector<R: Registration> {
#if os(macOS) || os(iOS) || os(watchOS) || os(tvOS)
typealias EventType = kevent
#elseif os(Linux) || os(Android)
#if !SWIFTNIO_USE_IO_URING
typealias EventType = Epoll.epoll_event
var earliestTimer: NIODeadline = .distantFuture
var eventFD: CInt = -1 // -1 == we're closed
var timerFD: CInt = -1 // -1 == we're closed
#else
typealias EventType = URingEvent
var eventFD: CInt = -1 // -1 == we're closed
var ring = URing()
let multishot = URing.io_uring_use_multishot_poll // if true, we run with streaming multishot polls
let deferReregistrations = true // if true we only flush once at reentring whenReady() - saves syscalls
var deferredReregistrationsPending = false // true if flush needed when reentring whenReady()
#endif
#else
#error("Unsupported platform, no suitable selector backend (we need kqueue or epoll support)")
#endif

View File

@ -0,0 +1,361 @@
//===----------------------------------------------------------------------===//
//
// This source file is part of the SwiftNIO open source project
//
// Copyright (c) 2021 Apple Inc. and the SwiftNIO project authors
// Licensed under Apache License v2.0
//
// See LICENSE.txt for license information
// See CONTRIBUTORS.txt for the list of SwiftNIO project authors
//
// SPDX-License-Identifier: Apache-2.0
//
//===----------------------------------------------------------------------===//
#if SWIFTNIO_USE_IO_URING
#if os(Linux) || os(Android)
/// Represents the `poll` filters/events we might use from io_uring:
///
/// - `hangup` corresponds to `POLLHUP`
/// - `readHangup` corresponds to `POLLRDHUP`
/// - `input` corresponds to `POLLIN`
/// - `output` corresponds to `POLLOUT`
/// - `error` corresponds to `POLLERR`
private struct URingFilterSet: OptionSet, Equatable {
typealias RawValue = UInt8
let rawValue: RawValue
static let _none = URingFilterSet([])
static let hangup = URingFilterSet(rawValue: 1 << 0)
static let readHangup = URingFilterSet(rawValue: 1 << 1)
static let input = URingFilterSet(rawValue: 1 << 2)
static let output = URingFilterSet(rawValue: 1 << 3)
static let error = URingFilterSet(rawValue: 1 << 4)
init(rawValue: RawValue) {
self.rawValue = rawValue
}
}
extension URingFilterSet {
/// Convert NIO's `SelectorEventSet` set to a `URingFilterSet`
init(selectorEventSet: SelectorEventSet) {
var thing: URingFilterSet = [.error, .hangup]
if selectorEventSet.contains(.read) {
thing.formUnion(.input)
}
if selectorEventSet.contains(.write) {
thing.formUnion(.output)
}
if selectorEventSet.contains(.readEOF) {
thing.formUnion(.readHangup)
}
self = thing
}
}
extension SelectorEventSet {
var uringEventSet: UInt32 {
assert(self != ._none)
// POLLERR | POLLHUP is always set unconditionally anyway but it's easier to understand if we explicitly ask.
var filter: UInt32 = URing.POLLERR | URing.POLLHUP
let uringFilters = URingFilterSet(selectorEventSet: self)
if uringFilters.contains(.input) {
filter |= URing.POLLIN
}
if uringFilters.contains(.output) {
filter |= URing.POLLOUT
}
if uringFilters.contains(.readHangup) {
filter |= URing.POLLRDHUP
}
assert(filter & URing.POLLHUP != 0) // both of these are reported
assert(filter & URing.POLLERR != 0) // always and can't be masked.
return filter
}
fileprivate init(uringEvent: UInt32) {
var selectorEventSet: SelectorEventSet = ._none
if uringEvent & URing.POLLIN != 0 {
selectorEventSet.formUnion(.read)
}
if uringEvent & URing.POLLOUT != 0 {
selectorEventSet.formUnion(.write)
}
if uringEvent & URing.POLLRDHUP != 0 {
selectorEventSet.formUnion(.readEOF)
}
if uringEvent & URing.POLLHUP != 0 || uringEvent & URing.POLLERR != 0 {
selectorEventSet.formUnion(.reset)
}
self = selectorEventSet
}
}
extension Selector: _SelectorBackendProtocol {
internal func _debugPrint(_ s: @autoclosure () -> String) {
#if SWIFTNIO_IO_URING_DEBUG_SELECTOR
print("S [\(NIOThread.current)] " + s())
#endif
}
func initialiseState0() throws {
try ring.io_uring_queue_init()
self.selectorFD = ring.fd
// eventfd are always singleshot and re-register each time around
// as certain use cases of nio seems to generate superfluous wakeups.
// (at least its tested for that in some of the performance tests
// e.g. future_whenallsucceed_100k_deferred_off_loop, future_whenallcomplete_100k_deferred_off_loop
// ) - if using normal ET multishots, we would get 100k events to handle basically.
// so using single shot for wakeups makes those tests run 30-35% faster approx.
self.eventFD = try EventFd.eventfd(initval: 0, flags: Int32(EventFd.EFD_CLOEXEC | EventFd.EFD_NONBLOCK))
ring.io_uring_prep_poll_add(fileDescriptor: self.eventFD,
pollMask: URing.POLLIN,
registrationID:SelectorRegistrationID(rawValue: 0),
multishot:false) // wakeups
self.lifecycleState = .open
_debugPrint("URingSelector up and running fd [\(self.selectorFD)] wakeups on event_fd [\(self.eventFD)]")
}
func deinitAssertions0() {
assert(self.eventFD == -1, "self.eventFD == \(self.eventFD) on deinitAssertions0 deinit, forgot close?")
}
func register0<S: Selectable>(selectable: S,
fileDescriptor: CInt,
interested: SelectorEventSet,
registrationID: SelectorRegistrationID) throws {
_debugPrint("register interested \(interested) uringEventSet [\(interested.uringEventSet)] registrationID[\(registrationID)]")
self.deferredReregistrationsPending = true
ring.io_uring_prep_poll_add(fileDescriptor: fileDescriptor,
pollMask: interested.uringEventSet,
registrationID: registrationID,
submitNow: !deferReregistrations,
multishot: multishot)
}
func reregister0<S: Selectable>(selectable: S,
fileDescriptor: CInt,
oldInterested: SelectorEventSet,
newInterested: SelectorEventSet,
registrationID: SelectorRegistrationID) throws {
_debugPrint("Re-register old \(oldInterested) new \(newInterested) uringEventSet [\(oldInterested.uringEventSet)] reg.uringEventSet [\(newInterested.uringEventSet)]")
self.deferredReregistrationsPending = true
if multishot {
ring.io_uring_poll_update(fileDescriptor: fileDescriptor,
newPollmask: newInterested.uringEventSet,
oldPollmask: oldInterested.uringEventSet,
registrationID: registrationID,
submitNow: !deferReregistrations,
multishot: true)
} else {
ring.io_uring_prep_poll_remove(fileDescriptor: fileDescriptor,
pollMask: oldInterested.uringEventSet,
registrationID: registrationID,
submitNow:!deferReregistrations,
link: true) // next event linked will cancel if this event fails
ring.io_uring_prep_poll_add(fileDescriptor: fileDescriptor,
pollMask: newInterested.uringEventSet,
registrationID: registrationID,
submitNow: !deferReregistrations,
multishot: false)
}
}
func deregister0<S: Selectable>(selectable: S, fileDescriptor: CInt, oldInterested: SelectorEventSet, registrationID: SelectorRegistrationID) throws {
_debugPrint("deregister interested \(selectable) reg.interested.uringEventSet [\(oldInterested.uringEventSet)]")
self.deferredReregistrationsPending = true
ring.io_uring_prep_poll_remove(fileDescriptor: fileDescriptor,
pollMask: oldInterested.uringEventSet,
registrationID: registrationID,
submitNow:!deferReregistrations)
}
private func shouldRefreshPollForEvent(selectorEvent:SelectorEventSet) -> Bool {
if selectorEvent.contains(.read) {
// as we don't do exhaustive reads, we need to prod the kernel for
// new events, would be even better if we knew if we had read all there is
return true
}
// If the event is fully handled, we can return false to avoid reregistration for multishot polls.
return false
}
/// Apply the given `SelectorStrategy` and execute `body` once it's complete (which may produce `SelectorEvent`s to handle).
///
/// - parameters:
/// - strategy: The `SelectorStrategy` to apply
/// - body: The function to execute for each `SelectorEvent` that was produced.
func whenReady0(strategy: SelectorStrategy, onLoopBegin loopStart: () -> Void, _ body: (SelectorEvent<R>) throws -> Void) throws -> Void {
assert(self.myThread == NIOThread.current)
guard self.lifecycleState == .open else {
throw IOError(errnoCode: EBADF, reason: "can't call whenReady for selector as it's \(self.lifecycleState).")
}
var ready: Int = 0
// flush reregisteration of pending modifications if needed (nop in SQPOLL mode)
// basically this elides all reregistrations and deregistrations into a single
// syscall instead of one for each. Future improvement would be to also merge
// the pending pollmasks (now each change will be queued, but we could also
// merge the masks for reregistrations) - but the most important thing is to
// only trap into the kernel once for the set of changes, so needs to be measured.
if deferReregistrations && self.deferredReregistrationsPending {
self.deferredReregistrationsPending = false
ring.io_uring_flush()
}
switch strategy {
case .now:
_debugPrint("whenReady.now")
ready = Int(ring.io_uring_peek_batch_cqe(events: events, maxevents: UInt32(eventsCapacity), multishot:multishot))
case .blockUntilTimeout(let timeAmount):
_debugPrint("whenReady.blockUntilTimeout")
ready = try Int(ring.io_uring_wait_cqe_timeout(events: events, maxevents: UInt32(eventsCapacity), timeout:timeAmount, multishot:multishot))
case .block:
_debugPrint("whenReady.block")
ready = Int(ring.io_uring_peek_batch_cqe(events: events, maxevents: UInt32(eventsCapacity), multishot:multishot)) // first try to consume any existing
if (ready <= 0) // otherwise block (only single supported, but we will use batch peek cqe next run around...
{
ready = try ring.io_uring_wait_cqe(events: events, maxevents: UInt32(eventsCapacity), multishot:multishot)
}
}
loopStart()
for i in 0..<ready {
let event = events[i]
switch event.fd {
case self.eventFD: // we don't run these as multishots to avoid tons of events when many wakeups are done
_debugPrint("wakeup successful for event.fd [\(event.fd)]")
var val = EventFd.eventfd_t()
ring.io_uring_prep_poll_add(fileDescriptor: self.eventFD,
pollMask: URing.POLLIN,
registrationID: SelectorRegistrationID(rawValue: 0),
submitNow: false,
multishot: false)
do {
_ = try EventFd.eventfd_read(fd: self.eventFD, value: &val) // consume wakeup event
_debugPrint("read val [\(val)] from event.fd [\(event.fd)]")
} catch {
}
default:
if let registration = registrations[Int(event.fd)] {
_debugPrint("We found a registration for event.fd [\(event.fd)]") // \(registration)
// The io_uring backend only has 16 bits available for the registration id
guard event.registrationID == UInt16(truncatingIfNeeded:registration.registrationID.rawValue) else {
_debugPrint("The event.registrationID [\(event.registrationID)] != registration.selectableregistrationID [\(registration.registrationID)], skipping to next event")
continue
}
var selectorEvent = SelectorEventSet(uringEvent: event.pollMask)
_debugPrint("selectorEvent [\(selectorEvent)] registration.interested [\(registration.interested)]")
// we only want what the user is currently registered for & what we got
selectorEvent = selectorEvent.intersection(registration.interested)
_debugPrint("intersection [\(selectorEvent)]")
if selectorEvent.contains(.readEOF) {
_debugPrint("selectorEvent.contains(.readEOF) [\(selectorEvent.contains(.readEOF))]")
}
if multishot == false { // must be before guard, otherwise lost wake
ring.io_uring_prep_poll_add(fileDescriptor: event.fd,
pollMask: registration.interested.uringEventSet,
registrationID: registration.registrationID,
submitNow: false,
multishot: false)
if event.pollCancelled {
_debugPrint("Received event.pollCancelled")
}
}
guard selectorEvent != ._none else {
_debugPrint("selectorEvent != ._none / [\(selectorEvent)] [\(registration.interested)] [\(SelectorEventSet(uringEvent: event.pollMask))] [\(event.pollMask)] [\(event.fd)]")
continue
}
// This is only needed due to the edge triggered nature of liburing, possibly
// we can get away with only updating (force triggering an event if available) for
// partial reads (where we currently give up after N iterations)
if multishot && self.shouldRefreshPollForEvent(selectorEvent:selectorEvent) { // can be after guard as it is multishot
ring.io_uring_poll_update(fileDescriptor: event.fd,
newPollmask: registration.interested.uringEventSet,
oldPollmask: registration.interested.uringEventSet,
registrationID: registration.registrationID,
submitNow: false)
}
_debugPrint("running body [\(NIOThread.current)] \(selectorEvent) \(SelectorEventSet(uringEvent: event.pollMask))")
try body((SelectorEvent(io: selectorEvent, registration: registration)))
} else { // remove any polling if we don't have a registration for it
_debugPrint("We had no registration for event.fd [\(event.fd)] event.pollMask [\(event.pollMask)] event.registrationID [\(event.registrationID)], it should be deregistered already")
if multishot == false {
ring.io_uring_prep_poll_remove(fileDescriptor: event.fd,
pollMask: event.pollMask,
registrationID: SelectorRegistrationID(rawValue: UInt32(event.registrationID)),
submitNow: false)
}
}
}
}
self.deferredReregistrationsPending = false // none pending as we will flush here
ring.io_uring_flush() // flush reregisteration of the polls if needed (nop in SQPOLL mode)
growEventArrayIfNeeded(ready: ready)
}
/// Close the `Selector`.
///
/// After closing the `Selector` it's no longer possible to use it.
public func close0() throws {
self.externalSelectorFDLock.withLock {
// We try! all of the closes because close can only fail in the following ways:
// - EINTR, which we eat in Posix.close
// - EIO, which can only happen for on-disk files
// - EBADF, which can't happen here because we would crash as EBADF is marked unacceptable
// Therefore, we assert here that close will always succeed and if not, that's a NIO bug we need to know
// about.
ring.io_uring_queue_exit() // This closes the ring selector fd for us
self.selectorFD = -1
try! Posix.close(descriptor: self.eventFD)
self.eventFD = -1
}
return
}
/* attention, this may (will!) be called from outside the event loop, ie. can't access mutable shared state (such as `self.open`) */
func wakeup0() throws {
assert(NIOThread.current != self.myThread)
try self.externalSelectorFDLock.withLock {
guard self.eventFD >= 0 else {
throw EventLoopError.shutdown
}
_ = try EventFd.eventfd_write(fd: self.eventFD, value: 1)
}
}
}
#endif
#endif

View File

@ -369,9 +369,9 @@ class StreamChannelTest: XCTestCase {
for _ in 0..<10 {
// We just spin here for a little while to check that there are no bogus events available on the
// selector.
XCTAssertNoThrow(try (receiver.eventLoop as! SelectableEventLoop)
._selector.testsOnly_withUnsafeSelectorFD { fd in
try assertNoSelectorChanges(fd: fd)
let eventLoop = (receiver.eventLoop as! SelectableEventLoop)
XCTAssertNoThrow(try eventLoop._selector.testsOnly_withUnsafeSelectorFD { fd in
try assertNoSelectorChanges(fd: fd, selector:eventLoop._selector)
}, "after \(sends.load()) sends, we got an unexpected selector event for \(receiver)")
usleep(10000)
}
@ -889,7 +889,7 @@ final class AccumulateAllReads: ChannelInboundHandler {
}
}
private func assertNoSelectorChanges(fd: CInt, file: StaticString = #file, line: UInt = #line) throws {
private func assertNoSelectorChanges(fd: CInt, selector: NIO.Selector<NIORegistration>, file: StaticString = #file, line: UInt = #line) throws {
struct UnexpectedSelectorChanges: Error, CustomStringConvertible {
let description: String
}
@ -902,12 +902,23 @@ private func assertNoSelectorChanges(fd: CInt, file: StaticString = #file, line:
throw UnexpectedSelectorChanges(description: "\(ev)")
}
#elseif os(Linux) || os(Android)
#if !SWIFTNIO_USE_IO_URING
var ev = Epoll.epoll_event()
let numberOfEvents = try Epoll.epoll_wait(epfd: fd, events: &ev, maxevents: 1, timeout: 0)
guard numberOfEvents == 0 else {
throw UnexpectedSelectorChanges(description: "\(ev) [userdata: \(EPollUserData(rawValue: ev.data.u64))]")
}
#else
let events: UnsafeMutablePointer<URingEvent> = UnsafeMutablePointer.allocate(capacity: 1)
events.initialize(to: URingEvent())
let numberOfEvents = try selector.ring.io_uring_wait_cqe_timeout(events: events, maxevents: 1, timeout: TimeAmount.seconds(0))
events.deinitialize(count: 1)
events.deallocate()
guard numberOfEvents == 0 else {
throw UnexpectedSelectorChanges(description: "\(selector)")
}
#endif
#else
#warning("assertNoSelectorChanges unsupported on this OS.")
#endif
}

View File

@ -414,11 +414,11 @@ swift_slowAlloc
in /home/ubuntu/bin/usr/lib/swift/linux/libswiftCore.so
Swift.String.utf8CString.getter : Swift.ContiguousArray<Swift.Int8>
in /home/ubuntu/bin/usr/lib/swift/linux/libswiftCore.so
NIO.Uring.getEnvironmentVar(Swift.String) -> Swift.String?
at /home/ubuntu/swiftnio/swift-nio/Sources/NIO/LinuxUring.swift:291
NIO.URing.getEnvironmentVar(Swift.String) -> Swift.String?
at /home/ubuntu/swiftnio/swift-nio/Sources/NIO/LinuxURing.swift:291
in /tmp/.nio_alloc_counter_tests_GRusAy/.build/x86_64-unknown-linux-gnu/release/test_1000_autoReadGetAndSet
NIO.Uring._debugPrint(@autoclosure () -> Swift.String) -> ()
at /home/ubuntu/swiftnio/swift-nio/Sources/NIO/LinuxUring.swift:297
NIO.URing._debugPrint(@autoclosure () -> Swift.String) -> ()
at /home/ubuntu/swiftnio/swift-nio/Sources/NIO/LinuxURing.swift:297
...
22196 temporary allocations of 22276 allocations in total (99.64%) from:
```
@ -426,10 +426,10 @@ swift_slowAlloc
And here we could fairly quickly see that the transient extra allocations was due to extra debug printing and querying of environment variables:
```
NIO.Uring.getEnvironmentVar(Swift.String) -> Swift.String?
at /home/ubuntu/swiftnio/swift-nio/Sources/NIO/LinuxUring.swift:291
NIO.URing.getEnvironmentVar(Swift.String) -> Swift.String?
at /home/ubuntu/swiftnio/swift-nio/Sources/NIO/LinuxURing.swift:291
in /tmp/.nio_alloc_counter_tests_GRusAy/.build/x86_64-unknown-linux-gnu/release/test_1000_autoReadGetAndSet
NIO.Uring._debugPrint(@autoclosure () -> Swift.String) -> ()
NIO.URing._debugPrint(@autoclosure () -> Swift.String) -> ()
```
And this code will be removed before final integration of the feature branch, so the diff will go away.

30
docs/io_uring.md Normal file
View File

@ -0,0 +1,30 @@
# SwiftNIO io_uring support status [Linux]
## Background
Linux has fairly recently received a new asynchrounus I/O interface — `io_uring` — that allows high-performance AIO with a minimum amount of system calls. Some background information at [LWN](https://lwn.net/Articles/810414/) and [kernel.dk](https://kernel.dk/io_uring.pdf)
## Current status and supported platforms
SwiftNIO has basic support for using `io_uring` as a notification mechanism. `io_uring` is available on Linux kernels starting from 5.1 and requires support for the POLL operations. SwiftNIO optionally supports multishot polls and poll updates through a configuration option - these are new interfaces that are expected to available from the future release 5.13 (as of this writing). The `io_uring` implementation has primarily been tested on version 5.12rc.
## Performance expectations
Currently the `io_uring` implementation does not yet use the underlying interface to its full potential - it is only used for polling and not yet for asynchronous operation of send/recv/read/write. This means performance characteristics currently will be similar to the default `epoll()` (or slightly worse), but you should test for your specific use case as YMMV. This is expected to change in the future as the `io_uring` interface is used more extensively.
## How to use and requirements
To enable `io_uring`, SwiftNIO needs to be built and run on a machine that has the [liburing](https://github.com/axboe/liburing) library installed (typically in `/usr/local`). If built with liburing, it must be available and work on the production machine.
To build SwiftNIO so it uses io_uring instead of epoll, you must specify additional flags when building:
`swift build -Xcc -DSWIFTNIO_USE_IO_URING=1 -Xlinker -luring -Xswiftc -DSWIFTNIO_USE_IO_URING`
To build it so also use the new poll update / multishot polls (will need kernel 5.13+) you need to do:
`swift build -Xcc -DSWIFTNIO_USE_IO_URING=1 -Xlinker -luring -Xswiftc -DSWIFTNIO_USE_IO_URING -Xswiftc -DSWIFTNIO_IO_URING_MULTISHOT`
## Debug output
There are currently three debug outputs that can be compiled in (e.g. to verify that `liburing` is used as expected and for development troubleshooting)
`-Xswiftc -DSWIFTNIO_IO_URING_DEBUG_SELECTOR`
`-Xswiftc -DSWIFTNIO_IO_URING_DEBUG_URING`
`-Xswiftc -DSWIFTNIO_IO_URING_DEBUG_DUMP_CQE`
## Roadmap
The initial support for `io_uring` is primarily intended to give people interested in it that ability to try it out and provide feedback - in the future the expectation is to add full support for `io_uring` which is expected to give significant performance improvements on Linux systems supporting it.
The idea is also to add SQPOLL together with fine tune settings such that it can be pinned on a specific CPU.