Skip to content

Commit

Permalink
add async walredo mode (disabled-by-default, opt-in via config) (#6548
Browse files Browse the repository at this point in the history
)

Before this PR, the `nix::poll::poll` call would stall the executor.

This PR refactors the `walredo::process` module to allow for different
implementations, and adds a new `async` implementation which uses
`tokio::process::ChildStd{in,out}` for IPC.

The `sync` variant remains the default for now; we'll do more testing in
staging and gradual rollout to prod using the config variable.

Performance
-----------

I updated `bench_walredo.rs`, demonstrating that a single `async`-based
walredo manager used by N=1...128 tokio tasks has lower latency and
higher throughput.

I further did manual less-micro-benchmarking in the real pageserver
binary.
Methodology & results are published here:

https://neondatabase.notion.site/2024-04-08-async-walredo-benchmarking-8c0ed3cc8d364a44937c4cb50b6d7019?pvs=4

tl;dr:
- use pagebench against a pageserver patched to answer getpage request &
small-enough working set to fit into PS PageCache / kernel page cache.
- compare knee in the latency/throughput curve
    - N tenants, each 1 pagebench clients
    - sync better throughput at N < 30, async better at higher N
    - async generally noticable but not much worse p99.X tail latencies
- eyeballing CPU efficiency in htop, `async` seems significantly more
CPU efficient at ca N=[0.5*ncpus, 1.5*ncpus], worse than `sync` outside
of that band

Mental Model For Walredo & Scheduler Interactions
-------------------------------------------------

Walredo is CPU-/DRAM-only work.
This means that as soon as the Pageserver writes to the pipe, the
walredo process becomes runnable.

To the Linux kernel scheduler, the `$ncpus` executor threads and the
walredo process thread are just `struct task_struct`, and it will divide
CPU time fairly among them.

In `sync` mode, there are always `$ncpus` runnable `struct task_struct`
because the executor thread blocks while `walredo` runs, and the
executor thread becomes runnable when the `walredo` process is done
handling the request.
In `async` mode, the executor threads remain runnable unless there are
no more runnable tokio tasks, which is unlikely in a production
pageserver.

The above means that in `sync` mode, there is an implicit concurrency
limit on concurrent walredo requests (`$num_runtimes *
$num_executor_threads_per_runtime`).
And executor threads do not compete in the Linux kernel scheduler for
CPU time, due to the blocked-runnable-ping-pong.
In `async` mode, there is no concurrency limit, and the walredo tasks
compete with the executor threads for CPU time in the kernel scheduler.

If we're not CPU-bound, `async` has a pipelining and hence throughput
advantage over `sync` because one executor thread can continue
processing requests while a walredo request is in flight.

If we're CPU-bound, under a fair CPU scheduler, the *fixed* number of
executor threads has to share CPU time with the aggregate of walredo
processes.
It's trivial to reason about this in `sync` mode due to the
blocked-runnable-ping-pong.
In `async` mode, at 100% CPU, the system arrives at some (potentially
sub-optiomal) equilibrium where the executor threads get just enough CPU
time to fill up the remaining CPU time with runnable walredo process.

Why `async` mode Doesn't Limit Walredo Concurrency
--------------------------------------------------

To control that equilibrium in `async` mode, one may add a tokio
semaphore to limit the number of in-flight walredo requests.
However, the placement of such a semaphore is non-trivial because it
means that tasks queuing up behind it hold on to their request-scoped
allocations.
In the case of walredo, that might be the entire reconstruct data.
We don't limit the number of total inflight Timeline::get (we only
throttle admission).
So, that queue might lead to an OOM.

The alternative is to acquire the semaphore permit *before* collecting
reconstruct data.
However, what if we need to on-demand download?

A combination of semaphores might help: one for reconstruct data, one
for walredo.
The reconstruct data semaphore permit is dropped after acquiring the
walredo semaphore permit.
This scheme effectively enables both a limit on in-flight reconstruct
data and walredo concurrency.

