Skip to content

Commit

Permalink
bypass PageCache for InMemoryLayer + avoid Value::deser on L0 f…
Browse files Browse the repository at this point in the history
…lush (#8537)

Part of [Epic: Bypass PageCache for user data
blocks](#7386).

# Problem

`InMemoryLayer` still uses the `PageCache` for all data stored in the
`VirtualFile` that underlies the `EphemeralFile`.

# Background

Before this PR, `EphemeralFile` is a fancy and (code-bloated) buffered
writer around a `VirtualFile` that supports `blob_io`.

The `InMemoryLayerInner::index` stores offsets into the `EphemeralFile`.
At those offset, we find a varint length followed by the serialized
`Value`.

Vectored reads (`get_values_reconstruct_data`) are not in fact vectored
- each `Value` that needs to be read is read sequentially.

The `will_init` bit of information which we use to early-exit the
`get_values_reconstruct_data` for a given key is stored in the
serialized `Value`, meaning we have to read & deserialize the `Value`
from the `EphemeralFile`.

The L0 flushing **also** needs to re-determine the `will_init` bit of
information, by deserializing each value during L0 flush.

# Changes

1. Store the value length and `will_init` information in the
`InMemoryLayer::index`. The `EphemeralFile` thus only needs to store the
values.
2. For `get_values_reconstruct_data`:
- Use the in-memory `index` figures out which values need to be read.
Having the `will_init` stored in the index enables us to do that.
- View the EphemeralFile as a byte array of "DIO chunks", each 512 bytes
in size (adjustable constant). A "DIO chunk" is the minimal unit that we
can read under direct IO.
- Figure out which chunks need to be read to retrieve the serialized
bytes for thes values we need to read.
- Coalesce chunk reads such that each DIO chunk is only read once to
serve all value reads that need data from that chunk.
- Merge adjacent chunk reads into larger
`EphemeralFile::read_exact_at_eof_ok` of up to 128k (adjustable
constant).
3. The new `EphemeralFile::read_exact_at_eof_ok` fills the IO buffer
from the underlying VirtualFile and/or its in-memory buffer.
4. The L0 flush code is changed to use the `index` directly, `blob_io` 
5. We can remove the `ephemeral_file::page_caching` construct now.

The `get_values_reconstruct_data` changes seem like a bit overkill but
they are necessary so we issue the equivalent amount of read system
calls compared to before this PR where it was highly likely that even if
the first PageCache access was a miss, remaining reads within the same
`get_values_reconstruct_data` call from the same `EphemeralFile` page
were a hit.

The "DIO chunk" stuff is truly unnecessary for page cache bypass, but,
since we're working on [direct
IO](#8130) and
#8719 specifically, we need
to do _something_ like this anyways in the near future.

# Alternative Design

The original plan was to use the `vectored_blob_io` code it relies on
the invariant of Delta&Image layers that `index order == values order`.

Further, `vectored_blob_io` code's strategy for merging IOs is limited
to adjacent reads. However, with direct IO, there is another level of
merging that should be done, specifically, if multiple reads map to the
same "DIO chunk" (=alignment-requirement-sized and -aligned region of
the file), then it's "free" to read the chunk into an IO buffer and
serve the two reads from that buffer.
=> #8719

# Testing / Performance

Correctness of the IO merging code is ensured by unit tests.

Additionally, minimal tests are added for the `EphemeralFile`
implementation and the bit-packed `InMemoryLayerIndexValue`.

Performance testing results are presented below.
All pref testing done on my M2 MacBook Pro, running a Linux VM.
It's a release build without `--features testing`.

We see definitive improvement in ingest performance microbenchmark and
an ad-hoc microbenchmark for getpage against InMemoryLayer.

```
baseline: commit 7c74112 origin/main
HEAD: ef1c55c
```

<details>

```
cargo bench --bench bench_ingest -- 'ingest 128MB/100b seq, no delta'

baseline

ingest-small-values/ingest 128MB/100b seq, no delta
                        time:   [483.50 ms 498.73 ms 522.53 ms]
                        thrpt:  [244.96 MiB/s 256.65 MiB/s 264.73 MiB/s]

HEAD

ingest-small-values/ingest 128MB/100b seq, no delta
                        time:   [479.22 ms 482.92 ms 487.35 ms]
                        thrpt:  [262.64 MiB/s 265.06 MiB/s 267.10 MiB/s]
```

</details>

We don't have a micro-benchmark for InMemoryLayer and it's quite
cumbersome to add one. So, I did manual testing in `neon_local`.

<details>

```

  ./target/release/neon_local stop
  rm -rf .neon
  ./target/release/neon_local init
  ./target/release/neon_local start
  ./target/release/neon_local tenant create --set-default
  ./target/release/neon_local endpoint create foo
  ./target/release/neon_local endpoint start foo
  psql 'postgresql://cloud_admin@127.0.0.1:55432/postgres'
psql (13.16 (Debian 13.16-0+deb11u1), server 15.7)

CREATE TABLE wal_test (
    id SERIAL PRIMARY KEY,
    data TEXT
);

DO $$
DECLARE
    i INTEGER := 1;
BEGIN
    WHILE i <= 500000 LOOP
        INSERT INTO wal_test (data) VALUES ('data');
        i := i + 1;
    END LOOP;
END $$;

-- => result is one L0 from initdb and one 137M-sized ephemeral-2

DO $$
DECLARE
    i INTEGER := 1;
    random_id INTEGER;
    random_record wal_test%ROWTYPE;
    start_time TIMESTAMP := clock_timestamp();
    selects_completed INTEGER := 0;
    min_id INTEGER := 1;  -- Minimum ID value
    max_id INTEGER := 100000;  -- Maximum ID value, based on your insert range
    iters INTEGER := 100000000;  -- Number of iterations to run
BEGIN
    WHILE i <= iters LOOP
        -- Generate a random ID within the known range
        random_id := min_id + floor(random() * (max_id - min_id + 1))::int;

        -- Select the row with the generated random ID
        SELECT * INTO random_record
        FROM wal_test
        WHERE id = random_id;

        -- Increment the select counter
        selects_completed := selects_completed + 1;

        -- Check if a second has passed
        IF EXTRACT(EPOCH FROM clock_timestamp() - start_time) >= 1 THEN
            -- Print the number of selects completed in the last second
            RAISE NOTICE 'Selects completed in last second: %', selects_completed;

            -- Reset counters for the next second
            selects_completed := 0;
            start_time := clock_timestamp();
        END IF;

        -- Increment the loop counter
        i := i + 1;
    END LOOP;
END $$;

./target/release/neon_local stop

baseline: commit 7c74112 origin/main

NOTICE:  Selects completed in last second: 1864
NOTICE:  Selects completed in last second: 1850
NOTICE:  Selects completed in last second: 1851
NOTICE:  Selects completed in last second: 1918
NOTICE:  Selects completed in last second: 1911
NOTICE:  Selects completed in last second: 1879
NOTICE:  Selects completed in last second: 1858
NOTICE:  Selects completed in last second: 1827
NOTICE:  Selects completed in last second: 1933

ours

NOTICE:  Selects completed in last second: 1915
NOTICE:  Selects completed in last second: 1928
NOTICE:  Selects completed in last second: 1913
NOTICE:  Selects completed in last second: 1932
NOTICE:  Selects completed in last second: 1846
NOTICE:  Selects completed in last second: 1955
NOTICE:  Selects completed in last second: 1991
NOTICE:  Selects completed in last second: 1973
```

NB: the ephemeral file sizes differ by ca 1MiB, ours being 1MiB smaller.

</details>

# Rollout

This PR changes the code in-place and  is not gated by a feature flag.
  • Loading branch information
problame authored Aug 28, 2024
1 parent 63a0d0d commit 9627747
Show file tree
Hide file tree
Showing 20 changed files with 1,757 additions and 654 deletions.
14 changes: 14 additions & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 2 additions & 0 deletions Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ axum = { version = "0.6.20", features = ["ws"] }
base64 = "0.13.0"
bincode = "1.3"
bindgen = "0.65"
bit_field = "0.10.2"
bstr = "1.0"
byteorder = "1.4"
bytes = "1.0"
Expand Down Expand Up @@ -145,6 +146,7 @@ rustls-split = "0.3"
scopeguard = "1.1"
sysinfo = "0.29.2"
sd-notify = "0.4.1"
send-future = "0.1.0"
sentry = { version = "0.32", default-features = false, features = ["backtrace", "contexts", "panic", "rustls", "reqwest" ] }
serde = { version = "1.0", features = ["derive"] }
serde_json = "1"
Expand Down
2 changes: 2 additions & 0 deletions pageserver/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ arc-swap.workspace = true
async-compression.workspace = true
async-stream.workspace = true
async-trait.workspace = true
bit_field.workspace = true
byteorder.workspace = true
bytes.workspace = true
camino.workspace = true
Expand Down Expand Up @@ -52,6 +53,7 @@ rand.workspace = true
range-set-blaze = { version = "0.1.16", features = ["alloc"] }
regex.workspace = true
scopeguard.workspace = true
send-future.workspace = true
serde.workspace = true
serde_json = { workspace = true, features = ["raw_value"] }
serde_path_to_error.workspace = true
Expand Down
4 changes: 2 additions & 2 deletions pageserver/benches/bench_ingest.rs
Original file line number Diff line number Diff line change
Expand Up @@ -103,13 +103,13 @@ async fn ingest(
batch.push((key.to_compact(), lsn, data_ser_size, data.clone()));
if batch.len() >= BATCH_SIZE {
let this_batch = std::mem::take(&mut batch);
let serialized = SerializedBatch::from_values(this_batch);
let serialized = SerializedBatch::from_values(this_batch).unwrap();
layer.put_batch(serialized, &ctx).await?;
}
}
if !batch.is_empty() {
let this_batch = std::mem::take(&mut batch);
let serialized = SerializedBatch::from_values(this_batch);
let serialized = SerializedBatch::from_values(this_batch).unwrap();
layer.put_batch(serialized, &ctx).await?;
}
layer.freeze(lsn + 1).await;
Expand Down
39 changes: 39 additions & 0 deletions pageserver/src/assert_u64_eq_usize.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
//! `u64`` and `usize`` aren't guaranteed to be identical in Rust, but life is much simpler if that's the case.

pub(crate) const _ASSERT_U64_EQ_USIZE: () = {
if std::mem::size_of::<usize>() != std::mem::size_of::<u64>() {
panic!("the traits defined in this module assume that usize and u64 can be converted to each other without loss of information");
}
};

pub(crate) trait U64IsUsize {
fn into_usize(self) -> usize;
}

impl U64IsUsize for u64 {
#[inline(always)]
fn into_usize(self) -> usize {
#[allow(clippy::let_unit_value)]
let _ = _ASSERT_U64_EQ_USIZE;
self as usize
}
}

pub(crate) trait UsizeIsU64 {
fn into_u64(self) -> u64;
}

impl UsizeIsU64 for usize {
#[inline(always)]
fn into_u64(self) -> u64 {
#[allow(clippy::let_unit_value)]
let _ = _ASSERT_U64_EQ_USIZE;
self as u64
}
}

pub const fn u64_to_usize(x: u64) -> usize {
#[allow(clippy::let_unit_value)]
let _ = _ASSERT_U64_EQ_USIZE;
x as usize
}
10 changes: 10 additions & 0 deletions pageserver/src/config.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ use utils::{

use crate::l0_flush::L0FlushConfig;
use crate::tenant::config::TenantConfOpt;
use crate::tenant::storage_layer::inmemory_layer::IndexEntry;
use crate::tenant::timeline::compaction::CompactL0Phase1ValueAccess;
use crate::tenant::vectored_blob_io::MaxVectoredReadBytes;
use crate::tenant::{TENANTS_SEGMENT_NAME, TIMELINES_SEGMENT_NAME};
Expand Down Expand Up @@ -1020,6 +1021,15 @@ impl PageServerConf {

conf.default_tenant_conf = t_conf.merge(TenantConf::default());

IndexEntry::validate_checkpoint_distance(conf.default_tenant_conf.checkpoint_distance)
.map_err(|msg| anyhow::anyhow!("{msg}"))
.with_context(|| {
format!(
"effective checkpoint distance is unsupported: {}",
conf.default_tenant_conf.checkpoint_distance
)
})?;

Ok(conf)
}

Expand Down
1 change: 1 addition & 0 deletions pageserver/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ pub mod l0_flush;
use futures::{stream::FuturesUnordered, StreamExt};
pub use pageserver_api::keyspace;
use tokio_util::sync::CancellationToken;
mod assert_u64_eq_usize;
pub mod aux_file;
pub mod metrics;
pub mod page_cache;
Expand Down
6 changes: 6 additions & 0 deletions pageserver/src/tenant.rs
Original file line number Diff line number Diff line change
Expand Up @@ -877,6 +877,12 @@ impl Tenant {
});
};

// TODO: should also be rejecting tenant conf changes that violate this check.
if let Err(e) = crate::tenant::storage_layer::inmemory_layer::IndexEntry::validate_checkpoint_distance(tenant_clone.get_checkpoint_distance()) {
make_broken(&tenant_clone, anyhow::anyhow!(e), BrokenVerbosity::Error);
return Ok(());
}

let mut init_order = init_order;
// take the completion because initial tenant loading will complete when all of
// these tasks complete.
Expand Down
4 changes: 2 additions & 2 deletions pageserver/src/tenant/blob_io.rs
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,7 @@ pub(super) const LEN_COMPRESSION_BIT_MASK: u8 = 0xf0;

/// The maximum size of blobs we support. The highest few bits
/// are reserved for compression and other further uses.
const MAX_SUPPORTED_LEN: usize = 0x0fff_ffff;
pub(crate) const MAX_SUPPORTED_BLOB_LEN: usize = 0x0fff_ffff;

pub(super) const BYTE_UNCOMPRESSED: u8 = 0x80;
pub(super) const BYTE_ZSTD: u8 = BYTE_UNCOMPRESSED | 0x10;
Expand Down Expand Up @@ -326,7 +326,7 @@ impl<const BUFFERED: bool> BlobWriter<BUFFERED> {
(self.write_all(io_buf.slice_len(), ctx).await, srcbuf)
} else {
// Write a 4-byte length header
if len > MAX_SUPPORTED_LEN {
if len > MAX_SUPPORTED_BLOB_LEN {
return (
(
io_buf.slice_len(),
Expand Down
23 changes: 0 additions & 23 deletions pageserver/src/tenant/block_io.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,6 @@
//! Low-level Block-oriented I/O functions
//!

use super::ephemeral_file::EphemeralFile;
use super::storage_layer::delta_layer::{Adapter, DeltaLayerInner};
use crate::context::RequestContext;
use crate::page_cache::{self, FileId, PageReadGuard, PageWriteGuard, ReadBufResult, PAGE_SZ};
Expand Down Expand Up @@ -81,9 +80,7 @@ impl<'a> Deref for BlockLease<'a> {
/// Unlike traits, we also support the read function to be async though.
pub(crate) enum BlockReaderRef<'a> {
FileBlockReader(&'a FileBlockReader<'a>),
EphemeralFile(&'a EphemeralFile),
Adapter(Adapter<&'a DeltaLayerInner>),
Slice(&'a [u8]),
#[cfg(test)]
TestDisk(&'a super::disk_btree::tests::TestDisk),
#[cfg(test)]
Expand All @@ -100,9 +97,7 @@ impl<'a> BlockReaderRef<'a> {
use BlockReaderRef::*;
match self {
FileBlockReader(r) => r.read_blk(blknum, ctx).await,
EphemeralFile(r) => r.read_blk(blknum, ctx).await,
Adapter(r) => r.read_blk(blknum, ctx).await,
Slice(s) => Self::read_blk_slice(s, blknum),
#[cfg(test)]
TestDisk(r) => r.read_blk(blknum),
#[cfg(test)]
Expand All @@ -111,24 +106,6 @@ impl<'a> BlockReaderRef<'a> {
}
}

impl<'a> BlockReaderRef<'a> {
fn read_blk_slice(slice: &[u8], blknum: u32) -> std::io::Result<BlockLease> {
let start = (blknum as usize).checked_mul(PAGE_SZ).unwrap();
let end = start.checked_add(PAGE_SZ).unwrap();
if end > slice.len() {
return Err(std::io::Error::new(
std::io::ErrorKind::UnexpectedEof,
format!("slice too short, len={} end={}", slice.len(), end),
));
}
let slice = &slice[start..end];
let page_sized: &[u8; PAGE_SZ] = slice
.try_into()
.expect("we add PAGE_SZ to start, so the slice must have PAGE_SZ");
Ok(BlockLease::Slice(page_sized))
}
}

///
/// A "cursor" for efficiently reading multiple pages from a BlockReader
///
Expand Down
Loading

1 comment on commit 9627747

@github-actions
Copy link

Choose a reason for hiding this comment

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

3780 tests run: 3673 passed, 1 failed, 106 skipped (full report)


Failures on Postgres 16

# Run all failed tests locally:
scripts/pytest -vv -n $(nproc) -k "test_gc_of_remote_layers[release-pg16]"
Flaky tests (5)

Postgres 16

Postgres 15

Test coverage report is not available

The comment gets automatically updated with the latest test results
9627747 at 2024-08-28T19:25:39.854Z :recycle:

Please sign in to comment.