Skip to content

Commit

Permalink
Revert "revert recent VirtualFile asyncification changes (#5291)" (#6309
Browse files Browse the repository at this point in the history
)

This reverts commit ab1f37e.
Thereby
fixes #5479

Updated Analysis
================

The problem with the original patch was that it, for the first time,
exposed the `VirtualFile` code to tokio task concurrency instead of just
thread-based concurrency. That caused the VirtualFile file descriptor
cache to start thrashing, effectively grinding the system to a halt.

Details
-------

At the time of the original patch, we had a _lot_ of runnable tasks in
the pageserver.
The symptom that prompted the revert (now being reverted in this PR) is
that our production systems fell into a valley of zero goodput, high
CPU, and zero disk IOPS shortly after PS restart.
We lay out the root cause for that behavior in this subsection.

At the time, there was no concurrency limit on the number of concurrent
initial logical size calculations.
Initial size calculation was initiated for all timelines within the
first 10 minutes as part of consumption metrics collection.
On a PS with 20k timelines, we'd thus have 20k runnable tasks.

Before the original patch, the `VirtualFile` code never returned
`Poll::Pending`.
That meant that once we entered it, the calling tokio task would not
yield to the tokio executor until we were done performing the
VirtualFile operation, i.e., doing a blocking IO system call.