However, sizing the amount of permits for the semaphores is tricky:
- Reconstruct data retrieval is a mix of disk IO and CPU work.
- If we need to do on-demand downloads, it's network IO + disk IO + CPU
work.
- At this time, we have no good data on how the wall clock time is
distributed.

It turns out that, in my benchmarking, the system worked fine without a
semaphore. So, we're shipping async walredo without one for now.

Future Work
-----------

We will do more testing of `async` mode and gradual rollout to prod
using the config flag.
Once that is done, we'll remove `sync` mode to avoid the temporary code
duplication introduced by this PR.
The flag will be removed.

The `wait()` for the child process to exit is still synchronous; the
comment [here](
https://github.com/neondatabase/neon/blob/655d3b64681b6562530665c9ab5f2f806f30ad01/pageserver/src/walredo.rs#L294-L306)
is still a valid argument in favor of that.

The `sync` mode had another implicit advantage: from tokio's
perspective, the calling task was using up coop budget.
But with `async` mode, that's no longer the case -- to tokio, the writes
to the child process pipe look like IO.
We could/should inform tokio about the CPU time budget consumed by the
task to achieve fairness similar to `sync`.
However, the [runtime function for this is
`tokio_unstable`](`https://docs.rs/tokio/latest/tokio/task/fn.consume_budget.html).


Refs
----

refs #6628 
refs #2975
  • Loading branch information
problame authored Apr 15, 2024
1 parent 110282e commit 2d5a846
Show file tree
Hide file tree
Showing 13 changed files with 1,185 additions and 456 deletions.
10 changes: 9 additions & 1 deletion libs/pageserver_api/src/models.rs
Original file line number Diff line number Diff line change
Expand Up @@ -747,10 +747,18 @@ pub struct TimelineGcRequest {
pub gc_horizon: Option<u64>,
}

#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct WalRedoManagerProcessStatus {
pub pid: u32,
/// The strum-generated `into::<&'static str>()` for `pageserver::walredo::ProcessKind`.
/// `ProcessKind` are a transitory thing, so, they have no enum representation in `pageserver_api`.
pub kind: Cow<'static, str>,
}

#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct WalRedoManagerStatus {
pub last_redo_at: Option<chrono::DateTime<chrono::Utc>>,
pub pid: Option<u32>,
pub process: Option<WalRedoManagerProcessStatus>,
}

/// The progress of a secondary tenant is mostly useful when doing a long running download: e.g. initiating
Expand Down
2 changes: 2 additions & 0 deletions libs/utils/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,8 @@ pub mod zstd;

pub mod env;

pub mod poison;

/// This is a shortcut to embed git sha into binaries and avoid copying the same build script to all packages
///
/// we have several cases:
Expand Down
121 changes: 121 additions & 0 deletions libs/utils/src/poison.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
//! Protect a piece of state from reuse after it is left in an inconsistent state.
//!
//! # Example
//!
//! ```
//! # tokio_test::block_on(async {
//! use utils::poison::Poison;
//! use std::time::Duration;
//!
//! struct State {
//! clean: bool,
//! }
//! let state = tokio::sync::Mutex::new(Poison::new("mystate", State { clean: true }));
//!
//! let mut mutex_guard = state.lock().await;
//! let mut poison_guard = mutex_guard.check_and_arm()?;
//! let state = poison_guard.data_mut();
//! state.clean = false;
//! // If we get cancelled at this await point, subsequent check_and_arm() calls will fail.
//! tokio::time::sleep(Duration::from_secs(10)).await;
//! state.clean = true;
//! poison_guard.disarm();
//! # Ok::<(), utils::poison::Error>(())
//! # });
//! ```
use tracing::warn;

pub struct Poison<T> {
what: &'static str,
state: State,
data: T,
}

#[derive(Clone, Copy)]
enum State {
Clean,
Armed,
Poisoned { at: chrono::DateTime<chrono::Utc> },
}

