RavenDB 7.1: One IO Ring to rule them all

One of the more interesting developments in terms of kernel API surface is the IO Ring. On Linux, it is called IO Uring and Windows has copied it shortly afterward. The idea started as a way to batch multiple IO operations at once but has evolved into a generic mechanism to make system calls more cheaply. On Linux, a large portion of the kernel features is exposed as part of the IO Uring API, while Windows exposes a far less rich API (basically, just reading and writing). The reason this matters is that you can use IO Ring to reduce the cost of making system calls, using both batching and asynchronous programming. As such, most new database engines have jumped on that sweet nectar of better performance results.As part of the overall re-architecture of how Voron manages writes, we have done the same. I/O for Voron is typically composed of writes to the journals and to the data file, so that makes it a really good fit, sort of. An ironic aspect of IO Uring is that despite it being an asynchronous mechanism, it is inherently single-threaded. There are good reasons for that, of course, but that means that if you want to use the IO Ring API in a multi-threaded environment, you need to take that into account. A common way to handle that is to use an event-driven system, where all the actual calls are generated from a single “event loop” thread or similar. This is how the Node.js API works, and how .NET itself manages IO for sockets (there is a single thread that listens to socket events by default). The whole point of IO Ring is that you can submit multiple operations for the kernel to run in as optimal a manner as possible. Here is one such case to consider, this is the part of the code where we write the modified pages to the data file:using (fileHandle) { for (int i = 0; i filefd, IORING_FSYNC_DATASYNC); break; case workitem_write: io_uring_prep_writev(sqe, work->filefd, work->op.write.iovecs, work->op.write.iovecs_count, work->offset); break; default: break; } work = work->next; } sumbit_and_wait: rc = must_wait ? io_uring_submit_and_wait(ring, must_wait) : io_uring_submit(ring); struct io_uring_cqe *cqe; uint32_t head = 0; uint32_t i = 0; io_uring_for_each_cqe(ring, head, cqe) { i++; // force another run of the inner loop, // to ensure that we call io_uring_submit again has_work = true; struct workitem *cur = io_uring_cqe_get_data(cqe); if (!cur) { // can be null if it is: // * a notification about eventfd write continue; } switch (cur->type) { case workitem_fsync: notify_work_completed(ring, cur); break; case workitem_write: if (/* partial write */) { // queue again continue; } notify_work_completed(ring, cur); break; } } io_uring_cq_advance(ring, i); } } return 0; }What does this code do? We start by checking if we want to use lower-priority I/O, this is because we don’t actually care how long those operations take. The purpose of writing the data to the disk is that it will reach it eventually. RavenDB has two types of writes:Journal writes (durable update to the write-ahead log, required to complete a transaction).Data flush / Data sync (background updates to the data file, currently buffered in memory, no user is waiting for that)As such, we are fine with explicitly prioritizing the journal writes (which users are waiting for) in favor of all other operations.What is this C code? I thought RavenDB was written in C#RavenDB is written in C#, but for very low-level system details, we found that it is far easier to write a Platform Abstraction Layer to hide system-specific concerns from the rest of the code. That way, we can simply submit the data to write and have the abstraction layer take care of all of that for us. This also ensures that we amortize the cost of PInvoke calls across many operations by submitting a big batch to the C code at once.After setting the IO priority, we start reading from what is effectively a thread-safe queue. We wait for eventfd() to signal that there is work to do, and then we grab the head of the queue and start running.The idea is that we fetch items from the queue, then we write those operations to the IO Ring as fast as we can manage. The IO Ring size is limited, however. So we need to handle the case where we have more work for the IO Ring than it can accept. When that happens, we will go to the submit_and_wait label and wait for something to complete. Note that there is some logic there to handle what is going on when the IO Ring is full. We submit all the work in the ring, wait for an operation to complete, and in the next run, we’ll continue processin

Mar 18, 2025 - 13:36
 0

One of the more interesting developments in terms of kernel API surface is the IO Ring. On Linux, it is called IO Uring and Windows has copied it shortly afterward. The idea started as a way to batch multiple IO operations at once but has evolved into a generic mechanism to make system calls more cheaply. On Linux, a large portion of the kernel features is exposed as part of the IO Uring API, while Windows exposes a far less rich API (basically, just reading and writing).

The reason this matters is that you can use IO Ring to reduce the cost of making system calls, using both batching and asynchronous programming. As such, most new database engines have jumped on that sweet nectar of better performance results.

As part of the overall re-architecture of how Voron manages writes, we have done the same. I/O for Voron is typically composed of writes to the journals and to the data file, so that makes it a really good fit, sort of.

An ironic aspect of IO Uring is that despite it being an asynchronous mechanism, it is inherently single-threaded. There are good reasons for that, of course, but that means that if you want to use the IO Ring API in a multi-threaded environment, you need to take that into account.

A common way to handle that is to use an event-driven system, where all the actual calls are generated from a single “event loop” thread or similar. This is how the Node.js API works, and how .NET itself manages IO for sockets (there is a single thread that listens to socket events by default).

The whole point of IO Ring is that you can submit multiple operations for the kernel to run in as optimal a manner as possible. Here is one such case to consider, this is the part of the code where we write the modified pages to the data file:


using (fileHandle)
{
    for (int i = 0; i < pages.Length; i++)
    {
        int numberOfPages = pages[i].GetNumberOfPages();


        var size = numberOfPages * Constants.Storage.PageSize;
        var offset = pages[i].PageNumber * Constants.Storage.PageSize;
        var span = new Span<byte>(pages[i].Pointer, size);
        RandomAccess.Write(fileHandle, span, offset);


        written += numberOfPages * Constants.Storage.PageSize;
    }
}


PID     LWP TTY          TIME CMD
  22334   22345 pts/0    00:00:00 iou-wrk-22343
  22334   22346 pts/0    00:00:00 iou-wrk-22343
  22334   22347 pts/0    00:00:00 iou-wrk-22334
  22334   22348 pts/0    00:00:00 iou-wrk-22334
  22334   22349 pts/0    00:00:00 iou-wrk-22334
  22334   22350 pts/0    00:00:00 iou-wrk-22334
  22334   22351 pts/0    00:00:00 iou-wrk-22334
  22334   22352 pts/0    00:00:00 iou-wrk-22334
  22334   22353 pts/0    00:00:00 iou-wrk-22334
  22334   22354 pts/0    00:00:00 iou-wrk-22334
  22334   22355 pts/0    00:00:00 iou-wrk-22334
  22334   22356 pts/0    00:00:00 iou-wrk-22334
  22334   22357 pts/0    00:00:00 iou-wrk-22334
  22334   22358 pts/0    00:00:00 iou-wrk-22334

Actually, those aren’t threads in the normal sense. Those are kernel tasks, generated by the IO Ring at the kernel level directly. It turns out that internally, IO Ring may spawn worker threads to do the async work at the kernel level. When we had a separate IO Ring per file, each one of them had its own pool of threads to do the work.

The way it usually works is really interesting. The IO Ring will attempt to complete the operation in a synchronous manner. For example, if you are writing to a file and doing buffered writes, we can just copy the buffer to the page pool and move on, no actual I/O took place. So the IO Ring will run through that directly in a synchronous manner.

However, if your operation requires actual blocking, it will be sent to a worker queue to actually execute it in the background. This is one way that the IO Ring is able to complete many operations so much more efficiently than the alternatives.

In our scenario, we have a pretty simple setup, we want to write to the file, making fully buffered writes. At the very least, being able to push all the writes to the OS in one shot (versus many separate system calls) is going to reduce our overhead. More interesting, however, is that eventually, the OS will want to start writing to the disk, so if we write a lot of data, some of the requests will be blocked. At that point, the IO Ring will switch them to a worker thread and continue executing.

The problem we had was that when we had a separate IO Ring per data file and put a lot of load on the system, we started seeing contention between the worker threads across all the files. Basically, each ring had its own separate pool, so there was a lot of work for each pool but no sharing.

If the IO Ring is single-threaded, but many separate threads lead to wasted resources, what can we do? The answer is simple, we’ll use a single global IO Ring and manage the threading concerns directly.

Here is the setup code for that (I removed all error handling to make it clearer):


void *do_ring_work(void *arg)
{
  int rc;
  if (g_cfg.low_priority_io)
  {
    syscall(SYS_ioprio_set, IOPRIO_WHO_PROCESS, 0, 
        IOPRIO_PRIO_VALUE(IOPRIO_CLASS_BE, 7));
  }
  pthread_setname_np(pthread_self(), "Rvn.Ring.Wrkr");
  struct io_uring *ring = &g_worker.ring;
  struct workitem *work = NULL;
  while (true)
  {
    do
    {
      // wait for any writes on the eventfd 
      // completion on the ring (associated with the eventfd)
      eventfd_t v;
      rc = read(g_worker.eventfd, &v, sizeof(eventfd_t));
    } while (rc < 0 && errno == EINTR);
    
    bool has_work = true;
    while (has_work)
    {
      int must_wait = 0;
      has_work = false;
      if (!work) 
      {
        // we may have _previous_ work to run through
        work = atomic_exchange(&g_worker.head, 0);
      }
      while (work)
      {
        has_work = true;


        struct io_uring_sqe *sqe = io_uring_get_sqe(ring);
        if (sqe == NULL)
        {
          must_wait = 1;
          goto sumbit_and_wait; // will retry
        }
        io_uring_sqe_set_data(sqe, work);
        switch (work->type)
        {
        case workitem_fsync:
          io_uring_prep_fsync(sqe, work->filefd, IORING_FSYNC_DATASYNC);
          break;
        case workitem_write:
          io_uring_prep_writev(sqe, work->filefd, work->op.write.iovecs,
                               work->op.write.iovecs_count, work->offset);
          break;
        default:
          break;
        }
        work = work->next;
      }
    sumbit_and_wait:
      rc = must_wait ? 
        io_uring_submit_and_wait(ring, must_wait) : 
        io_uring_submit(ring);
      struct io_uring_cqe *cqe;
      uint32_t head = 0;
      uint32_t i = 0;


      io_uring_for_each_cqe(ring, head, cqe)
      {
        i++;
        // force another run of the inner loop, 
        // to ensure that we call io_uring_submit again
        has_work = true; 
        struct workitem *cur = io_uring_cqe_get_data(cqe);
        if (!cur)
        {
          // can be null if it is:
          // *  a notification about eventfd write
          continue;
        }
        switch (cur->type)
        {
        case workitem_fsync:
          notify_work_completed(ring, cur);
          break;
        case workitem_write:
          if (/* partial write */)
          {
            // queue again
            continue;
          }
          notify_work_completed(ring, cur);
          break;
        }
      }
      io_uring_cq_advance(ring, i);
    }
  }
  return 0;
}

What does this code do?

We start by checking if we want to use lower-priority I/O, this is because we don’t actually care how long those operations take. The purpose of writing the data to the disk is that it will reach it eventually. RavenDB has two types of writes:

  • Journal writes (durable update to the write-ahead log, required to complete a transaction).
  • Data flush / Data sync (background updates to the data file, currently buffered in memory, no user is waiting for that)

As such, we are fine with explicitly prioritizing the journal writes (which users are waiting for) in favor of all other operations.

What is this C code? I thought RavenDB was written in C#

RavenDB is written in C#, but for very low-level system details, we found that it is far easier to write a Platform Abstraction Layer to hide system-specific concerns from the rest of the code. That way, we can simply submit the data to write and have the abstraction layer take care of all of that for us. This also ensures that we amortize the cost of PInvoke calls across many operations by submitting a big batch to the C code at once.

After setting the IO priority, we start reading from what is effectively a thread-safe queue. We wait for eventfd() to signal that there is work to do, and then we grab the head of the queue and start running.

The idea is that we fetch items from the queue, then we write those operations to the IO Ring as fast as we can manage. The IO Ring size is limited, however. So we need to handle the case where we have more work for the IO Ring than it can accept. When that happens, we will go to the submit_and_wait label and wait for something to complete.

Note that there is some logic there to handle what is going on when the IO Ring is full. We submit all the work in the ring, wait for an operation to complete, and in the next run, we’ll continue processing from where we left off.

The rest of the code is processing the completed operations and reporting the result back to their origin. This is done using the following function, which I find absolutely hilarious:


int32_t rvn_write_io_ring(
    void *handle,
    struct page_to_write *buffers,
    int32_t count,
    int32_t *detailed_error_code)
{
    int32_t rc = SUCCESS;
    struct handle *handle_ptr = handle;
    if (count == 0)
        return SUCCESS;


    if (pthread_mutex_lock(&handle_ptr->global_state->writes_arena.lock))
    {
        *detailed_error_code = errno;
        return FAIL_MUTEX_LOCK;
    }
    size_t max_req_size = (size_t)count * 
                      (sizeof(struct iovec) + sizeof(struct workitem));
    if (handle_ptr->global_state->writes_arena.arena_size < max_req_size)
    {
        // allocate arena space
    }
    void *buf = handle_ptr->global_state->writes_arena.arena;
    struct workitem *prev = NULL;
    int eventfd = handle_ptr->global_state->writes_arena.eventfd;
    for (int32_t curIdx = 0; curIdx < count; curIdx++)
    {
        int64_t offset = buffers[curIdx].page_num * VORON_PAGE_SIZE;
        int64_t size = (int64_t)buffers[curIdx].count_of_pages *
                       VORON_PAGE_SIZE;
        int64_t after = offset + size;


        struct workitem *work = buf;
        *work = (struct workitem){
            .op.write.iovecs_count = 1,
            .op.write.iovecs = buf + sizeof(struct workitem),
            .completed = 0,
            .type = workitem_write,
            .filefd = handle_ptr->file_fd,
            .offset = offset,
            .errored = false,
            .result = 0,
            .prev = prev,
            .notifyfd = eventfd,
        };
        prev = work;
        work->op.write.iovecs[0] = (struct iovec){
            .iov_len = size, 
            .iov_base = buffers[curIdx].ptr
        };
        buf += sizeof(struct workitem) + sizeof(struct iovec);


        for (size_t nextIndex = curIdx + 1; 
            nextIndex < count && work->op.write.iovecs_count < IOV_MAX; 
            nextIndex++)
        {
            int64_t dest = buffers[nextIndex].page_num * VORON_PAGE_SIZE;
            if (after != dest)
                break;


            size = (int64_t)buffers[nextIndex].count_of_pages *
                              VORON_PAGE_SIZE;
            after = dest + size;
            work->op.write.iovecs[work->op.write.iovecs_count++] = 
                (struct iovec){
                .iov_base = buffers[nextIndex].ptr,
                .iov_len = size,
            };
            curIdx++;
            buf += sizeof(struct iovec);
        }
        queue_work(work);
    }
    rc = wait_for_work_completion(handle_ptr, prev, eventfd, 
detailed_error_code);
    pthread_mutex_unlock(&handle_ptr->global_state->writes_arena.lock)
    return rc;
}

Remember that when we submit writes to the data file, we must wait until they are all done. The async nature of IO Ring is meant to help us push the writes to the OS as soon as possible, as well as push writes to multiple separate files at once. For that reason, we use anothereventfd() to wait (as the submitter) for the IO Ring to complete the operation. I love the code above because it is actually using the IO Ring itself to do the work we need to do here, saving us an actual system call in most cases.

Here is how we submit the work to the worker thread:


void queue_work(struct workitem *work)
{
    struct workitem *head = atomic_load(&g_worker.head);
    do
    {
        work->next = head;
    } while (!atomic_compare_exchange_weak(&g_worker.head, &head, work));
}

This function handles the submission of a set of pages to write to a file. Note that we protect against concurrent work on the same file. That isn’t actually needed since the caller code already handles that, but an uncontended lock is cheap, and it means that I don’t need to think about concurrency or worry about changes in the caller code in the future.

We ensure that we have sufficient buffer space, and then we create a work item. A work item is a single write to the file at a given location. However, we are using vectored writes, so we’ll merge writes to the consecutive pages into a single write operation. That is the purpose of the huge for loop in the code. The pages arrive already sorted, so we just need to do a single scan & merge for this.

Pay attention to the fact that the struct workitem actually belongs to two different linked lists. We have the next pointer, which is used to send work to the worker thread, and the prev pointer, which is used to iterate over the entire set of operations we submitted on completion (we’ll cover this in a bit).

Queuing work is done using the following method:


int32_t
wait_for_work_completion(struct handle *handle_ptr, 
    struct workitem *prev, 
    int eventfd, 
    int32_t *detailed_error_code)
{
    // wake worker thread
    eventfd_write(g_worker.eventfd, 1);
    
    bool all_done = false;
    while (!all_done)
    {
        all_done = true;
        *detailed_error_code = 0;


        eventfd_t v;
        int rc = read(eventfd, &v, sizeof(eventfd_t));
        struct workitem *work = prev;
        while (work)
        {
            all_done &= atomic_load(&work->completed);
            work = work->prev;
        }
    }
    return SUCCESS;
}

The idea is pretty simple. We first wake the worker thread by writing to its eventfd(), and then we wait on our own eventfd() for the worker to signal us that (at least some) of the work is done.

Note that we handle the submission of multiple work items by iterating over them in reverse order, using the prev pointer. Only when all the work is done can we return to our caller.

The end result of all this behavior is that we have a completely new way to deal with background I/O operations (remember, journal writes are handled differently). We can control both the volume of load we put on the system by adjusting the size of the IO Ring as well as changing its priority.

The fact that we have a single global IO Ring means that we can get much better usage out of the worker thread pool that IO Ring utilizes. We also give the OS a lot more opportunities to optimize RavenDB’s I/O.

The code in this post shows the Linux implementation, but RavenDB also supports IO Ring on Windows if you are running a recent edition.

We aren’t done yet, mind, I still have more exciting things to tell you about how RavenDB 7.1 is optimizing writes and overall performance. In the next post, we’ll discuss what I call the High Occupancy Lane vs. Critical Lane for I/O and its impact on our performance.