The original patch switched the VirtualFile file descriptor cache's
synchronization primitives to those from `tokio::sync`.
It did not change that we were doing synchronous IO system calls.
And the cache had more slots than we have tokio executor threads.
So, these primitives never actually needed to return `Poll::Pending`.
But, the tokio scheduler makes tokio sync primitives return `Pending`
*artificially*, as a mechanism for the scheduler to get back into
control more often
([example](https://docs.rs/tokio/1.35.1/src/tokio/sync/batch_semaphore.rs.html#570)).

So, the new reality was that VirtualFile calls could now yield to the
tokio executor.
Tokio would pick one of the other 19999 runnable tasks to run.
These tasks were also using VirtualFile.
So, we now had a lot more concurrency in that area of the code.

The problem with more concurrency was that caches started thrashing,
most notably the VirtualFile file descriptor cache: each time a task
would be rescheduled, it would want to do its next VirtualFile
operation. For that, it would first need to evict another (task's)
VirtualFile fd from the cache to make room for its own fd. It would then
do one VirtualFile operation before hitting an await point and yielding
to the executor again. The executor would run the other 19999 tasks for
fairness before circling back to the first task, which would find its fd
evicted.

The other cache that would theoretically be impacted in a similar way is
the pageserver's `PageCache`.
However, for initial logical size calculation, it seems much less
relevant in experiments, likely because of the random access nature of
initial logical size calculation.

Fixes
=====

We fixed the above problems by
- raising VirtualFile cache sizes
  - neondatabase/cloud#8351
- changing code to ensure forward-progress once cache slots have been
acquired
  - #5480
  - #5482
  - tbd: #6065
- reducing the amount of runnable tokio tasks
  - #5578
  - #6000
- fix bugs that caused unnecessary concurrency induced by connection
handlers
  - #5993

I manually verified that this PR doesn't negatively affect startup
performance as follows:
create a pageserver in production configuration, with 20k
tenants/timelines, 9 tiny L0 layer files each; Start it, and observe

```
INFO Startup complete (368.009s since start) elapsed_ms=368009
```

I further verified in that same setup that, when using `pagebench`'s
getpage benchmark at as-fast-as-possible request rate against 5k of the
20k tenants, the achieved throughput is identical. The VirtualFile cache
isn't thrashing in that case.

Future Work
===========

We will still exposed to the cache thrashing risk from outside factors,
e.g., request concurrency is unbounded, and initial size calculation
skips the concurrency limiter when we establish a walreceiver
connection.

Once we start thrashing, we will degrade non-gracefully, i.e., encounter
a valley as was seen with the original patch.

However, we have sufficient means to deal with that unlikely situation:
1. we have dashboards & metrics to monitor & alert on cache thrashing
2. we can react by scaling the bottleneck resources (cache size) or by
manually shedding load through tenant relocation

Potential systematic solutions are future work:
* global concurrency limiting
* per-tenant rate limiting => #5899
* pageserver-initiated load shedding

Related Issues
==============

This PR unblocks the introduction of tokio-epoll-uring for asynchronous
disk IO ([Epic](#4744)).
  • Loading branch information
problame committed Jan 11, 2024
1 parent 544284c commit fc66ba4
Showing 1 changed file with 111 additions and 64 deletions.
175 changes: 111 additions & 64 deletions pageserver/src/virtual_file.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,8 @@ use std::fs::{self, File, OpenOptions};
use std::io::{Error, ErrorKind, Seek, SeekFrom};
use std::os::unix::fs::FileExt;
use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering};
use std::sync::{RwLock, RwLockWriteGuard};
use tokio::sync::{RwLock, RwLockReadGuard, RwLockWriteGuard};
use tokio::time::Instant;
use utils::fs_ext;

///
Expand Down Expand Up @@ -111,7 +112,7 @@ impl OpenFiles {
///
/// On return, we hold a lock on the slot, and its 'tag' has been updated
/// recently_used has been set. It's all ready for reuse.
fn find_victim_slot(&self) -> (SlotHandle, RwLockWriteGuard<SlotInner>) {
async fn find_victim_slot(&self) -> (SlotHandle, RwLockWriteGuard<SlotInner>) {
//
// Run the clock algorithm to find a slot to replace.
//
Expand Down Expand Up @@ -143,7 +144,7 @@ impl OpenFiles {
}
retries += 1;
} else {
slot_guard = slot.inner.write().unwrap();
slot_guard = slot.inner.write().await;
index = next;
break;
}
Expand Down Expand Up @@ -250,6 +251,29 @@ impl<T> MaybeFatalIo<T> for std::io::Result<T> {
}
}

/// Observe duration for the given storage I/O operation
///
/// Unlike `observe_closure_duration`, this supports async,
/// where "support" means that we measure wall clock time.
macro_rules! observe_duration {
($op:expr, $($body:tt)*) => {{
let instant = Instant::now();
let result = $($body)*;
let elapsed = instant.elapsed().as_secs_f64();
STORAGE_IO_TIME_METRIC
.get($op)
.observe(elapsed);
result
}}
}

macro_rules! with_file {
($this:expr, $op:expr, | $ident:ident | $($body:tt)*) => {{
let $ident = $this.lock_file().await?;
observe_duration!($op, $($body)*)
}};
}

impl VirtualFile {
/// Open a file in read-only mode. Like File::open.
pub async fn open(path: &Utf8Path) -> Result<VirtualFile, std::io::Error> {
Expand Down Expand Up @@ -286,14 +310,12 @@ impl VirtualFile {
tenant_id = "*".to_string();
timeline_id = "*".to_string();
}
let (handle, mut slot_guard) = get_open_files().find_victim_slot();
let (handle, mut slot_guard) = get_open_files().find_victim_slot().await;

// NB: there is also StorageIoOperation::OpenAfterReplace which is for the case
// where our caller doesn't get to use the returned VirtualFile before its
// slot gets re-used by someone else.
let file = STORAGE_IO_TIME_METRIC
.get(StorageIoOperation::Open)
.observe_closure_duration(|| open_options.open(path))?;
let file = observe_duration!(StorageIoOperation::Open, open_options.open(path))?;

// Strip all options other than read and write.
//
Expand Down Expand Up @@ -366,22 +388,24 @@ impl VirtualFile {

/// Call File::sync_all() on the underlying File.
pub async fn sync_all(&self) -> Result<(), Error> {
self.with_file(StorageIoOperation::Fsync, |file| file.sync_all())
.await?
with_file!(self, StorageIoOperation::Fsync, |file| file
.as_ref()
.sync_all())
}

pub async fn metadata(&self) -> Result<fs::Metadata, Error> {
self.with_file(StorageIoOperation::Metadata, |file| file.metadata())
.await?
with_file!(self, StorageIoOperation::Metadata, |file| file
.as_ref()
.metadata())
}

/// Helper function that looks up the underlying File for this VirtualFile,
/// opening it and evicting some other File if necessary. It calls 'func'
/// with the physical File.
async fn with_file<F, R>(&self, op: StorageIoOperation, mut func: F) -> Result<R, Error>
where
F: FnMut(&File) -> R,
{
/// Helper function internal to `VirtualFile` that looks up the underlying File,
/// opens it and evicts some other File if necessary. The passed parameter is
/// assumed to be a function available for the physical `File`.
///
/// We are doing it via a macro as Rust doesn't support async closures that
/// take on parameters with lifetimes.
async fn lock_file(&self) -> Result<FileGuard<'_>, Error> {
let open_files = get_open_files();

let mut handle_guard = {
Expand All @@ -391,27 +415,23 @@ impl VirtualFile {
// We only need to hold the handle lock while we read the current handle. If
// another thread closes the file and recycles the slot for a different file,
// we will notice that the handle we read is no longer valid and retry.
let mut handle = *self.handle.read().unwrap();
let mut handle = *self.handle.read().await;
loop {
// Check if the slot contains our File
{
let slot = &open_files.slots[handle.index];
let slot_guard = slot.inner.read().unwrap();
if slot_guard.tag == handle.tag {
if let Some(file) = &slot_guard.file {
// Found a cached file descriptor.
slot.recently_used.store(true, Ordering::Relaxed);
return Ok(STORAGE_IO_TIME_METRIC
.get(op)
.observe_closure_duration(|| func(file)));
}
let slot_guard = slot.inner.read().await;
if slot_guard.tag == handle.tag && slot_guard.file.is_some() {
// Found a cached file descriptor.
slot.recently_used.store(true, Ordering::Relaxed);
return Ok(FileGuard { slot_guard });
}
}

// The slot didn't contain our File. We will have to open it ourselves,
// but before that, grab a write lock on handle in the VirtualFile, so
// that no other thread will try to concurrently open the same file.
let handle_guard = self.handle.write().unwrap();
let handle_guard = self.handle.write().await;

// If another thread changed the handle while we were not holding the lock,
// then the handle might now be valid again. Loop back to retry.
Expand All @@ -425,28 +445,26 @@ impl VirtualFile {

// We need to open the file ourselves. The handle in the VirtualFile is
// now locked in write-mode. Find a free slot to put it in.
let (handle, mut slot_guard) = open_files.find_victim_slot();
let (handle, mut slot_guard) = open_files.find_victim_slot().await;

// Re-open the physical file.
// NB: we use StorageIoOperation::OpenAferReplace for this to distinguish this
// case from StorageIoOperation::Open. This helps with identifying thrashing
// of the virtual file descriptor cache.
let file = STORAGE_IO_TIME_METRIC
.get(StorageIoOperation::OpenAfterReplace)
.observe_closure_duration(|| self.open_options.open(&self.path))?;

// Perform the requested operation on it
let result = STORAGE_IO_TIME_METRIC
.get(op)
.observe_closure_duration(|| func(&file));
let file = observe_duration!(
StorageIoOperation::OpenAfterReplace,
self.open_options.open(&self.path)
)?;

// Store the File in the slot and update the handle in the VirtualFile
// to point to it.
slot_guard.file.replace(file);

*handle_guard = handle;

Ok(result)
return Ok(FileGuard {
slot_guard: slot_guard.downgrade(),
});
}

pub fn remove(self) {
Expand All @@ -461,11 +479,9 @@ impl VirtualFile {
self.pos = offset;
}
SeekFrom::End(offset) => {
self.pos = self
.with_file(StorageIoOperation::Seek, |mut file| {
file.seek(SeekFrom::End(offset))
})
.await??
self.pos = with_file!(self, StorageIoOperation::Seek, |file| file
.as_ref()
.seek(SeekFrom::End(offset)))?
}
SeekFrom::Current(offset) => {
let pos = self.pos as i128 + offset as i128;
Expand Down Expand Up @@ -553,9 +569,9 @@ impl VirtualFile {
}

pub async fn read_at(&self, buf: &mut [u8], offset: u64) -> Result<usize, Error> {
let result = self
.with_file(StorageIoOperation::Read, |file| file.read_at(buf, offset))
.await?;
let result = with_file!(self, StorageIoOperation::Read, |file| file
.as_ref()
.read_at(buf, offset));
if let Ok(size) = result {
STORAGE_IO_SIZE
.with_label_values(&["read", &self.tenant_id, &self.timeline_id])
Expand All @@ -565,9 +581,9 @@ impl VirtualFile {
}

async fn write_at(&self, buf: &[u8], offset: u64) -> Result<usize, Error> {
let result = self
.with_file(StorageIoOperation::Write, |file| file.write_at(buf, offset))
.await?;
let result = with_file!(self, StorageIoOperation::Write, |file| file
.as_ref()
.write_at(buf, offset));
if let Ok(size) = result {
STORAGE_IO_SIZE
.with_label_values(&["write", &self.tenant_id, &self.timeline_id])
Expand All @@ -577,6 +593,18 @@ impl VirtualFile {
}
}

struct FileGuard<'a> {
slot_guard: RwLockReadGuard<'a, SlotInner>,
}

impl<'a> AsRef<File> for FileGuard<'a> {
fn as_ref(&self) -> &File {
// This unwrap is safe because we only create `FileGuard`s
// if we know that the file is Some.
self.slot_guard.file.as_ref().unwrap()
}
}

#[cfg(test)]
impl VirtualFile {
pub(crate) async fn read_blk(
Expand Down Expand Up @@ -609,22 +637,41 @@ impl VirtualFile {
impl Drop for VirtualFile {
/// If a VirtualFile is dropped, close the underlying file if it was open.
fn drop(&mut self) {
let handle = self.handle.get_mut().unwrap();

// We could check with a read-lock first, to avoid waiting on an
// unrelated I/O.
let slot = &get_open_files().slots[handle.index];
let mut slot_guard = slot.inner.write().unwrap();
if slot_guard.tag == handle.tag {
slot.recently_used.store(false, Ordering::Relaxed);
// there is also operation "close-by-replace" for closes done on eviction for
// comparison.
if let Some(fd) = slot_guard.file.take() {
STORAGE_IO_TIME_METRIC
.get(StorageIoOperation::Close)
.observe_closure_duration(|| drop(fd));
let handle = self.handle.get_mut();

fn clean_slot(slot: &Slot, mut slot_guard: RwLockWriteGuard<'_, SlotInner>, tag: u64) {
if slot_guard.tag == tag {
slot.recently_used.store(false, Ordering::Relaxed);
// there is also operation "close-by-replace" for closes done on eviction for
// comparison.
if let Some(fd) = slot_guard.file.take() {
STORAGE_IO_TIME_METRIC
.get(StorageIoOperation::Close)
.observe_closure_duration(|| drop(fd));
}
}
}

// We don't have async drop so we cannot directly await the lock here.
// Instead, first do a best-effort attempt at closing the underlying
// file descriptor by using `try_write`, and if that fails, spawn
// a tokio task to do it asynchronously: we just want it to be
// cleaned up eventually.
// Most of the time, the `try_lock` should succeed though,
// as we have `&mut self` access. In other words, if the slot
// is still occupied by our file, there should be no access from
// other I/O operations; the only other possible place to lock
// the slot is the lock algorithm looking for free slots.
let slot = &get_open_files().slots[handle.index];
if let Ok(slot_guard) = slot.inner.try_write() {
clean_slot(slot, slot_guard, handle.tag);
} else {
let tag = handle.tag;
tokio::spawn(async move {
let slot_guard = slot.inner.write().await;
clean_slot(slot, slot_guard, tag);
});
};
}
}

Expand Down

1 comment on commit fc66ba4

@github-actions
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

2306 tests run: 2213 passed, 0 failed, 93 skipped (full report)


Flaky tests (1)

Postgres 15

  • test_empty_branch_remote_storage_upload: release

Test coverage report is not available

The comment gets automatically updated with the latest test results
fc66ba4 at 2024-01-11T11:21:57.335Z :recycle:

Please sign in to comment.