impl<T> Poison<T> {
/// We log `what` `warning!` level if the [`Guard`] gets dropped without being [`Guard::disarm`]ed.
pub fn new(what: &'static str, data: T) -> Self {
Self {
what,
state: State::Clean,
data,
}
}

/// Check for poisoning and return a [`Guard`] that provides access to the wrapped state.
pub fn check_and_arm(&mut self) -> Result<Guard<T>, Error> {
match self.state {
State::Clean => {
self.state = State::Armed;
Ok(Guard(self))
}
State::Armed => unreachable!("transient state"),
State::Poisoned { at } => Err(Error::Poisoned {
what: self.what,
at,
}),
}
}
}

/// Use [`Self::data`] and [`Self::data_mut`] to access the wrapped state.
/// Once modifications are done, use [`Self::disarm`].
/// If [`Guard`] gets dropped instead of calling [`Self::disarm`], the state is poisoned
/// and subsequent calls to [`Poison::check_and_arm`] will fail with an error.
pub struct Guard<'a, T>(&'a mut Poison<T>);

impl<'a, T> Guard<'a, T> {
pub fn data(&self) -> &T {
&self.0.data
}
pub fn data_mut(&mut self) -> &mut T {
&mut self.0.data
}

pub fn disarm(self) {
match self.0.state {
State::Clean => unreachable!("we set it to Armed in check_and_arm()"),
State::Armed => {
self.0.state = State::Clean;
}
State::Poisoned { at } => {
unreachable!("we fail check_and_arm() if it's in that state: {at}")
}
}
}
}

impl<'a, T> Drop for Guard<'a, T> {
fn drop(&mut self) {
match self.0.state {
State::Clean => {
// set by disarm()
}
State::Armed => {
// still armed => poison it
let at = chrono::Utc::now();
self.0.state = State::Poisoned { at };
warn!(at=?at, "poisoning {}", self.0.what);
}
State::Poisoned { at } => {
unreachable!("we fail check_and_arm() if it's in that state: {at}")
}
}
}
}

#[derive(thiserror::Error, Debug)]
pub enum Error {
#[error("poisoned at {at}: {what}")]
Poisoned {
what: &'static str,
at: chrono::DateTime<chrono::Utc>,
},
}
147 changes: 97 additions & 50 deletions pageserver/benches/bench_walredo.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,30 +27,50 @@
//!
//! # Reference Numbers
//!
//! 2024-04-04 on i3en.3xlarge
//! 2024-04-15 on i3en.3xlarge
//!
//! ```text
//! short/1 time: [25.925 µs 26.060 µs 26.209 µs]
//! short/2 time: [31.277 µs 31.483 µs 31.722 µs]
//! short/4 time: [45.496 µs 45.831 µs 46.182 µs]
//! short/8 time: [84.298 µs 84.920 µs 85.566 µs]
//! short/16 time: [185.04 µs 186.41 µs 187.88 µs]
//! short/32 time: [385.01 µs 386.77 µs 388.70 µs]
//! short/64 time: [770.24 µs 773.04 µs 776.04 µs]
//! short/128 time: [1.5017 ms 1.5064 ms 1.5113 ms]
//! medium/1 time: [106.65 µs 107.20 µs 107.85 µs]
//! medium/2 time: [153.28 µs 154.24 µs 155.56 µs]
//! medium/4 time: [325.67 µs 327.01 µs 328.71 µs]
//! medium/8 time: [646.82 µs 650.17 µs 653.91 µs]
//! medium/16 time: [1.2645 ms 1.2701 ms 1.2762 ms]
//! medium/32 time: [2.4409 ms 2.4550 ms 2.4692 ms]
//! medium/64 time: [4.6814 ms 4.7114 ms 4.7408 ms]
//! medium/128 time: [8.7790 ms 8.9037 ms 9.0282 ms]
//! async-short/1 time: [24.584 µs 24.737 µs 24.922 µs]
//! async-short/2 time: [33.479 µs 33.660 µs 33.888 µs]
//! async-short/4 time: [42.713 µs 43.046 µs 43.440 µs]
//! async-short/8 time: [71.814 µs 72.478 µs 73.240 µs]
//! async-short/16 time: [132.73 µs 134.45 µs 136.22 µs]
//! async-short/32 time: [258.31 µs 260.73 µs 263.27 µs]
//! async-short/64 time: [511.61 µs 514.44 µs 517.51 µs]
//! async-short/128 time: [992.64 µs 998.23 µs 1.0042 ms]
//! async-medium/1 time: [110.11 µs 110.50 µs 110.96 µs]
//! async-medium/2 time: [153.06 µs 153.85 µs 154.99 µs]
//! async-medium/4 time: [317.51 µs 319.92 µs 322.85 µs]
//! async-medium/8 time: [638.30 µs 644.68 µs 652.12 µs]
//! async-medium/16 time: [1.2651 ms 1.2773 ms 1.2914 ms]
//! async-medium/32 time: [2.5117 ms 2.5410 ms 2.5720 ms]
//! async-medium/64 time: [4.8088 ms 4.8555 ms 4.9047 ms]
//! async-medium/128 time: [8.8311 ms 8.9849 ms 9.1263 ms]
//! sync-short/1 time: [25.503 µs 25.626 µs 25.771 µs]
//! sync-short/2 time: [30.850 µs 31.013 µs 31.208 µs]
//! sync-short/4 time: [45.543 µs 45.856 µs 46.193 µs]
//! sync-short/8 time: [84.114 µs 84.639 µs 85.220 µs]
//! sync-short/16 time: [185.22 µs 186.15 µs 187.13 µs]
//! sync-short/32 time: [377.43 µs 378.87 µs 380.46 µs]
//! sync-short/64 time: [756.49 µs 759.04 µs 761.70 µs]
//! sync-short/128 time: [1.4825 ms 1.4874 ms 1.4923 ms]
//! sync-medium/1 time: [105.66 µs 106.01 µs 106.43 µs]
//! sync-medium/2 time: [153.10 µs 153.84 µs 154.72 µs]
//! sync-medium/4 time: [327.13 µs 329.44 µs 332.27 µs]
//! sync-medium/8 time: [654.26 µs 658.73 µs 663.63 µs]
//! sync-medium/16 time: [1.2682 ms 1.2748 ms 1.2816 ms]
//! sync-medium/32 time: [2.4456 ms 2.4595 ms 2.4731 ms]
//! sync-medium/64 time: [4.6523 ms 4.6890 ms 4.7256 ms]
//! sync-medium/128 time: [8.7215 ms 8.8323 ms 8.9344 ms]
//! ```
use bytes::{Buf, Bytes};
use criterion::{BenchmarkId, Criterion};
use pageserver::{config::PageServerConf, walrecord::NeonWalRecord, walredo::PostgresRedoManager};
use pageserver::{
config::PageServerConf,
walrecord::NeonWalRecord,
walredo::{PostgresRedoManager, ProcessKind},
};
use pageserver_api::{key::Key, shard::TenantShardId};
use std::{
sync::Arc,
Expand All @@ -60,44 +80,56 @@ use tokio::{sync::Barrier, task::JoinSet};
use utils::{id::TenantId, lsn::Lsn};

fn bench(c: &mut Criterion) {
{
let nclients = [1, 2, 4, 8, 16, 32, 64, 128];
for nclients in nclients {
let mut group = c.benchmark_group("short");
group.bench_with_input(
BenchmarkId::from_parameter(nclients),
&nclients,
|b, nclients| {
let redo_work = Arc::new(Request::short_input());
b.iter_custom(|iters| bench_impl(Arc::clone(&redo_work), iters, *nclients));
},
);
for process_kind in &[ProcessKind::Async, ProcessKind::Sync] {
{
let nclients = [1, 2, 4, 8, 16, 32, 64, 128];
for nclients in nclients {
let mut group = c.benchmark_group(format!("{process_kind}-short"));
group.bench_with_input(
BenchmarkId::from_parameter(nclients),
&nclients,
|b, nclients| {
let redo_work = Arc::new(Request::short_input());
b.iter_custom(|iters| {
bench_impl(*process_kind, Arc::clone(&redo_work), iters, *nclients)
});
},
);
}
}
}

{
let nclients = [1, 2, 4, 8, 16, 32, 64, 128];
for nclients in nclients {
let mut group = c.benchmark_group("medium");
group.bench_with_input(
BenchmarkId::from_parameter(nclients),
&nclients,
|b, nclients| {
let redo_work = Arc::new(Request::medium_input());
b.iter_custom(|iters| bench_impl(Arc::clone(&redo_work), iters, *nclients));
},
);
{
let nclients = [1, 2, 4, 8, 16, 32, 64, 128];
for nclients in nclients {
let mut group = c.benchmark_group(format!("{process_kind}-medium"));
group.bench_with_input(
BenchmarkId::from_parameter(nclients),
&nclients,
|b, nclients| {
let redo_work = Arc::new(Request::medium_input());
b.iter_custom(|iters| {
bench_impl(*process_kind, Arc::clone(&redo_work), iters, *nclients)
});
},
);
}
}
}
}
criterion::criterion_group!(benches, bench);
criterion::criterion_main!(benches);

// Returns the sum of each client's wall-clock time spent executing their share of the n_redos.
fn bench_impl(redo_work: Arc<Request>, n_redos: u64, nclients: u64) -> Duration {
fn bench_impl(
process_kind: ProcessKind,
redo_work: Arc<Request>,
n_redos: u64,
nclients: u64,
) -> Duration {
let repo_dir = camino_tempfile::tempdir_in(env!("CARGO_TARGET_TMPDIR")).unwrap();

let conf = PageServerConf::dummy_conf(repo_dir.path().to_path_buf());
let mut conf = PageServerConf::dummy_conf(repo_dir.path().to_path_buf());
conf.walredo_process_kind = process_kind;
let conf = Box::leak(Box::new(conf));
let tenant_shard_id = TenantShardId::unsharded(TenantId::generate());

Expand All @@ -113,25 +145,40 @@ fn bench_impl(redo_work: Arc<Request>, n_redos: u64, nclients: u64) -> Duration
let manager = PostgresRedoManager::new(conf, tenant_shard_id);
let manager = Arc::new(manager);

// divide the amount of work equally among the clients.
let nredos_per_client = n_redos / nclients;
for _ in 0..nclients {
rt.block_on(async {
tasks.spawn(client(
Arc::clone(&manager),
Arc::clone(&start),
Arc::clone(&redo_work),
// divide the amount of work equally among the clients
n_redos / nclients,
nredos_per_client,
))
});
}

rt.block_on(async move {
let mut total_wallclock_time = std::time::Duration::from_millis(0);
let elapsed = rt.block_on(async move {
let mut total_wallclock_time = Duration::ZERO;
while let Some(res) = tasks.join_next().await {
total_wallclock_time += res.unwrap();
}
total_wallclock_time
})
});

// consistency check to ensure process kind setting worked
if nredos_per_client > 0 {
assert_eq!(
manager
.status()
.process
.map(|p| p.kind)
.expect("the benchmark work causes a walredo process to be spawned"),
std::borrow::Cow::Borrowed(process_kind.into())
);
}

elapsed
}

async fn client(
Expand Down
1 change: 1 addition & 0 deletions pageserver/src/bin/pageserver.rs
Original file line number Diff line number Diff line change
Expand Up @@ -285,6 +285,7 @@ fn start_pageserver(
))
.unwrap();
pageserver::preinitialize_metrics();
pageserver::metrics::wal_redo::set_process_kind_metric(conf.walredo_process_kind);

// If any failpoints were set from FAILPOINTS environment variable,
// print them to the log for debugging purposes
Expand Down
Loading

1 comment on commit 2d5a846

@github-actions
Copy link

Choose a reason for hiding this comment

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

2840 tests run: 2709 passed, 0 failed, 131 skipped (full report)


Flaky tests (2)

Postgres 16

Postgres 15

  • test_delete_timeline_client_hangup: debug

Code coverage* (full report)

  • functions: 28.0% (6436 of 23015 functions)
  • lines: 46.5% (45059 of 96823 lines)

* collected from Rust tests only


The comment gets automatically updated with the latest test results
2d5a846 at 2024-04-15T21:08:57.959Z :recycle:

Please sign in to comment.