diff --git a/Cargo.lock b/Cargo.lock index f17f0569f9295..4d12f6f9cb15a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7077,6 +7077,7 @@ dependencies = [ "serde_json", "smallvec", "stacker", + "tempfile", "thiserror 2.0.18", "tokio", "tokio-native-tls", @@ -8348,9 +8349,11 @@ dependencies = [ "either", "futures-util", "lgalloc", + "lz4_flex", "mz-ore", "num-traits", "serde", + "tempfile", "timely", "tokio", "tracing", diff --git a/Cargo.toml b/Cargo.toml index 1f71ea92e6c7b..9332161cda7dd 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -385,6 +385,7 @@ launchdarkly-server-sdk = { version = "2.6.2", default-features = false } lgalloc = "0.6.0" libc = "0.2.184" lru = "0.16.3" +lz4_flex = { version = "0.12.1", default-features = false, features = ["frame", "std"] } maplit = "1.0.2" mappings = "0.7.2" md-5 = "0.10.6" diff --git a/doc/developer/design/20260504_pager.md b/doc/developer/design/20260504_pager.md new file mode 100644 index 0000000000000..ba327504dbaa9 --- /dev/null +++ b/doc/developer/design/20260504_pager.md @@ -0,0 +1,411 @@ +# Pager + +* Associated: [CLU-65](https://linear.app/materializeinc/issue/CLU-65/pager), depends on [CLU-64](https://linear.app/materializeinc/issue/CLU-64/remove-lgalloc-from-columnar) (`Column::Aligned` becomes `Vec`). + +## The problem + +Materialize spills working sets to Linux swap. +The kernel decides which anonymous pages to evict, conflating application state with arbitrary heap allocations and forcing user threads into direct reclaim when memory pressure rises. +Direct reclaim shows up as user-visible latency in dataflow operators and as elevated `pgscan_direct` in `/proc/vmstat` during hydration. +We need an explicit pager so the application can mark cold data, ask for it back on demand, and choose whether the cold storage lives in anonymous memory (with kernel hints) or on a dedicated scratch volume. +This unblocks the columnar end-to-end project: once `Column::Aligned` is a `Vec` (CLU-64), the pager is the natural seam between in-memory columnar buffers and out-of-core storage. + +## Success criteria + +The design succeeds when: + +* Cold columnar data can be paged out and back in via a single API regardless of backend. +* The swap backend introduces no copies on the page-out path; the page-in path costs at most a `madvise` plus the unavoidable read. +* The file backend uses one syscall per logical operation where the OS supports it (`writev`, `pread`/coalesced `pread`). +* Switching backends is a runtime configuration flip, not a recompile or a restart. +* Existing handles remain usable across backend flips. +* The API supports both write-once/read-once 2 MiB-scale spill blocks and write-once/read-many large blobs with random offset and length access. +* Caller-side allocation is reusable: the buffers handed to `pageout` and the buffers passed to `pagein`-style reads return to the caller in a state that preserves their capacity where the backend permits. +* Out-of-core scenarios (working set 2x system RAM) actually offload pages: peak resident set size drops compared to leaving the data resident. + +## Out of scope + +* Eviction policy. + The pager is a mechanism; callers decide what and when to page out. +* Generic element types. + The API is `Vec` only. + We will revisit if a non-`u64` consumer appears. +* Compression and encryption. + Both can wrap the pager externally. +* Async or `io_uring`. + Sync syscalls only for v1. + An async wrapper is a follow-up if profiles motivate it. +* Cross-handle file pooling (one shared scratch file with a free-list). + Each handle owns one named file in the scratch directory; pooling is a follow-up if inode pressure shows up. +* Non-Linux production support. + Both backends compile on macOS and others as in-memory no-ops, but the production target is Linux. + +## Solution proposal + +A single `mz_ore::pager` module exposes a backend-agnostic API around a `Handle` type. +A global atomic selects the backend at `pageout` time; the chosen backend is baked into each handle so live flips do not invalidate existing data. +Two backends ship: `Swap` (keep allocations resident, hint the kernel via `MADV_COLD`) and `File` (write to a named file in a per-process scratch subdirectory). +The API is sync; the file backend uses `writev` and `pread` to keep syscall counts low. +The file backend never holds a file descriptor in the handle: each operation opens, runs syscalls, and closes the fd, so per-handle state is a tiny `(scratch_id, length)` tuple regardless of how many handles a process has alive. + +### Architecture + +```mermaid +flowchart LR + Caller -- "pageout(&mut [Vec])" --> Pager + Pager -- "Backend::current()" --> Sel{Atomic backend} + Sel -- Swap --> SwapBackend["Swap: hold Vec(s), MADV_COLD"] + Sel -- File --> FileBackend["File: writev to scratch/{pid}/{id}.bin, close fd"] + Caller -- "read_at_many(&Handle, ranges, &mut dst)" --> Pager + Caller -- "take(Handle, &mut dst)" --> Pager +``` + +The handle's variant captures which backend produced it, so reads dispatch on the handle alone. +A backend flip changes only future `pageout` calls; a handle taken under `Swap` continues to read from memory after a flip to `File`. + +### API + +```rust +//! src/ore/src/pager.rs + +pub enum Backend { Swap, File } + +/// Sets the active backend for future pageouts. Existing handles are unaffected. +pub fn set_backend(b: Backend); + +pub fn backend() -> Backend; + +/// Configures the scratch directory for the file backend. +/// Must be called before the first file-backend pageout. +pub fn set_scratch_dir(path: PathBuf); + +pub struct Handle { /* private: SwapInner | FileInner */ } + +impl Handle { + /// Logical length in u64s. + pub fn len(&self) -> usize; + pub fn len_bytes(&self) -> usize { self.len() * 8 } + pub fn is_empty(&self) -> bool { self.len() == 0 } +} + +impl Drop for Handle { /* swap: drops Vec(s). file: unlinks the scratch file. */ } + +/// Scatter pageout. Logical layout = chunks concatenated in input order. +/// After return: each Vec in `chunks` is empty. +/// File backend preserves capacity; swap backend moves the alloc into the handle. +/// Empty input returns a `len == 0` handle and performs no I/O. +pub fn pageout(chunks: &mut [Vec]) -> Handle; + +/// Reads multiple ranges. Output appended to `dst` in request order (concat). +/// Panics if any range is out of bounds. +pub fn read_at_many(handle: &Handle, ranges: &[(usize, usize)], dst: &mut Vec); + +/// Single-range convenience. +pub fn read_at(handle: &Handle, offset: usize, len: usize, dst: &mut Vec); + +/// Consumes handle, writing the entire payload into `dst` (cleared first), then reclaims storage. +/// Swap fast path: single-chunk handle into empty `dst` swaps in place, no copy. +pub fn take(handle: Handle, dst: &mut Vec); +``` + +### Swap backend + +Storage is `Vec>` plus a prefix-sum `Vec` of cumulative lengths in u64s. + +`pageout`. +Move each input Vec via `mem::take`. +For each chunk, compute the page-aligned subrange of its byte buffer and call `madvise(ptr, len, MADV_COLD)`. +`MADV_COLD` deactivates the pages without freeing them; the kernel reclaims under pressure without a synchronous swap-out. +Skip the syscall when the page-aligned region is empty (sub-page chunks). + +`read_at_many`. +For each range, binary-search the prefix-sum to find the starting chunk, then `extend_from_slice` across chunk boundaries. +Optionally call `MADV_WILLNEED` on the touched pages before the copy. +Output appends in request order to `dst`. + +`take`. +A single-chunk handle paired with an empty `dst` triggers `mem::swap` for a zero-copy take. +Otherwise concat all chunks into `dst`. +The handle drops; the `Vec>` reclaims. + +### File backend + +Storage per handle is `(scratch_id: u64, len_u64s: usize)`. +File descriptors are never retained: the handle is 16 bytes regardless of file state. +The pager owns a per-process subdirectory `{scratch_dir}/mz-pager-{pid}-{boot_nonce}/` and writes each handle to `{subdir}/{scratch_id}.bin` where `scratch_id` is allocated from a process-wide `AtomicU64`. + +`pageout`. +Allocate `scratch_id`. +`File::create_new(path)` to open exclusively, build an iovec from each chunk's byte slice, issue one `writev` covering all chunks, then close the fd. +Clear each input Vec after the write so the caller keeps capacity. +On I/O failure, log a warning, unlink the path if it was created, and fall back by constructing a `SwapInner` from the same chunks; the contract on input Vecs and the returned handle is unchanged. + +`read_at_many`. +`File::open(path)`, then for each range compute byte offset and byte length and call `pread`. +Coalesce adjacent ranges (`offset_i + len_i == offset_{i+1}`) into a single `pread`. +Append into `dst`, then close the fd. +A future optimization is `preadv2` once a profile shows it matters. + +`take`. +`File::open(path)`, one `pread` for the whole length into `dst`, close the fd, `unlink(path)`, drop the handle. + +`Drop` (without `take`). +`unlink(path)`. The kernel reclaims the inode. + +### Scratch directory lifecycle + +`set_scratch_dir(root)` creates `{root}/mz-pager-{pid}-{boot_nonce}/`. +`boot_nonce` is a random 64-bit value sampled at config time so two processes that briefly share the same pid cannot collide. +On `set_scratch_dir`, the pager runs a reaper that walks `{root}` and removes any sibling `mz-pager-*` subdirectory whose owning pid is no longer alive (`/proc/{pid}` missing on Linux); this reclaims storage from crashed predecessors. +On clean process exit, a `Drop` on the global pager state removes the per-process subdirectory. +The reaper is best-effort and logs failures rather than panicking. + +### Configuration + +Two pieces of global state, both behind atomics or `OnceLock`: + +* `BACKEND: AtomicU8`, set by `set_backend`. +* `SCRATCH_DIR: OnceLock`, set by `set_scratch_dir` before the first file-backend pageout. Subsequent calls log a warning and become no-ops to avoid mid-run path changes. + +A LaunchDarkly-style param wires `set_backend` from cluster configuration, mirroring `mz_ore::region::ENABLE_LGALLOC_REGION`. + +### Concurrency + +`Handle: Send`. +`pageout` and `take` consume by value, so they are single-threaded with respect to a handle by construction. +`read_at` and `read_at_many` take `&Handle` and are concurrent-safe: the file path uses `pread` (thread-safe positional read); the swap path reads through an immutable `&Vec`. +`set_backend` racing with `pageout` is benign: `pageout` reads the atomic once at entry, and existing handles keep their backend. + +### Errors + +File I/O on `pageout` failure: log at `warn`, fall back to a swap-backed handle. +File I/O on read failure: panic. +The data lives only on the spill path; partial reads indicate corruption or device loss, both unrecoverable. +Out-of-bounds range in `read_at*`: panic. +Empty input to `pageout`: returns a `len == 0` handle with no syscalls on either backend. + +### File layout + +``` +src/ore/src/pager.rs # Public API, dispatch, Handle enum, config +src/ore/src/pager/swap.rs # Swap backend +src/ore/src/pager/file.rs # File backend (per-process subdir, writev, pread) +src/ore/benches/pager.rs # Criterion benches +src/ore/Cargo.toml # Feature `pager`: deps libc, bytemuck +``` + +The `pager` feature gates the module so non-Linux builds compile. +On non-Linux, both backends degrade to no-op variants that hold data in memory and skip syscalls. + +## Minimal viable prototype + +A working in-tree prototype is the first implementation step: both backends, end-to-end tests, and the benchmark harness described below. +The prototype validates three risks early. +First, that `MADV_COLD` actually offloads pages under pressure; we measure this by allocating 2x system RAM in handles and watching peak RSS via `/proc/self/status`. +Second, that the file backend's vectored I/O is competitive with swap on sequential workloads; the bench compares 1 x 2 MiB and 64 x 32 KiB layouts on both backends. +Third, that the API survives integration with `Column` post-CLU-64; a follow-up branch wires `Column::Aligned` to `pageout`/`take` end-to-end and runs an existing columnar bench. + +The bench harness lives at `src/ore/benches/pager.rs` and uses Criterion. +Knobs: backend (Swap, File), payload size (4 KiB, 64 KiB, 1 MiB, 16 MiB), chunk count (1, 2, 64), scratch dir (`MZ_PAGER_SCRATCH` env var, default `$TMPDIR`). +Cases: + +* `pageout` wall time, single chunk, varying size. +* `pageout` scatter, fixed total size, varying chunk count, both backends; isolates `writev` benefit. +* `read_at` whole-block after a configurable idle delay so the kernel actually reclaims swap pages. +* `read_at_many` random ranges, 1, 8, 64 ranges per call, sorted vs unsorted to exercise coalescing. +* Sustained round-trip `pageout` -> `read_at` -> drop, measuring ops/s. +* Working-set scenario: 2x system RAM in handles, page out half, read random handles; gated behind `cargo bench --features pager-stress` since CI cannot run it. + +Unit tests in `src/ore/src/pager.rs` and per-backend modules cover round-trip on both backends, scatter/gather correctness (random ranges including overlapping and adjacent), drop-without-read reclaim, the swap fast path for `take` (assert pointer identity), backend flip mid-run (handle taken under one backend reads through after flip), and capacity-preservation rules. +miri runs the swap backend; the file backend skips on miri due to syscalls. + +## Operational characteristics + +A merge-batcher-style example (`src/ore/examples/pager_merge.rs`) builds two chains of 2 MiB chunks, then merges them while reading every cache line of the input. +Run under `systemd-run --user --scope -p MemoryMax=...` to constrain memory and force real eviction. + +Pager behavior depends sharply on disk topology. +Slow-storage boxes hit a kernel reclaim ceiling well below the disk and the swap backend looks catastrophic; fast-storage boxes expose new bottlenecks and the swap-vs-file gap collapses at high thread counts. +Two benches below: a single encrypted NVMe (~1.4 GB/s ceiling, akin to typical EBS-backed clusters) and an r8gd.16xlarge with two striped local instance NVMes (~7 GB/s ceiling). +`through` is total bytes pumped through the merge divided by wall time, summed across threads. + +### Bench A: encrypted NVMe, 32 GiB working set + +Single encrypted NVMe, ~1.4 GB/s sustained R+W ceiling. `--chain-gib 16` (32 GiB working set). + +| RAM cap | threads | swap GiB/s | file GiB/s | file/swap | +|--------:|--------:|-----------:|-----------:|----------:| +| 16 G | 1 | 0.15 | 0.50 | 3.4× | +| 16 G | 16 | 0.36 | 1.47 | 4.0× | +| 8 G | 1 | 0.12 | 0.44 | 3.5× | +| 8 G | 16 | 0.37 | 0.79 | 2.1× | +| 4 G | 1 | 0.12 | 0.36 | 3.0× | +| 4 G | 16 | 0.36 | 1.21 | 3.4× | + +* File saturates the disk: 1.47 GiB/s at 16 G cap, 16 threads ≈ NVMe ceiling. +* Swap caps at ~0.36 GiB/s, leaving 70 %+ of disk capacity unused. +* File scales with parallelism (3× from 1 → 16 threads); swap floors near 90 s wall. + +### Bench B: r8gd.16xlarge, striped instance NVMe + +64 vCPU, 512 GiB RAM, 2× 1.7 TiB local instance NVMe partitioned 50/50: half striped via mdadm RAID0 for ext4 scratch (~7 GB/s combined sequential), other halves attached as swap with equal priority (kernel-side striping). +Pressure ratio fixed at 4× (workload : RAM cap). + +#### File backend, 1 TiB working set (`--chain-gib 512`, cap = 256 G) + +| threads | wall | overall GiB/s | merge phase | +|--------:|-------:|--------------:|------------:| +| 1 | 1195 s | 0.86 | 888 s | +| 4 | 598 s | 1.71 | 301 s | +| 16 | 593 s | 1.73 | 310 s | +| 32 | 591 s | 1.73 | 310 s | +| 64 | 617 s | 1.66 | 336 s | + +File backend saturates at 4 threads; beyond that, scheduling overhead slightly hurts. +Combined disk during merge ≈ 7 GB/s (~3.5 read + ~3.5 write), at the two-NVMe stripe ceiling. +Build-phase write rate hits the same ceiling even at 1 thread, so build is disk-bound regardless of parallelism — only merge benefits from concurrency. + +#### Swap backend, 128 GiB working set (`--chain-gib 64`, cap = 32 G) + +| threads | wall | overall GiB/s | merge GiB/s | +|--------:|------:|--------------:|------------:| +| 1 | 773 s | 0.17 | 0.19 | +| 4 | 229 s | 0.56 | 0.69 | +| 16 | 106 s | 1.21 | 1.87 | +| 32 | 88 s | 1.45 | 2.45 | +| 64 | 85 s | 1.51 | 2.51 | + +Swap-backend merge scales 13× from 1 → 64 threads, plateauing at ~2.5 GiB/s through (~5 GB/s combined disk). +That is ~75 % of the file backend's saturation throughput on the same hardware. +Below ~16 threads the kernel serializes on per-memcg reclaim and we see Bench-A-like floors; above 32 threads, enough independent direct-reclaim contexts run in parallel to keep the swap stripe nearly busy. + +### Headlines (revised) + +* The "swap caps at 0.36 GiB/s regardless of parallelism" claim from Bench A is true only at low thread counts. + With 64 threads each entering direct reclaim independently, swap-backend merge reaches ~2.5 GiB/s on fast storage. +* File backend still wins three ways: ~25 % higher peak throughput on fast disk, far lower RSS (376 MB at 64 threads vs the cgroup cap pinned by swap), and consistent performance below 4 threads where swap collapses. +* Hardware floor matters more than expected. + On 1.4 GB/s encrypted NVMe, kernel reclaim is 4× below disk and swap is dominated. + On 7 GB/s striped local NVMe, reclaim is no longer the bottleneck at high parallelism. + +### Why swap stalls (single-thread regime) + +Single-threaded merge with a 4 GiB chain (8 GiB working set) under a 2 G cap, instrumented via `perf stat` plus `/proc/vmstat` deltas: + +| metric | swap | file | ratio | +|---|---:|---:|---:| +| wall time | 65.5 s | 24.0 s | 2.7× | +| sys time | 63.8 s | 9.0 s | 7.1× | +| user time | 1.5 s | 1.5 s | 1× | +| major-faults | 65,516 | 1,913 | 34× | +| minor-faults | 5,187,868 | 3,986 | 1300× | +| dTLB-load-misses | 42 M | 12 M | 3.4× | +| pswpin (4 KiB pages) | 2.12 M | 2.2 K | 970× | +| pswpout | 3.64 M | 3.1 K | 1180× | + +Of swap's 65 s wall, 64 s is sys time — the kernel runs the user thread's fault handler. +2.1 M page-ins through the swap path means every 4 KiB granule of the 8 GiB working set page-faults synchronously on the user thread. +The disk does not become the bottleneck: 8 GiB / 65 s = 130 MB/s, less than 10 % of NVMe capacity. +Page-table churn from `MADV_COLD` reclaim and subsequent re-faulting also drives 3.4× more dTLB misses on the swap path; each unmap broadcasts a TLB-shootdown IPI to every CPU running the task. + +The file backend issues one `writev` per chunk on pageout and one `pread` per coalesced range on read, lets kernel readahead overlap I/O with the user thread's compute, and never pays the per-page fault tax. +9 s of sys time covers all of its kernel work for the same 8 GiB scan. + +The single-thread analysis generalizes: per-thread fault cost bounds swap throughput, and the only escape is enough threads in flight that the kernel can run multiple reclaim contexts concurrently. +Bench B confirms this — at 64 threads on fast disks, swap reaches ~75 % of file throughput rather than the ~10 % implied by single-thread numbers. + +### Operational guidance + +* Pick the swap backend when the working set is comfortably resident. + `MADV_COLD` is essentially free in that regime and operations run at memory bandwidth. +* Pick the file backend whenever the working set may exceed RAM **and** the workload runs with few enough threads that swap-merge would serialize on reclaim. + Below ~16 threads, the file backend wins by ~3–5×. +* For highly parallel workloads (vCPU-count threads) on fast local storage, swap and file are within ~25 % of each other on throughput; the choice can shift toward swap if the operator prefers not to provision separate scratch — at the cost of pinning the cap's worth of RAM. +* Pick the file backend if multi-tenant memory pressure matters: file-backend RSS stays at the working window (hundreds of MB), while swap-backend RSS pins to the cgroup cap (hundreds of GB). +* The runtime atomic switch is the right place for an operator-level policy: a controller can flip the global at startup based on cluster size or under a pressure signal. + +## Alternatives + +### Generic over `T: Pod` + +The columnar use case is `Vec` and the spec is explicit on `&[u64]`. +A generic over `T: bytemuck::Pod` would let callers spill `Vec` or `Vec` without manual casts. +The cost is API-wide: the handle must track element size, the file backend must validate alignment on read, and `take` round-tripping a `Vec` cannot return a `Vec` without reallocation. +The simpler `u64`-only design wins until a concrete non-`u64` consumer appears; an additive `pageout_pod` later would not break existing callers. + +### Per-pager configuration instead of a global atomic + +A `Pager` struct constructed with its backend would compose better than a global, especially in tests. +But the project intent is "the cluster runs on swap or on file, not both at once", and a global atomic encodes that operational reality directly. +A per-pager design would either duplicate the global flag at the struct level or invite confusion about which configuration wins. +We can add a per-instance constructor later for tests if the global proves awkward; the global stays as the production path. + +### One pager per use case (transient spill vs long-lived blob) + +The two named use cases differ only in handle lifetime and access pattern, not in storage. +Both want `pageout` once, both want random-offset reads, both want `Drop` to reclaim. +Two pagers would duplicate the global flag, the scratch directory, and the syscall code paths. +One pager covers both with the same API; UC1 calls `pageout` then `take`; UC2 calls `pageout` once and `read_at` many times. + +### Async API + +The dataflow callers run on synchronous timely worker threads, and bridging async out of an operator costs context switches and complicates lifetimes. +Sync `pread`/`writev` on a thread that already exists is the simplest correct choice for v1. +An async wrapper that offloads to a blocking pool can be added later without breaking the sync core. + +### `MADV_PAGEOUT` / `MADV_SWAPOUT` + +`MADV_PAGEOUT` (Linux 5.4+) actively reclaims pages synchronously, which is the closest to the file backend's eager-write semantics. +The cost is a synchronous, expensive syscall that we would issue from operator threads; under pressure this re-creates the direct-reclaim problem we are trying to escape. +`MADV_COLD` deactivates pages and lets the kernel reclaim asynchronously when it actually wants to, which matches our goal of moving work off the user thread. +We pick `MADV_COLD` for v1; if profiles show pages are not reclaimed quickly enough, `MADV_PAGEOUT` is a one-line swap on a feature-gate. + +### `O_TMPFILE` with the fd held in the handle + +`O_TMPFILE` creates an unnamed inode that auto-deletes when the last fd closes, so it would skip the explicit `unlink` step on reclaim. +The cost is one fd per handle: 100k live handles would exhaust the process fd ulimit and require an OS configuration change to operate at scale. +The chosen design instead opens a named file, writes, and closes the fd within `pageout`, so the handle holds 16 bytes and no fd, and reads reopen as needed. + +### Single shared scratch file with an offset table + +One file with handle-owned offsets reduces inode pressure and enables better physical layout. +It also requires a free-list, fragmentation handling, and a different reclaim story (truncation does not free arbitrary middles). +The complexity is unjustified at expected handle counts; revisit if inode count per handle becomes a measurable bottleneck. + +## Open questions + +* Should `pageout` accept `impl IntoIterator>` for ergonomics? + Iterators lose caller capacity reuse on the file path because we have no `&mut` access to put the cleared Vec back. + Recommend sticking with `&mut [Vec]`; revisit if the slice form proves awkward at call sites. + +* Should `set_backend` be allowed to flip multiple times during a process lifetime? + Yes for v1, since live LaunchDarkly flips are an explicit goal. + The per-handle stability rule (existing handles keep their backend) keeps this safe; document it in the rustdoc. + +* Should we add `read_at_into(&Handle, offset: usize, dst: &mut [u64])` for callers that have a sized buffer and do not want `Vec` semantics? + The columnar consumer is `Vec` and `read_at` already reuses the caller's allocation via `&mut Vec`. + Defer; add additively if a caller needs slice-only reads. + +* Should we add `pageout_each(&mut [Vec]) -> Vec` for timely-spill-style integration? + Timely's `BytesSpill` needs one handle per chunk; with the fd-less file backend, a "shared file under N handles" design becomes "N handles each pointing at independent files" and the `writev`-batching benefit disappears. + An alternative is `pageout_each` that writes all chunks into one shared scratch file and returns N handles each carrying `(scratch_id, byte_offset, byte_len)` plus a refcount so the file is unlinked once all handles drop. + Defer until the timely-spill integration concretely wants it; the existing `pageout` already covers the columnar primary use case. + +## Interaction with timely-dataflow spill + +Timely's `MergeQueue` exposes `BytesSpill`/`BytesFetch` traits ([PR 791](https://github.com/TimelyDataflow/timely-dataflow/pull/791) demonstrates the file-backed strategy). +The shapes differ: timely's `spill(&mut Vec, &mut Vec>)` takes N chunks and returns N independent fetch handles; the pager takes N chunks and returns one composite handle. +A `mz_timely_util::spill` adapter is the integration point, not the pager itself. +The adapter can be implemented in two ways once both pieces exist. + +The simple form calls `pager::pageout` once per chunk and stores each `pager::Handle` inside a `BytesFetch` impl. +This costs one scratch file per chunk; for 256 KiB chunks at 50 GiB total, that's roughly 200k inodes, which is workable on tmpfs but stressful on disk filesystems. +The richer form is the deferred `pageout_each` API above, which lets one writev produce one file with N handles and matches timely's design exactly. + +The element-type boundary needs care. +Timely passes `bytes::arc::Bytes` (byte-aligned); the pager wants `Vec` (8-byte aligned). +Materialize's columnar serialization already produces 8-byte-aligned bytes, so the adapter can cast where alignment is statically guaranteed and copy where it is not. +A future enhancement is a parallel byte-oriented pager API; deferred until the adapter exists and motivates it. + +The pager's swap backend is novel relative to timely's example: timely's "no-spill" baseline relies on the OS to manage memory, while the pager actively hints `MADV_COLD`. +This makes the swap backend an additional spill strategy that timely does not currently offer, suitable for cases where eager file write is too expensive but kernel-driven reclaim alone is too slow. diff --git a/src/ore/Cargo.toml b/src/ore/Cargo.toml index 8078cef5ccf14..583b117f655f1 100644 --- a/src/ore/Cargo.toml +++ b/src/ore/Cargo.toml @@ -93,6 +93,7 @@ mz-ore = { path = "../ore", features = ["id_gen", "chrono"] } proptest.workspace = true scopeguard.workspace = true serde_json.workspace = true +tempfile.workspace = true tokio.workspace = true tokio-test.workspace = true tracing-subscriber.workspace = true @@ -145,6 +146,7 @@ assert-no-tracing = [] assert = ["assert-no-tracing", "ctor", "tracing"] proptest = ["dep:proptest", "proptest-derive"] overflowing = ["assert"] +pager = ["dep:bytemuck", "libc", "rand", "dep:tracing"] [[test]] name = "future" @@ -167,6 +169,15 @@ name = "bytes" harness = false required-features = ["bytes", "region", "tracing"] +[[bench]] +name = "pager" +harness = false +required-features = ["pager"] + +[[example]] +name = "pager_merge" +required-features = ["pager"] + [package.metadata.cargo-udeps.ignore] # Only used in doc-tests. development = ["tokio-test"] diff --git a/src/ore/benches/pager.rs b/src/ore/benches/pager.rs new file mode 100644 index 0000000000000..f087dcc1c5fb8 --- /dev/null +++ b/src/ore/benches/pager.rs @@ -0,0 +1,141 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License in the LICENSE file at the +// root of this repository, or online at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#![cfg(feature = "pager")] + +use std::hint::black_box; +use std::path::PathBuf; +use std::time::{Duration, Instant}; + +use criterion::{BenchmarkId, Criterion, Throughput, criterion_group, criterion_main}; +use mz_ore::cast::CastFrom; +use mz_ore::pager::{self, Backend, Handle}; + +const PAGE_BYTES: usize = 4096; +const PAGE_U64S: usize = PAGE_BYTES / 8; + +fn ensure_scratch() { + static INIT: std::sync::Once = std::sync::Once::new(); + INIT.call_once(|| { + let dir: PathBuf = std::env::var_os("MZ_PAGER_SCRATCH") + .map(PathBuf::from) + .unwrap_or_else(std::env::temp_dir); + pager::set_scratch_dir(dir); + }); +} + +fn fill_payload(len_u64s: usize) -> Vec { + (0..u64::cast_from(len_u64s)).collect() +} + +/// Reads one `u64` from each page of `buf` to force the kernel to fault them in. +/// Returns a side-effecting sum so the compiler cannot elide the loads. +fn touch_every_page(buf: &[u64]) -> u64 { + let mut s: u64 = 0; + let mut i = 0; + while i < buf.len() { + s = s.wrapping_add(buf[i]); + i += PAGE_U64S; + } + s +} + +/// Round-trip a single-chunk payload through the pager and touch every page on +/// readback. Reuses the buffer between iterations so allocation/page-fault tax +/// is paid once at setup, not measured. +fn round_trip_single(c: &mut Criterion) { + ensure_scratch(); + let mut group = c.benchmark_group("pager/round_trip_touch/single"); + group.measurement_time(Duration::from_secs(5)); + for size_kib in [4usize, 64, 1024, 2048, 16384] { + let len = (size_kib * 1024) / 8; + for backend in [Backend::Swap, Backend::File] { + pager::set_backend(backend); + group.throughput(Throughput::Bytes(u64::cast_from(size_kib * 1024))); + group.bench_function(BenchmarkId::new(format!("{backend:?}"), size_kib), |b| { + b.iter_custom(|iters| { + let mut payload = fill_payload(len); + let mut tmp: Vec = Vec::with_capacity(len); + let start = Instant::now(); + for _ in 0..iters { + let mut chunks = [std::mem::take(&mut payload)]; + let h: Handle = pager::pageout(&mut chunks); + pager::take(h, &mut tmp); + black_box(touch_every_page(&tmp)); + payload = std::mem::take(&mut tmp); + tmp = Vec::with_capacity(len); + } + start.elapsed() + }); + }); + } + } + group.finish(); +} + +/// Round-trip a scatter-input (multiple chunks forming one logical 2 MiB block). +/// Measures the same touch-every-page readback pattern as `round_trip_single`. +fn round_trip_scatter_2mib(c: &mut Criterion) { + ensure_scratch(); + let mut group = c.benchmark_group("pager/round_trip_touch/scatter_2MiB"); + group.measurement_time(Duration::from_secs(5)); + let total_bytes: usize = 2 * 1024 * 1024; + for chunk_count in [1usize, 2, 8, 64] { + let chunk_bytes = total_bytes / chunk_count; + let chunk_len_u64s = chunk_bytes / 8; + for backend in [Backend::Swap, Backend::File] { + pager::set_backend(backend); + group.throughput(Throughput::Bytes(u64::cast_from(total_bytes))); + group.bench_function(BenchmarkId::new(format!("{backend:?}"), chunk_count), |b| { + b.iter_custom(|iters| { + let mut payload: Vec> = (0..chunk_count) + .map(|_| fill_payload(chunk_len_u64s)) + .collect(); + let mut tmp: Vec = Vec::with_capacity(total_bytes / 8); + let start = Instant::now(); + for _ in 0..iters { + let h: Handle = pager::pageout(payload.as_mut_slice()); + pager::take(h, &mut tmp); + black_box(touch_every_page(&tmp)); + // Rebuild the input from `tmp` for the next iteration: + // swap the consolidated buffer back into chunk 0, leave + // the other chunks empty (they were drained by the + // swap backend's `mem::take`). The file backend already + // preserved their capacity, so this still amortizes its + // allocation cost. + payload[0] = std::mem::take(&mut tmp); + tmp = Vec::with_capacity(total_bytes / 8); + // For chunk_count > 1, refill the trailing chunks by + // splitting payload[0] back into the original shape. + if chunk_count > 1 { + let mut head = std::mem::take(&mut payload[0]); + for i in 1..chunk_count { + let take_len = std::cmp::min(chunk_len_u64s, head.len()); + let tail = head.split_off(head.len() - take_len); + payload[i] = tail; + } + payload[0] = head; + } + } + start.elapsed() + }); + }); + } + } + group.finish(); +} + +criterion_group!(benches, round_trip_single, round_trip_scatter_2mib); +criterion_main!(benches); diff --git a/src/ore/examples/pager_merge.rs b/src/ore/examples/pager_merge.rs new file mode 100644 index 0000000000000..3903b37e1357d --- /dev/null +++ b/src/ore/examples/pager_merge.rs @@ -0,0 +1,214 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License in the LICENSE file at the +// root of this repository, or online at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Merge-batcher-style workload for the pager. +//! +//! Builds two chains of 2 MiB chunks (`--chain-gib` each), then performs a +//! merge pass that takes one chunk from each input, reads every cache line, +//! and emits two output chunks. Reports build/merge throughput. +//! +//! Run with constrained memory via `systemd-run --user --scope -p MemoryMax=...`. +//! +//! ```bash +//! cargo build --release --features pager --example pager_merge +//! systemd-run --user --scope -p MemoryMax=16G -p MemorySwapMax=64G --quiet \ +//! --setenv=MZ_PAGER_SCRATCH=/path/to/scratch \ +//! -- target/release/examples/pager_merge --chain-gib 16 --backend swap --threads 1 +//! ``` + +#![cfg(feature = "pager")] + +use std::env; +use std::path::PathBuf; +use std::sync::{Arc, Barrier}; +use std::thread; +use std::time::{Duration, Instant}; + +use mz_ore::cast::{CastFrom, CastLossy}; +use mz_ore::pager::{self, Backend, Handle}; + +const CHUNK_BYTES: usize = 2 * 1024 * 1024; +const CHUNK_U64: usize = CHUNK_BYTES / 8; +const CACHE_LINE_BYTES: usize = 64; +const CACHE_LINE_U64: usize = CACHE_LINE_BYTES / 8; + +fn main() { + let args: Vec = env::args().collect(); + let chain_gib: usize = parse_arg(&args, "--chain-gib", 16); + let threads: usize = parse_arg(&args, "--threads", 1).max(1); + let backend = parse_backend(&args); + let scratch: PathBuf = env::var_os("MZ_PAGER_SCRATCH") + .map(PathBuf::from) + .unwrap_or_else(env::temp_dir); + + pager::set_scratch_dir(scratch); + pager::set_backend(backend); + + let total_chain_bytes = chain_gib * 1024 * 1024 * 1024; + let per_thread_chain_bytes = total_chain_bytes / threads; + let chunks_per_chain = per_thread_chain_bytes / CHUNK_BYTES; + + println!( + "backend={backend:?} threads={threads} per_thread_chain_chunks={chunks_per_chain} chunk={CHUNK_BYTES}B total_chain={chain_gib}GiB" + ); + + let barrier = Arc::new(Barrier::new(threads)); + let start = Instant::now(); + let mut handles = Vec::with_capacity(threads); + for tid in 0..threads { + let barrier = Arc::clone(&barrier); + handles.push(thread::spawn(move || { + run_worker(tid, chunks_per_chain, &barrier) + })); + } + let mut per_thread = Vec::with_capacity(threads); + for h in handles { + per_thread.push(h.join().expect("worker panic")); + } + let total = start.elapsed(); + + // Total bytes through the merge across all threads (each thread reads + // 2 chain shares end-to-end, regardless of thread count). + let total_bytes = chunks_per_chain * threads * 2 * CHUNK_BYTES; + println!( + "wall: {:.2?} ({:.2} GiB/s through)", + total, + gib_per_sec(total_bytes, total) + ); + for (tid, t) in per_thread.iter().enumerate() { + println!( + " worker {tid}: build_a={:.2?} build_b={:.2?} merge={:.2?}", + t.build_a, t.build_b, t.merge + ); + } +} + +struct WorkerTimings { + build_a: Duration, + build_b: Duration, + merge: Duration, +} + +fn run_worker(_tid: usize, chunks_per_chain: usize, barrier: &Barrier) -> WorkerTimings { + barrier.wait(); + let (chain_a, build_a) = time(|| build_chain(chunks_per_chain)); + let (chain_b, build_b) = time(|| build_chain(chunks_per_chain)); + let (chain_c, merge) = time(|| merge_pass(chain_a, chain_b)); + drop(chain_c); + WorkerTimings { + build_a, + build_b, + merge, + } +} + +fn build_chain(n_chunks: usize) -> Vec { + let mut chain = Vec::with_capacity(n_chunks); + let mut buf: Vec = vec![0; CHUNK_U64]; + for i in 0..n_chunks { + // Fill with non-zero, position-dependent data so the kernel cannot + // share zero pages. + for (j, w) in buf.iter_mut().enumerate() { + *w = u64::cast_from(i) ^ u64::cast_from(j); + } + let mut chunks = [std::mem::take(&mut buf)]; + chain.push(pager::pageout(&mut chunks)); + // Reallocate; the swap backend stole the buffer, the file backend + // left an empty Vec with original capacity, but we don't keep it. + buf = vec![0; CHUNK_U64]; + } + chain +} + +fn merge_pass(a: Vec, b: Vec) -> Vec { + let n = a.len().min(b.len()); + let mut a: Vec> = a.into_iter().map(Some).collect(); + let mut b: Vec> = b.into_iter().map(Some).collect(); + let mut out = Vec::with_capacity(2 * n); + let mut tmp_a: Vec = Vec::with_capacity(CHUNK_U64); + let mut tmp_b: Vec = Vec::with_capacity(CHUNK_U64); + let mut sink: u64 = 0; + for i in 0..n { + let ha = a[i].take().expect("handle a present"); + let hb = b[i].take().expect("handle b present"); + pager::take(ha, &mut tmp_a); + pager::take(hb, &mut tmp_b); + // Touch every cache line of both inputs (1 u64 per 64-byte line). + sink = touch_cache_lines(&tmp_a, sink); + sink = touch_cache_lines(&tmp_b, sink); + // Emit two output chunks, simulating a merged run that doubles the + // chunk count. Each output is 2 MiB; we hand the original buffers + // straight to `pageout`, which transfers ownership cleanly on the + // swap backend. + { + let mut chunks = [std::mem::take(&mut tmp_a)]; + out.push(pager::pageout(&mut chunks)); + tmp_a = Vec::with_capacity(CHUNK_U64); + } + { + let mut chunks = [std::mem::take(&mut tmp_b)]; + out.push(pager::pageout(&mut chunks)); + tmp_b = Vec::with_capacity(CHUNK_U64); + } + } + std::hint::black_box(sink); + out +} + +#[inline] +fn touch_cache_lines(buf: &[u64], mut sink: u64) -> u64 { + let mut i = 0; + while i < buf.len() { + sink = sink.wrapping_add(buf[i]); + i += CACHE_LINE_U64; + } + sink +} + +fn time(f: impl FnOnce() -> T) -> (T, Duration) { + let start = Instant::now(); + let v = f(); + (v, start.elapsed()) +} + +fn gib_per_sec(bytes: usize, d: Duration) -> f64 { + let secs = d.as_secs_f64(); + if secs == 0.0 { + return 0.0; + } + let gib = f64::cast_lossy(bytes) / (1024.0 * 1024.0 * 1024.0); + gib / secs +} + +fn parse_arg(args: &[String], flag: &str, default: usize) -> usize { + args.iter() + .position(|a| a == flag) + .and_then(|i| args.get(i + 1)) + .and_then(|v| v.parse().ok()) + .unwrap_or(default) +} + +fn parse_backend(args: &[String]) -> Backend { + let pos = args + .iter() + .position(|a| a == "--backend") + .and_then(|i| args.get(i + 1)); + match pos.map(String::as_str) { + Some("file") => Backend::File, + Some("swap") => Backend::Swap, + Some(other) => panic!("unknown backend {other:?}; use 'swap' or 'file'"), + None => Backend::Swap, + } +} diff --git a/src/ore/src/lib.rs b/src/ore/src/lib.rs index 9cbafa6350c88..2c8a4c0e8d946 100644 --- a/src/ore/src/lib.rs +++ b/src/ore/src/lib.rs @@ -62,6 +62,9 @@ pub mod option; #[cfg_attr(nightly_doc_features, doc(cfg(feature = "overflowing")))] #[cfg(feature = "overflowing")] pub mod overflowing; +#[cfg_attr(nightly_doc_features, doc(cfg(feature = "pager")))] +#[cfg(feature = "pager")] +pub mod pager; #[cfg(not(target_family = "wasm"))] #[cfg(feature = "panic")] pub mod panic; diff --git a/src/ore/src/pager.rs b/src/ore/src/pager.rs new file mode 100644 index 0000000000000..75b4bc1f38dd1 --- /dev/null +++ b/src/ore/src/pager.rs @@ -0,0 +1,214 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License in the LICENSE file at the +// root of this repository, or online at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Explicit pager for cold data. See `doc/developer/design/20260504_pager.md`. + +use std::sync::atomic::{AtomicU8, Ordering}; + +mod file; +mod swap; + +pub use file::set_scratch_dir; + +use crate::pager::file::FileInner; +use crate::pager::swap::SwapInner; + +/// An opaque handle to data paged out via [`pageout`]. The handle's backend variant +/// is fixed at `pageout` time and is independent of any later `set_backend` call. +#[derive(Debug)] +pub struct Handle { + inner: HandleInner, +} + +#[derive(Debug)] +enum HandleInner { + Swap(SwapInner), + File(FileInner), +} + +impl Handle { + /// Returns the logical length of the handle's payload in `u64`s. + pub fn len(&self) -> usize { + match &self.inner { + HandleInner::Swap(s) => *s.prefix.last().unwrap_or(&0), + HandleInner::File(f) => f.len_u64s, + } + } + + /// Returns the logical length of the handle's payload in bytes (`len() * 8`). + pub fn len_bytes(&self) -> usize { + self.len() * 8 + } + + /// Returns `true` if the handle holds no data. + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + pub(crate) fn from_swap(inner: SwapInner) -> Self { + Self { + inner: HandleInner::Swap(inner), + } + } + + pub(crate) fn from_file(inner: FileInner) -> Self { + Self { + inner: HandleInner::File(inner), + } + } + + pub(crate) fn swap_inner(&self) -> Option<&SwapInner> { + match &self.inner { + HandleInner::Swap(s) => Some(s), + HandleInner::File(_) => None, + } + } + + pub(crate) fn file_inner(&self) -> Option<&FileInner> { + match &self.inner { + HandleInner::File(f) => Some(f), + HandleInner::Swap(_) => None, + } + } + + pub(crate) fn into_swap_inner(self) -> Option { + match self.inner { + HandleInner::Swap(s) => Some(s), + HandleInner::File(_) => None, + } + } + + pub(crate) fn into_file_inner(self) -> Option { + match self.inner { + HandleInner::File(f) => Some(f), + HandleInner::Swap(_) => None, + } + } +} + +/// Selects which backend stores paged-out data. +#[derive(Copy, Clone, Debug, Eq, PartialEq)] +pub enum Backend { + /// Hold allocations resident; hint the kernel via `MADV_COLD`. + Swap, + /// Write to a named scratch file; no file descriptor retained. + File, +} + +const BACKEND_SWAP: u8 = 0; +const BACKEND_FILE: u8 = 1; + +static BACKEND: AtomicU8 = AtomicU8::new(BACKEND_SWAP); + +/// Returns the currently active backend. +pub fn backend() -> Backend { + match BACKEND.load(Ordering::Relaxed) { + BACKEND_SWAP => Backend::Swap, + BACKEND_FILE => Backend::File, + _ => unreachable!("BACKEND atomic holds invalid discriminant"), + } +} + +/// Sets the active backend for future `pageout` calls. Existing handles are unaffected. +pub fn set_backend(b: Backend) { + let raw = match b { + Backend::Swap => BACKEND_SWAP, + Backend::File => BACKEND_FILE, + }; + BACKEND.store(raw, Ordering::Relaxed); +} + +/// Scatter pageout. Logical layout = chunks concatenated in order. +/// After return, each `Vec` in `chunks` is empty. +/// File backend preserves capacity; swap backend moves the alloc into the handle. +/// Empty input returns a `len == 0` handle and performs no I/O. +pub fn pageout(chunks: &mut [Vec]) -> Handle { + pageout_with(backend(), chunks) +} + +/// Same as [`pageout`], but selects the backend explicitly. Bypasses the global +/// atomic so callers (such as the column-pager layer) can dispatch per call +/// without racing other writers. +pub fn pageout_with(b: Backend, chunks: &mut [Vec]) -> Handle { + if total_len(chunks) == 0 { + return Handle::from_swap(SwapInner::new(Vec::new())); + } + match b { + Backend::Swap => swap::pageout_swap(chunks), + Backend::File => file::pageout_file(chunks), + } +} + +/// Reads multiple ranges. Output appended to `dst` in request order (concat). +/// Panics if any range is out of bounds. +pub fn read_at_many(handle: &Handle, ranges: &[(usize, usize)], dst: &mut Vec) { + match &handle.inner { + HandleInner::Swap(_) => swap::read_at_swap(handle, ranges, dst), + HandleInner::File(_) => file::read_at_file(handle, ranges, dst), + } +} + +/// Reads a single range. Convenience wrapper around `read_at_many`. +pub fn read_at(handle: &Handle, offset: usize, len: usize, dst: &mut Vec) { + read_at_many(handle, &[(offset, len)], dst); +} + +/// Consumes handle, writing the entire payload into `dst` (cleared first), then reclaims storage. +/// Swap fast path: single-chunk handle into empty `dst` swaps in place, no copy. +pub fn take(handle: Handle, dst: &mut Vec) { + match &handle.inner { + HandleInner::Swap(_) => swap::take_swap(handle, dst), + HandleInner::File(_) => file::take_file(handle, dst), + } +} + +fn total_len(chunks: &[Vec]) -> usize { + chunks.iter().map(|c| c.len()).sum() +} + +#[cfg(test)] +mod tests { + use super::*; + + #[mz_ore::test] + fn backend_round_trip() { + set_backend(Backend::File); + assert_eq!(backend(), Backend::File); + set_backend(Backend::Swap); + assert_eq!(backend(), Backend::Swap); + } +} + +#[cfg(test)] +mod dispatch_tests { + use super::*; + + #[mz_ore::test] + fn end_to_end_swap() { + set_backend(Backend::Swap); + let mut chunks = [vec![1u64, 2, 3, 4]]; + let h = pageout(&mut chunks); + assert_eq!(h.len(), 4); + assert!(chunks[0].is_empty()); + + let mut dst = Vec::new(); + read_at(&h, 1, 2, &mut dst); + assert_eq!(dst, vec![2, 3]); + + let mut dst2 = Vec::new(); + take(h, &mut dst2); + assert_eq!(dst2, vec![1, 2, 3, 4]); + } +} diff --git a/src/ore/src/pager/file.rs b/src/ore/src/pager/file.rs new file mode 100644 index 0000000000000..1e468d03f5478 --- /dev/null +++ b/src/ore/src/pager/file.rs @@ -0,0 +1,380 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License in the LICENSE file at the +// root of this repository, or online at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! File backend for the pager. See `mz_ore::pager` for the public API. + +use std::path::{Path, PathBuf}; + +use crate::cast::CastFrom; +use std::sync::atomic::{AtomicU64, Ordering}; +use std::sync::{Once, OnceLock}; + +static SCRATCH_DIR: OnceLock = OnceLock::new(); +static SUBDIR: OnceLock = OnceLock::new(); +static SCRATCH_ID: AtomicU64 = AtomicU64::new(0); +static SCRATCH_INIT: Once = Once::new(); + +/// Configures the scratch directory for the file backend. Idempotent across multiple +/// calls with the same path; logs and ignores subsequent calls with a different path. +pub fn set_scratch_dir(root: PathBuf) { + SCRATCH_INIT.call_once(|| { + if let Err(err) = init_subdir(&root) { + tracing::warn!(?root, %err, "mz_ore::pager: failed to initialize scratch subdir"); + } + let _ = SCRATCH_DIR.set(root.clone()); + }); + if let Some(existing) = SCRATCH_DIR.get() { + if *existing != root { + tracing::warn!( + ?root, + ?existing, + "mz_ore::pager scratch dir already set; ignoring", + ); + } + } +} + +fn init_subdir(root: &Path) -> std::io::Result<()> { + let nonce: u64 = rand::random(); + let pid = std::process::id(); + let subdir = root.join(format!("mz-pager-{pid}-{nonce:016x}")); + std::fs::create_dir_all(&subdir)?; + let _ = SUBDIR.set(subdir); + reap_stale(root); + Ok(()) +} + +fn reap_stale(root: &Path) { + let entries = match std::fs::read_dir(root) { + Ok(e) => e, + Err(err) => { + tracing::warn!(?root, %err, "mz_ore::pager: scratch dir scan failed"); + return; + } + }; + for entry in entries.flatten() { + let name = entry.file_name(); + let name = match name.to_str() { + Some(s) => s, + None => continue, + }; + let Some(rest) = name.strip_prefix("mz-pager-") else { + continue; + }; + let pid: u32 = match rest.split_once('-').and_then(|(p, _)| p.parse().ok()) { + Some(p) => p, + None => continue, + }; + if pid == std::process::id() { + continue; + } + if std::path::Path::new(&format!("/proc/{pid}")).exists() { + continue; + } + if let Err(err) = std::fs::remove_dir_all(entry.path()) { + tracing::warn!(path = ?entry.path(), %err, "mz_ore::pager: reap failed"); + } + } +} + +pub(crate) fn scratch_path(id: u64) -> PathBuf { + SUBDIR + .get() + .expect("mz_ore::pager file backend used before set_scratch_dir") + .join(format!("{id}.bin")) +} + +pub(crate) fn alloc_scratch_id() -> u64 { + SCRATCH_ID.fetch_add(1, Ordering::Relaxed) +} + +/// Storage for a file-backed handle. The file at `scratch_path(id)` holds the bytes. +/// No file descriptor is retained. +#[derive(Debug)] +pub(crate) struct FileInner { + pub(crate) id: u64, + pub(crate) len_u64s: usize, +} + +impl FileInner { + pub(crate) fn new(id: u64, len_u64s: usize) -> Self { + Self { id, len_u64s } + } +} + +impl Drop for FileInner { + fn drop(&mut self) { + let path = scratch_path(self.id); + if let Err(err) = std::fs::remove_file(&path) { + // ENOENT is fine: a successful `take` already unlinked. + if err.kind() != std::io::ErrorKind::NotFound { + tracing::warn!(?path, %err, "mz_ore::pager: failed to unlink scratch file"); + } + } + } +} + +use std::fs::File; +use std::io::IoSlice; + +use crate::pager::Handle; +use crate::pager::swap::{SwapInner, pageout_swap}; + +pub(crate) fn pageout_file(chunks: &mut [Vec]) -> Handle { + let total: usize = chunks.iter().map(|c| c.len()).sum(); + if total == 0 { + return Handle::from_swap(SwapInner::new(Vec::new())); + } + let id = alloc_scratch_id(); + let path = scratch_path(id); + match write_chunks(&path, chunks) { + Ok(()) => { + for c in chunks.iter_mut() { + c.clear(); + } + Handle::from_file(FileInner::new(id, total)) + } + Err(err) => { + tracing::warn!(?path, %err, "mz_ore::pager: file pageout failed; falling back to swap"); + let _ = std::fs::remove_file(&path); + pageout_swap(chunks) + } + } +} + +fn write_chunks(path: &Path, chunks: &[Vec]) -> std::io::Result<()> { + let file = File::options().write(true).create_new(true).open(path)?; + let mut slices: Vec> = chunks + .iter() + .filter(|c| !c.is_empty()) + .map(|c| IoSlice::new(bytemuck::cast_slice(c.as_slice()))) + .collect(); + write_all_vectored(&file, slices.as_mut_slice())?; + Ok(()) +} + +fn write_all_vectored(mut file: &File, mut slices: &mut [IoSlice<'_>]) -> std::io::Result<()> { + use std::io::Write; + while !slices.is_empty() { + let written = file.write_vectored(slices)?; + if written == 0 { + return Err(std::io::Error::new( + std::io::ErrorKind::WriteZero, + "write_vectored returned 0", + )); + } + IoSlice::advance_slices(&mut slices, written); + } + Ok(()) +} + +pub(crate) fn read_at_file(handle: &Handle, ranges: &[(usize, usize)], dst: &mut Vec) { + use std::os::unix::fs::FileExt; + + let inner = handle + .file_inner() + .expect("read_at_file called on non-file handle"); + let total = inner.len_u64s; + for &(off, len) in ranges { + let end = off.checked_add(len).expect("range offset+len overflow"); + assert!( + end <= total, + "read range out of bounds: {off}+{len} > {total}" + ); + } + let path = scratch_path(inner.id); + let file = match File::open(&path) { + Ok(f) => f, + Err(err) => panic!("mz_ore::pager: failed to open scratch file {path:?}: {err}"), + }; + + let coalesced = coalesce(ranges); + for (off, len) in coalesced { + let byte_off = u64::cast_from(off * 8); + let byte_len = len * 8; + let buf_start = dst.len(); + dst.resize(buf_start + len, 0); + let buf: &mut [u8] = bytemuck::cast_slice_mut(&mut dst[buf_start..buf_start + len]); + let mut filled = 0; + while filled < byte_len { + let pos = byte_off + u64::cast_from(filled); + let n = file + .read_at(&mut buf[filled..byte_len], pos) + .expect("pager pread failed"); + if n == 0 { + panic!("pager pread short: expected {byte_len} got {filled}"); + } + filled += n; + } + } +} + +fn coalesce(ranges: &[(usize, usize)]) -> Vec<(usize, usize)> { + let mut out: Vec<(usize, usize)> = Vec::with_capacity(ranges.len()); + for &(off, len) in ranges { + if let Some(last) = out.last_mut() { + if last.0 + last.1 == off { + last.1 += len; + continue; + } + } + out.push((off, len)); + } + out +} + +pub(crate) fn take_file(handle: Handle, dst: &mut Vec) { + use std::os::unix::fs::FileExt; + + let inner = handle + .into_file_inner() + .expect("take_file called on non-file handle"); + dst.clear(); + let path = scratch_path(inner.id); + let file = File::open(&path).unwrap_or_else(|err| panic!("pager take: open {path:?}: {err}")); + dst.resize(inner.len_u64s, 0); + let buf: &mut [u8] = bytemuck::cast_slice_mut(dst.as_mut_slice()); + let mut filled = 0; + while filled < buf.len() { + let pos = u64::cast_from(filled); + let n = file + .read_at(&mut buf[filled..], pos) + .unwrap_or_else(|err| panic!("pager take: pread {path:?}: {err}")); + if n == 0 { + panic!("pager take: short read at {filled}"); + } + filled += n; + } + drop(file); + // FileInner::drop will unlink the scratch file. + drop(inner); +} + +#[cfg(test)] +mod backend_tests { + use super::*; + + fn setup_dir() { + let _ = super::tests::shared_scratch(); + } + + #[mz_ore::test] + fn pageout_writes_file_and_clears_capacity() { + setup_dir(); + let mut chunks = [vec![10u64, 20, 30], vec![40, 50]]; + let cap_before_0 = chunks[0].capacity(); + let cap_before_1 = chunks[1].capacity(); + let h = pageout_file(&mut chunks); + assert_eq!(h.len(), 5); + assert!(chunks[0].is_empty()); + assert!(chunks[1].is_empty()); + // File backend preserves capacity: + assert_eq!(chunks[0].capacity(), cap_before_0); + assert_eq!(chunks[1].capacity(), cap_before_1); + + let inner = h.file_inner().expect("file inner"); + let path = scratch_path(inner.id); + assert!(path.exists()); + let bytes = std::fs::read(&path).expect("read scratch"); + assert_eq!(bytes.len(), 5 * 8); + } + + #[mz_ore::test] + fn file_read_at_basic() { + setup_dir(); + let mut chunks = [vec![1u64, 2, 3, 4, 5]]; + let h = pageout_file(&mut chunks); + let mut dst = Vec::new(); + read_at_file(&h, &[(1, 3)], &mut dst); + assert_eq!(dst, vec![2, 3, 4]); + } + + #[mz_ore::test] + fn file_read_at_many_concats_and_coalesces() { + setup_dir(); + let mut chunks = [vec![10u64, 20, 30, 40, 50, 60]]; + let h = pageout_file(&mut chunks); + let mut dst = Vec::new(); + // (0,2) and (2,2) are adjacent => single pread internally. + read_at_file(&h, &[(0, 2), (2, 2), (5, 1)], &mut dst); + assert_eq!(dst, vec![10, 20, 30, 40, 60]); + } + + #[mz_ore::test] + #[should_panic(expected = "out of bounds")] + fn file_read_at_panics_on_oob() { + setup_dir(); + let mut chunks = [vec![1u64, 2]]; + let h = pageout_file(&mut chunks); + let mut dst = Vec::new(); + read_at_file(&h, &[(0, 99)], &mut dst); + } + + #[mz_ore::test] + fn file_take_returns_data_and_unlinks() { + setup_dir(); + let mut chunks = [vec![7u64; 100]]; + let h = pageout_file(&mut chunks); + let inner_id = h.file_inner().unwrap().id; + let path = scratch_path(inner_id); + assert!(path.exists()); + let mut dst = Vec::new(); + take_file(h, &mut dst); + assert_eq!(dst, vec![7u64; 100]); + assert!(!path.exists(), "scratch file should be unlinked after take"); + } + + #[mz_ore::test] + fn file_drop_unlinks_when_not_taken() { + setup_dir(); + let mut chunks = [vec![1u64, 2, 3]]; + let h = pageout_file(&mut chunks); + let id = h.file_inner().unwrap().id; + let path = scratch_path(id); + assert!(path.exists()); + drop(h); + assert!(!path.exists(), "scratch file should be unlinked on drop"); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use tempfile::tempdir; + + static TEST_DIR: std::sync::OnceLock = std::sync::OnceLock::new(); + + pub(super) fn shared_scratch() -> &'static std::path::Path { + let dir = TEST_DIR.get_or_init(|| tempdir().expect("tempdir")); + set_scratch_dir(dir.path().to_owned()); + dir.path() + } + + #[mz_ore::test] + fn set_scratch_dir_creates_subdir() { + let root = shared_scratch(); + let subdir = SUBDIR.get().expect("subdir was initialized"); + assert!(subdir.exists()); + assert!(subdir.starts_with(root)); + assert!( + subdir + .file_name() + .unwrap() + .to_str() + .unwrap() + .starts_with("mz-pager-") + ); + } +} diff --git a/src/ore/src/pager/swap.rs b/src/ore/src/pager/swap.rs new file mode 100644 index 0000000000000..9c87cb24c20fa --- /dev/null +++ b/src/ore/src/pager/swap.rs @@ -0,0 +1,235 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License in the LICENSE file at the +// root of this repository, or online at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Swap backend for the pager. See `mz_ore::pager` for the public API. + +use crate::pager::Handle; + +/// Storage for a swap-backed handle. +#[derive(Debug)] +pub(crate) struct SwapInner { + /// Logical chunks; logical layout is concatenation in this order. + pub(crate) chunks: Vec>, + /// Cumulative element counts; `prefix[i]` = sum of `chunks[..i]` lengths. + /// `prefix[0] == 0`, `prefix.last() == total_len`. + pub(crate) prefix: Vec, +} + +impl SwapInner { + pub(crate) fn new(chunks: Vec>) -> Self { + let mut prefix = Vec::with_capacity(chunks.len() + 1); + prefix.push(0); + let mut sum = 0; + for c in &chunks { + sum += c.len(); + prefix.push(sum); + } + Self { chunks, prefix } + } + + pub(crate) fn total_len(&self) -> usize { + *self.prefix.last().unwrap_or(&0) + } +} + +pub(crate) fn pageout_swap(chunks: &mut [Vec]) -> Handle { + let mut taken: Vec> = Vec::with_capacity(chunks.len()); + for c in chunks.iter_mut() { + taken.push(std::mem::take(c)); + } + for c in &taken { + madvise_cold(c); + } + Handle::from_swap(SwapInner::new(taken)) +} + +#[cfg(target_os = "linux")] +fn madvise_cold(chunk: &[u64]) { + if chunk.is_empty() { + return; + } + let page = page_size(); + let base_ptr = chunk.as_ptr(); + let base_addr = base_ptr.addr(); + let len_bytes = chunk.len() * std::mem::size_of::(); + let aligned_start_addr = (base_addr + page - 1) & !(page - 1); + let aligned_end_addr = (base_addr + len_bytes) & !(page - 1); + if aligned_end_addr <= aligned_start_addr { + return; + } + let aligned_len = aligned_end_addr - aligned_start_addr; + // SAFETY: `aligned_start_addr` lies within `[base_addr, base_addr+len_bytes]`, + // i.e. inside the live `&[u64]`. Reconstructing the pointer via `byte_add` + // preserves provenance. + let aligned_ptr = unsafe { base_ptr.byte_add(aligned_start_addr - base_addr) } + .cast::() + .cast_mut(); + // SAFETY: pointer/length describe a fully page-aligned subrange contained + // within the live `&[u64]`. `MADV_COLD` does not mutate the contents. + unsafe { + libc::madvise(aligned_ptr, aligned_len, libc::MADV_COLD); + } +} + +#[cfg(not(target_os = "linux"))] +fn madvise_cold(_chunk: &[u64]) {} + +#[cfg(target_os = "linux")] +fn page_size() -> usize { + // SAFETY: `sysconf` with a valid argument is safe. + let raw = unsafe { libc::sysconf(libc::_SC_PAGESIZE) }; + usize::try_from(raw).expect("page size is positive and fits usize") +} + +#[cfg(not(target_os = "linux"))] +fn page_size() -> usize { + 4096 +} + +pub(crate) fn read_at_swap(handle: &Handle, ranges: &[(usize, usize)], dst: &mut Vec) { + let inner = handle + .swap_inner() + .expect("read_at_swap called on non-swap handle"); + let total = inner.total_len(); + let total_out: usize = ranges.iter().map(|(_, l)| *l).sum(); + dst.reserve(total_out); + for &(off, len) in ranges { + let end = off.checked_add(len).expect("range offset+len overflow"); + assert!( + end <= total, + "read range out of bounds: {off}+{len} > {total}" + ); + copy_range(inner, off, len, dst); + } +} + +fn copy_range(inner: &SwapInner, off: usize, len: usize, dst: &mut Vec) { + if len == 0 { + return; + } + let mut remaining = len; + let mut cur = off; + let mut idx = match inner.prefix.binary_search(&cur) { + Ok(i) => i, + Err(i) => i.saturating_sub(1), + }; + while remaining > 0 { + let chunk_start = inner.prefix[idx]; + let chunk = &inner.chunks[idx]; + let local = cur - chunk_start; + let take = std::cmp::min(remaining, chunk.len() - local); + dst.extend_from_slice(&chunk[local..local + take]); + cur += take; + remaining -= take; + idx += 1; + } +} + +pub(crate) fn take_swap(handle: Handle, dst: &mut Vec) { + let inner = match handle.into_swap_inner() { + Some(s) => s, + None => panic!("take_swap called on non-swap handle"), + }; + dst.clear(); + let mut chunks = inner.chunks; + if chunks.len() == 1 && dst.capacity() == 0 { + let only = chunks.pop().unwrap(); + *dst = only; + return; + } + let total: usize = chunks.iter().map(|c| c.len()).sum(); + dst.reserve(total); + for c in chunks { + dst.extend_from_slice(&c); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::pager::Handle; + + #[mz_ore::test] + fn pageout_takes_chunks_and_records_lengths() { + let a = vec![1u64, 2, 3]; + let b = vec![4u64, 5]; + let mut chunks = [a, b]; + let h: Handle = pageout_swap(&mut chunks); + assert_eq!(h.len(), 5); + assert!(chunks[0].is_empty()); + assert!(chunks[1].is_empty()); + } + + #[mz_ore::test] + fn read_at_within_single_chunk() { + let mut chunks = [vec![10u64, 11, 12, 13, 14]]; + let h = pageout_swap(&mut chunks); + let mut dst = Vec::new(); + read_at_swap(&h, &[(1, 3)], &mut dst); + assert_eq!(dst, vec![11, 12, 13]); + } + + #[mz_ore::test] + fn read_at_spans_chunks() { + let mut chunks = [vec![1u64, 2, 3], vec![4, 5, 6]]; + let h = pageout_swap(&mut chunks); + let mut dst = Vec::new(); + read_at_swap(&h, &[(2, 3)], &mut dst); + assert_eq!(dst, vec![3, 4, 5]); + } + + #[mz_ore::test] + fn read_at_many_concats() { + let mut chunks = [vec![1u64, 2, 3, 4, 5]]; + let h = pageout_swap(&mut chunks); + let mut dst = Vec::new(); + read_at_swap(&h, &[(0, 2), (3, 2)], &mut dst); + assert_eq!(dst, vec![1, 2, 4, 5]); + } + + #[mz_ore::test] + #[should_panic(expected = "out of bounds")] + fn read_at_panics_on_oob() { + let mut chunks = [vec![1u64, 2]]; + let h = pageout_swap(&mut chunks); + let mut dst = Vec::new(); + read_at_swap(&h, &[(1, 5)], &mut dst); + } + + #[mz_ore::test] + fn take_single_chunk_zero_copy() { + let v = vec![100u64; 1024]; + let ptr_before = v.as_ptr(); + let mut chunks = [v]; + let h = pageout_swap(&mut chunks); + let mut dst = Vec::new(); + take_swap(h, &mut dst); + assert_eq!(dst.len(), 1024); + assert_eq!( + dst.as_ptr(), + ptr_before, + "single-chunk take should be zero-copy" + ); + } + + #[mz_ore::test] + fn take_multi_chunk_concats() { + let mut chunks = [vec![1u64, 2], vec![3, 4, 5]]; + let h = pageout_swap(&mut chunks); + let mut dst = Vec::new(); + take_swap(h, &mut dst); + assert_eq!(dst, vec![1, 2, 3, 4, 5]); + } +} diff --git a/src/ore/tests/pager_integration.rs b/src/ore/tests/pager_integration.rs new file mode 100644 index 0000000000000..2436daacf88d9 --- /dev/null +++ b/src/ore/tests/pager_integration.rs @@ -0,0 +1,89 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License in the LICENSE file at the +// root of this repository, or online at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#![cfg(feature = "pager")] + +use mz_ore::pager::{Backend, Handle, pageout, read_at, set_backend, set_scratch_dir, take}; +use tempfile::tempdir; + +fn ensure_scratch() { + static INIT: std::sync::OnceLock = std::sync::OnceLock::new(); + let dir = INIT.get_or_init(|| tempdir().unwrap()); + set_scratch_dir(dir.path().to_owned()); +} + +#[test] // allow(test-attribute) +fn round_trip_swap() { + set_backend(Backend::Swap); + let payload: Vec = (0..1024).collect(); + let mut chunks = [payload.clone()]; + let h = pageout(&mut chunks); + let mut dst = Vec::new(); + take(h, &mut dst); + assert_eq!(dst, payload); +} + +#[test] // allow(test-attribute) +fn round_trip_file() { + ensure_scratch(); + set_backend(Backend::File); + let payload: Vec = (0..4096).collect(); + let mut chunks = [payload.clone()]; + let h = pageout(&mut chunks); + let mut dst = Vec::new(); + take(h, &mut dst); + assert_eq!(dst, payload); + set_backend(Backend::Swap); +} + +#[test] // allow(test-attribute) +fn handle_survives_backend_flip() { + ensure_scratch(); + set_backend(Backend::File); + let payload: Vec = (0..256).collect(); + let mut chunks = [payload.clone()]; + let h: Handle = pageout(&mut chunks); + + // Flip to Swap; existing handle should still be readable as File. + set_backend(Backend::Swap); + + let mut dst = Vec::new(); + read_at(&h, 0, payload.len(), &mut dst); + assert_eq!(dst, payload); + + let mut dst2 = Vec::new(); + take(h, &mut dst2); + assert_eq!(dst2, payload); +} + +#[test] // allow(test-attribute) +fn empty_input_yields_zero_len_handle() { + set_backend(Backend::Swap); + let mut chunks: [Vec; 0] = []; + let h = pageout(&mut chunks); + assert_eq!(h.len(), 0); + assert!(h.is_empty()); +} + +#[test] // allow(test-attribute) +fn scatter_round_trip() { + set_backend(Backend::Swap); + let mut chunks = [vec![1u64, 2, 3], vec![4, 5], vec![6, 7, 8, 9]]; + let h = pageout(&mut chunks); + assert_eq!(h.len(), 9); + let mut dst = Vec::new(); + take(h, &mut dst); + assert_eq!(dst, vec![1, 2, 3, 4, 5, 6, 7, 8, 9]); +} diff --git a/src/timely-util/Cargo.toml b/src/timely-util/Cargo.toml index c893177bcca52..83943fc96d622 100644 --- a/src/timely-util/Cargo.toml +++ b/src/timely-util/Cargo.toml @@ -17,6 +17,10 @@ harness = false name = "consolidating_column_builder" harness = false +[[bench]] +name = "column_pager" +harness = false + [dependencies] ahash.workspace = true bincode.workspace = true @@ -27,7 +31,8 @@ differential-dataflow.workspace = true either.workspace = true futures-util.workspace = true lgalloc.workspace = true -mz-ore = { path = "../ore", default-features = false, features = ["async", "process", "tracing", "test", "num-traits", "region", "differential-dataflow", "overflowing"] } +lz4_flex.workspace = true +mz-ore = { path = "../ore", default-features = false, features = ["async", "process", "tracing", "test", "num-traits", "region", "differential-dataflow", "overflowing", "pager"] } num-traits.workspace = true serde.workspace = true timely.workspace = true @@ -42,6 +47,7 @@ allocation-counter = { workspace = true, optional = true } [dev-dependencies] criterion.workspace = true +tempfile.workspace = true [features] default = ["mz-ore/default"] diff --git a/src/timely-util/benches/column_pager.rs b/src/timely-util/benches/column_pager.rs new file mode 100644 index 0000000000000..8fbf5adfd9f53 --- /dev/null +++ b/src/timely-util/benches/column_pager.rs @@ -0,0 +1,179 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0. + +//! Throughput benchmark for [`mz_timely_util::column_pager`]. +//! +//! Two shapes: +//! +//! * `rt`: round-trip cost of `page` immediately followed by `take` on a +//! freshly built column. Measures the full encode/decode path including +//! pager I/O and (for the lz4 axis) compression and decompression. +//! * `loop_`: operator-loop cost — refill an existing column then `page` it, +//! without `take`. Mirrors how a spill operator amortizes allocations and +//! measures the cheaper write-only side of the cycle. +//! +//! Axes: +//! +//! * Column size, in target uncompressed bytes: 4 KiB, 256 KiB, 4 MiB. +//! * Pager backend: `Swap`, `File`. +//! * Codec: uncompressed, lz4. +//! +//! ## Caveat: swap backend numbers are the warm fast path +//! +//! The pager's swap backend keeps the body `Vec` resident and hints +//! `MADV_COLD` to the kernel. This bench round-trips one column at a time +//! and never accumulates enough working set to exceed system RAM, so the +//! kernel never actually evicts. Swap-backend results therefore measure +//! `pageout = move-Vec-into-handle` and `take = move-Vec-out` plus +//! bookkeeping — essentially memcpy at the configured size — not the real +//! cost of a page-in from disk under memory pressure. +//! +//! To distinguish the cases, swap-backend results are labelled +//! `swap-warm` rather than `swap`. A separate `column_pager_pressure` +//! bench (TODO) will hold many paged handles alive under a constrained +//! cgroup (`systemd-run --user --scope -p MemoryMax=...`) so the kernel +//! is forced to evict, and time `take` on a cold handle. +//! +//! Run with: +//! +//! cargo bench -p mz-timely-util --bench column_pager + +use std::sync::Arc; + +use criterion::{BatchSize, Criterion, Throughput, criterion_group, criterion_main}; +use mz_ore::pager::{self, Backend}; +use mz_timely_util::column_pager::{ + Codec, ColumnPager, PageDecision, PageEvent, PageHint, PagedColumn, PagingPolicy, +}; +use mz_timely_util::columnar::Column; +use timely::container::PushInto; +use timely::dataflow::channels::ContainerBytes; + +/// Stub policy that always returns the configured decision. Records nothing. +struct AlwaysPage { + backend: Backend, + codec: Option, +} + +impl PagingPolicy for AlwaysPage { + fn decide(&self, _hint: PageHint) -> PageDecision { + PageDecision::Page { + backend: self.backend, + codec: self.codec, + } + } + fn record(&self, _event: PageEvent) {} +} + +/// Builds a `Column` whose serialized byte size is approximately +/// `target_bytes`. The actual size is reported by [`ContainerBytes::length_in_bytes`] +/// and used for throughput accounting. +fn build_column(target_bytes: usize) -> Column { + // i64 typed columns serialize to roughly 8 bytes per element plus header + // overhead. Aim a touch high and trust `length_in_bytes` for accounting. + let n = i64::try_from((target_bytes / 8).max(1)).expect("fits in i64"); + let mut c: Column = Default::default(); + for v in 0..n { + c.push_into(v); + } + c +} + +fn label(prefix: &str, target: usize, backend: Backend, codec: Option) -> String { + let size = match target { + n if n >= 1 << 20 => format!("{}MiB", n >> 20), + n if n >= 1 << 10 => format!("{}KiB", n >> 10), + n => format!("{n}B"), + }; + let codec = match codec { + None => "raw", + Some(Codec::Lz4) => "lz4", + }; + // `swap-warm` flags that this measures the in-memory fast path: the + // bench never builds enough working set to push the system into actual + // swap eviction, so swap-backend numbers reflect pageout/pagein as + // memcpy + bookkeeping, not kernel paging cost. See module docs. + let backend = match backend { + Backend::Swap => "swap-warm", + Backend::File => "file", + }; + format!("{prefix}/{size}/{backend}/{codec}") +} + +fn bench_round_trip(c: &mut Criterion, target: usize, backend: Backend, codec: Option) { + let policy: Arc = Arc::new(AlwaysPage { backend, codec }); + let cp = ColumnPager::new(policy); + let prototype = build_column(target); + let actual_bytes = prototype.length_in_bytes(); + + let mut group = c.benchmark_group("column_pager"); + group.throughput(Throughput::Bytes(u64::try_from(actual_bytes).unwrap())); + group.bench_function(label("rt", target, backend, codec), |b| { + b.iter_batched( + || build_column(target), + |mut col| { + let p = cp.page(&mut col); + let _ = cp.take(p); + }, + BatchSize::LargeInput, + ); + }); + group.finish(); +} + +fn bench_loop(c: &mut Criterion, target: usize, backend: Backend, codec: Option) { + let policy: Arc = Arc::new(AlwaysPage { backend, codec }); + let cp = ColumnPager::new(policy); + let prototype = build_column(target); + let actual_bytes = prototype.length_in_bytes(); + + let mut group = c.benchmark_group("column_pager"); + group.throughput(Throughput::Bytes(u64::try_from(actual_bytes).unwrap())); + group.bench_function(label("loop", target, backend, codec), |b| { + let mut col = build_column(target); + b.iter(|| { + // Operator loop: refill the column, then page it. Drop the paged + // result without `take`, simulating a write-only spill operator. + if col.length_in_bytes() == 0 { + col = build_column(target); + } + let paged: PagedColumn = cp.page(&mut col); + // Refill before next iteration so the column carries data again. + col = build_column(target); + std::mem::drop(paged); + }); + }); + group.finish(); +} + +fn benches(c: &mut Criterion) { + // The File backend writes to a scratch directory chosen at process + // startup; tests do this via `tempfile`. For the bench we use the + // platform default, which `pager::file` will create under + // `/tmp//...` if no override is set. + let scratch = std::env::temp_dir().join(format!("column-pager-bench-{}", std::process::id())); + let _ = std::fs::create_dir_all(&scratch); + pager::set_scratch_dir(scratch); + + let sizes = [4 * 1024, 256 * 1024, 4 * 1024 * 1024]; + let backends = [Backend::Swap, Backend::File]; + let codecs = [None, Some(Codec::Lz4)]; + + for &size in &sizes { + for &backend in &backends { + for &codec in &codecs { + bench_round_trip(c, size, backend, codec); + bench_loop(c, size, backend, codec); + } + } + } +} + +criterion_group!(column_pager_benches, benches); +criterion_main!(column_pager_benches); diff --git a/src/timely-util/src/column_pager.rs b/src/timely-util/src/column_pager.rs new file mode 100644 index 0000000000000..e66c9a31a2f81 --- /dev/null +++ b/src/timely-util/src/column_pager.rs @@ -0,0 +1,553 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License in the LICENSE file at the +// root of this repository, or online at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Column-aware pager. Pages [`Column`] instances out via [`mz_ore::pager`], +//! optionally compressing with lz4. +//! +//! The pager (`mz_ore::pager`) deals in `Vec` blobs and two backends. This +//! module adds: +//! +//! 1. A [`PagingPolicy`] trait that decides _whether_ to page out, _which +//! backend_, and _whether to compress_. Decisions live in the policy +//! implementation, not in the global atomic the pager exposes. +//! 2. A [`ColumnPager`] that drains a `Column` into a [`PagedColumn`] and +//! rehydrates it on demand. +//! 3. Lz4 frame-format compression as an optional codec. +//! +//! The serialization uses the existing [`ContainerBytes`] protocol on +//! `Column`, so we get a single byte layout that both raw and compressed +//! paths share. See `doc/developer/design/20260504_pager.md` for background. + +#![deny(missing_docs)] + +pub mod policy; + +use std::io::{self, Read}; +use std::sync::Arc; + +use columnar::Columnar; +use lz4_flex::frame::{FrameDecoder, FrameEncoder}; +use mz_ore::pager::{self, Backend, Handle}; +use timely::bytes::arc::BytesMut; +use timely::dataflow::channels::ContainerBytes; + +use crate::columnar::Column; + +// --------------------------------------------------------------------------- +// Codec +// --------------------------------------------------------------------------- + +/// Compression codec applied to a paged-out column. +#[derive(Copy, Clone, Debug, Eq, PartialEq)] +pub enum Codec { + /// lz4 frame format (`lz4_flex::frame`). Self-delimiting, streams via + /// `io::Read`/`io::Write`, no random access. + Lz4, +} + +// --------------------------------------------------------------------------- +// Policy +// --------------------------------------------------------------------------- + +/// Inputs to a pageout decision. +#[derive(Copy, Clone, Debug)] +pub struct PageHint { + /// Uncompressed body size in bytes (matches [`ContainerBytes::length_in_bytes`]). + pub len_bytes: usize, +} + +/// Outcome of a policy decision. +#[derive(Copy, Clone, Debug)] +pub enum PageDecision { + /// Keep the column resident; no I/O, no compression. + Skip, + /// Page out using the given backend and (optionally) codec. + Page { + /// Pager backend to use. + backend: Backend, + /// Compression codec, or `None` for raw bytes. + codec: Option, + }, +} + +/// Notifications the column-pager sends back to the policy. Implementations +/// typically forward to metrics counters. +#[derive(Debug)] +pub enum PageEvent { + /// A successful pageout. `bytes_in` is the uncompressed body size, + /// `bytes_out` is the on-storage payload size (after compression). + PagedOut { + /// Uncompressed body size handed to the pager. + bytes_in: usize, + /// On-storage payload size after compression and padding. + bytes_out: usize, + /// Backend selected by the policy. + backend: Backend, + /// Codec selected by the policy. + codec: Option, + }, + /// A successful page-in. `bytes` is the uncompressed body size delivered to + /// the caller. + PagedIn { + /// Uncompressed body size delivered to the caller. + bytes: usize, + }, + /// A pageout failure surfaced via the underlying pager. + Failed { + /// Backend that produced the error. + backend: Backend, + /// Underlying I/O error. + err: io::Error, + }, + /// A resident column has been dropped. Fires from [`ResidentTicket::drop`] + /// when the [`PagedColumn::Resident`] holding the ticket is consumed by + /// [`ColumnPager::take`] or dropped without being taken. Policies use this + /// to return budget allocated when [`PagingPolicy::decide`] answered + /// [`PageDecision::Skip`]. + ResidentReleased { + /// Uncompressed body size returned to the policy. + bytes: usize, + }, +} + +/// Decides whether/how to page a column out, and records page events. +/// +/// Implementations carry their own state (counters, atomics, configuration) +/// via interior mutability. Methods take `&self` so a single policy can be +/// shared across operator threads. +pub trait PagingPolicy: Send + Sync { + /// Returns the action to take for a column with the given hint. + fn decide(&self, hint: PageHint) -> PageDecision; + /// Records a pageout/pagein/failure event for metrics or adaptive decisions. + fn record(&self, event: PageEvent); +} + +// --------------------------------------------------------------------------- +// Meta + PagedColumn +// --------------------------------------------------------------------------- + +/// Sizing metadata captured at pageout time. Stored alongside the payload so +/// `take` can size buffers. +#[derive(Clone, Debug)] +pub struct Meta { + /// Uncompressed body size in bytes. + pub len_bytes: usize, +} + +/// A column whose body may be resident, paged out, or paged out and compressed. +/// +/// Each variant corresponds to one of the [`PageDecision`] outcomes. +pub enum PagedColumn { + /// Body kept resident. Returned when the policy answered + /// [`PageDecision::Skip`]. The accompanying [`ResidentTicket`] fires a + /// [`PageEvent::ResidentReleased`] when the variant is dropped or + /// consumed by [`ColumnPager::take`], so the policy can reclaim the + /// budget it granted in [`PagingPolicy::decide`]. + Resident(Column, ResidentTicket), + /// Raw `ContainerBytes` payload stored via [`pager::Handle`]. The backend + /// (Swap or File) is baked into the handle. + Paged { + /// Pager handle owning the raw payload. + handle: Handle, + /// Sizing metadata. + meta: Meta, + }, + /// Lz4-framed serialized form. The framed bytes themselves may live in + /// memory or in the pager (see [`CompressedInner`]). + Compressed { + /// Where the framed bytes live. + inner: CompressedInner, + /// Sizing metadata. + meta: Meta, + }, +} + +/// Drop guard that returns budget to a [`PagingPolicy`] when a +/// [`PagedColumn::Resident`] is destroyed. +/// +/// The ticket holds an `Arc` to the policy and the byte count it was charged +/// for at [`PagingPolicy::decide`] time. On drop it fires a +/// [`PageEvent::ResidentReleased`] event; the policy implementation decides +/// what to credit and where (local pool, shared pool, both). +pub struct ResidentTicket { + bytes: usize, + policy: Arc, +} + +impl Drop for ResidentTicket { + fn drop(&mut self) { + self.policy + .record(PageEvent::ResidentReleased { bytes: self.bytes }); + } +} + +/// Storage location for the lz4-framed bytes inside a compressed paged column. +pub enum CompressedInner { + /// Owned `Vec` held resident in the caller's address space. + Memory(Vec), + /// Framed bytes padded to a `u64` boundary and handed to the pager. The + /// frame trailer self-delimits, so the trailing pad is ignored on read. + Paged(Handle), +} + +// --------------------------------------------------------------------------- +// ColumnPager +// --------------------------------------------------------------------------- + +/// Pages typed [`Column`]s out and back in, driven by a [`PagingPolicy`]. +/// +/// Cheap to clone (it's an `Arc`). Hold one per operator if you want per-site +/// policy state, or share globally if you want one policy. +#[derive(Clone)] +pub struct ColumnPager { + policy: Arc, +} + +impl ColumnPager { + /// Constructs a column pager driven by `policy`. + pub fn new(policy: Arc) -> Self { + Self { policy } + } + + /// Drains `col` into a [`PagedColumn`]. After return `col` is left as a + /// fresh `Column::default()` (typed, empty), ready to be refilled by the + /// caller on the next loop iteration. + /// + /// Backend / codec semantics: + /// + /// * Uncompressed, [`Column::Align`]: the inner `Vec` is moved into + /// the pager handle with no copies. Swap backend keeps the allocation + /// resident; file backend writes it out and drops it. + /// * Uncompressed, other variants: the column is serialized via + /// [`ContainerBytes::into_bytes`] into a `Vec`, copied into a + /// u64-aligned `Vec`, then handed to the pager. + /// * Compressed: the column is serialized through an [`FrameEncoder`] + /// directly into the output buffer. No intermediate uncompressed + /// `Vec` is materialized. + pub fn page(&self, col: &mut Column) -> PagedColumn { + let len_bytes = col.length_in_bytes(); + let hint = PageHint { len_bytes }; + + let (backend, codec) = match self.policy.decide(hint) { + PageDecision::Skip => { + let ticket = ResidentTicket { + bytes: len_bytes, + policy: Arc::clone(&self.policy), + }; + return PagedColumn::Resident(std::mem::take(col), ticket); + } + PageDecision::Page { backend, codec } => (backend, codec), + }; + let meta = Meta { len_bytes }; + + match codec { + None => { + // Raw path: the body must end up as u64-aligned bytes for the + // pager. `Column::Align` already is; other variants are + // serialized and copied. + debug_assert_eq!(len_bytes % 8, 0); + let body: Vec = match std::mem::take(col) { + Column::Align(v) => v, + other => { + let mut buf = Vec::with_capacity(len_bytes); + other.into_bytes(&mut buf); + debug_assert_eq!(buf.len() % 8, 0); + bytemuck::allocation::pod_collect_to_vec::(&buf) + } + }; + let handle = pager::pageout_with(backend, &mut [body]); + self.policy.record(PageEvent::PagedOut { + bytes_in: len_bytes, + bytes_out: handle.len_bytes(), + backend, + codec: None, + }); + PagedColumn::Paged { handle, meta } + } + Some(Codec::Lz4) => { + // Stream serialized bytes straight into lz4 — no intermediate + // uncompressed `Vec`. + let mut out = Vec::with_capacity(len_bytes / 4); + { + let mut enc = FrameEncoder::new(&mut out); + col.into_bytes(&mut enc); + enc.finish().expect("lz4 finish into Vec is infallible"); + } + *col = Column::default(); + self.policy.record(PageEvent::PagedOut { + bytes_in: len_bytes, + bytes_out: out.len(), + backend, + codec: Some(Codec::Lz4), + }); + let inner = match backend { + Backend::Swap => CompressedInner::Memory(out), + Backend::File => { + let padded = pad_u8_to_u64(out); + let handle = pager::pageout_with(Backend::File, &mut [padded]); + CompressedInner::Paged(handle) + } + }; + PagedColumn::Compressed { inner, meta } + } + } + } + + /// Rehydrates `paged` into a [`Column`]. Consumes the handle and + /// reclaims its storage (file backend unlinks; swap backend drops the + /// `Vec`). + pub fn take(&self, paged: PagedColumn) -> Column { + match paged { + // `_ticket` drops here and fires `PageEvent::ResidentReleased`. + PagedColumn::Resident(c, _ticket) => c, + PagedColumn::Paged { handle, meta } => { + let mut body: Vec = Vec::with_capacity(handle.len()); + pager::take(handle, &mut body); + debug_assert_eq!(body.len() * 8, meta.len_bytes); + self.policy.record(PageEvent::PagedIn { + bytes: meta.len_bytes, + }); + Column::Align(body) + } + PagedColumn::Compressed { inner, meta } => { + let mut decoded = Vec::with_capacity(meta.len_bytes); + match inner { + CompressedInner::Memory(v) => { + FrameDecoder::new(&v[..]) + .read_to_end(&mut decoded) + .expect("lz4 decode from memory"); + } + CompressedInner::Paged(h) => { + let mut padded = Vec::with_capacity(h.len()); + pager::take(h, &mut padded); + let src: &[u8] = bytemuck::cast_slice(&padded); + FrameDecoder::new(src) + .read_to_end(&mut decoded) + .expect("lz4 decode from pager"); + } + } + debug_assert_eq!(decoded.len(), meta.len_bytes); + self.policy.record(PageEvent::PagedIn { + bytes: decoded.len(), + }); + // `BytesMut::from` wraps the `Vec` without copying; `freeze` + // produces the refcounted `Bytes` that `ContainerBytes` expects. + Column::from_bytes(BytesMut::from(decoded).freeze()) + } + } + } +} + +// --------------------------------------------------------------------------- +// Helpers +// --------------------------------------------------------------------------- + +/// Reinterprets `bytes` as a `Vec` by trailing-zero padding to a multiple +/// of 8 and copying. The lz4 frame trailer self-delimits so the trailing pad is +/// invisible to [`FrameDecoder`]. +fn pad_u8_to_u64(mut bytes: Vec) -> Vec { + let pad = bytes.len().next_multiple_of(8) - bytes.len(); + if pad != 0 { + bytes.resize(bytes.len() + pad, 0); + } + debug_assert_eq!(bytes.len() % 8, 0); + // `Vec` and `Vec` have different layouts (size + align), so we + // can't transmute the allocation. Copy into a fresh, properly aligned + // `Vec`. The cost is one `len_bytes/8`-word memcpy per pageout. + let len_u64s = bytes.len() / 8; + let mut out = vec![0u64; len_u64s]; + let dst: &mut [u8] = bytemuck::cast_slice_mut(&mut out); + dst.copy_from_slice(&bytes); + out +} + +// --------------------------------------------------------------------------- +// Tests +// --------------------------------------------------------------------------- + +#[cfg(test)] +#[allow(clippy::clone_on_ref_ptr)] +mod tests { + use std::sync::atomic::{AtomicUsize, Ordering}; + + use columnar::Index; + use timely::container::PushInto; + + use super::*; + + /// Promotes a typed policy `Arc` to `Arc`. Hides the + /// unsize coercion behind a `clone()` so the trait object is constructed + /// without the now-discouraged `as` cast. + fn as_dyn(p: &Arc) -> Arc { + p.clone() + } + + /// Recording policy: configurable decision, counts events. + struct TestPolicy { + decision: PageDecision, + out: AtomicUsize, + r#in: AtomicUsize, + } + + impl TestPolicy { + fn new(decision: PageDecision) -> Arc { + Arc::new(Self { + decision, + out: AtomicUsize::new(0), + r#in: AtomicUsize::new(0), + }) + } + } + + impl PagingPolicy for TestPolicy { + fn decide(&self, _hint: PageHint) -> PageDecision { + self.decision + } + fn record(&self, event: PageEvent) { + match event { + PageEvent::PagedOut { .. } => { + self.out.fetch_add(1, Ordering::Relaxed); + } + PageEvent::PagedIn { .. } => { + self.r#in.fetch_add(1, Ordering::Relaxed); + } + PageEvent::ResidentReleased { .. } | PageEvent::Failed { .. } => {} + } + } + } + + /// Builds a sample typed column of `i64`s. + fn sample_typed() -> Column { + let mut col: Column = Default::default(); + for v in 0i64..1024 { + col.push_into(v); + } + col + } + + /// Drains a column into a `Vec` for comparison via `borrow`. + fn collect_i64(col: &Column) -> Vec { + col.borrow().into_index_iter().copied().collect() + } + + #[mz_ore::test] + fn skip_policy_keeps_resident() { + let policy = TestPolicy::new(PageDecision::Skip); + let cp = ColumnPager::new(as_dyn(&policy)); + let mut col = sample_typed(); + let paged = cp.page(&mut col); + assert!(matches!(paged, PagedColumn::Resident(_, _))); + let rt = cp.take(paged); + assert_eq!(collect_i64(&rt), (0i64..1024).collect::>()); + assert_eq!(policy.out.load(Ordering::Relaxed), 0); + assert_eq!(policy.r#in.load(Ordering::Relaxed), 0); + } + + #[mz_ore::test] + fn round_trip_swap_uncompressed() { + let policy = TestPolicy::new(PageDecision::Page { + backend: Backend::Swap, + codec: None, + }); + let cp = ColumnPager::new(as_dyn(&policy)); + let mut col = sample_typed(); + let paged = cp.page(&mut col); + assert!(matches!(paged, PagedColumn::Paged { .. })); + let rt = cp.take(paged); + assert_eq!(collect_i64(&rt), (0i64..1024).collect::>()); + assert_eq!(policy.out.load(Ordering::Relaxed), 1); + assert_eq!(policy.r#in.load(Ordering::Relaxed), 1); + } + + #[mz_ore::test] + fn round_trip_swap_lz4() { + let policy = TestPolicy::new(PageDecision::Page { + backend: Backend::Swap, + codec: Some(Codec::Lz4), + }); + let cp = ColumnPager::new(as_dyn(&policy)); + let mut col = sample_typed(); + let paged = cp.page(&mut col); + assert!(matches!( + paged, + PagedColumn::Compressed { + inner: CompressedInner::Memory(_), + .. + } + )); + let rt = cp.take(paged); + assert_eq!(collect_i64(&rt), (0i64..1024).collect::>()); + } + + #[mz_ore::test] + fn round_trip_file_uncompressed() { + let dir = tempfile::tempdir().unwrap(); + pager::set_scratch_dir(dir.path().to_path_buf()); + let policy = TestPolicy::new(PageDecision::Page { + backend: Backend::File, + codec: None, + }); + let cp = ColumnPager::new(as_dyn(&policy)); + let mut col = sample_typed(); + let paged = cp.page(&mut col); + assert!(matches!(paged, PagedColumn::Paged { .. })); + let rt = cp.take(paged); + assert_eq!(collect_i64(&rt), (0i64..1024).collect::>()); + } + + #[mz_ore::test] + fn round_trip_file_lz4() { + let dir = tempfile::tempdir().unwrap(); + pager::set_scratch_dir(dir.path().to_path_buf()); + let policy = TestPolicy::new(PageDecision::Page { + backend: Backend::File, + codec: Some(Codec::Lz4), + }); + let cp = ColumnPager::new(as_dyn(&policy)); + let mut col = sample_typed(); + let paged = cp.page(&mut col); + assert!(matches!( + paged, + PagedColumn::Compressed { + inner: CompressedInner::Paged(_), + .. + } + )); + let rt = cp.take(paged); + assert_eq!(collect_i64(&rt), (0i64..1024).collect::>()); + } + + #[mz_ore::test] + fn align_variant_fast_path() { + // Construct an Align column directly to exercise the move-only raw path. + let policy = TestPolicy::new(PageDecision::Page { + backend: Backend::Swap, + codec: None, + }); + let cp = ColumnPager::new(as_dyn(&policy)); + let body: Vec = (1u64..=512).collect(); + let mut col: Column = Column::Align(body.clone()); + let paged = cp.page(&mut col); + assert!(matches!(paged, PagedColumn::Paged { .. })); + // After paging an Align variant, `col` is reset to the typed default. + assert!(matches!(col, Column::Typed(_))); + let rt = cp.take(paged); + // Round-tripped column should produce identical bytes. + match rt { + Column::Align(v) => assert_eq!(v, body), + other => panic!("expected Align, got {:?}", std::mem::discriminant(&other)), + } + } +} diff --git a/src/timely-util/src/column_pager/policy.rs b/src/timely-util/src/column_pager/policy.rs new file mode 100644 index 0000000000000..b631fcb115a05 --- /dev/null +++ b/src/timely-util/src/column_pager/policy.rs @@ -0,0 +1,288 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License in the LICENSE file at the +// root of this repository, or online at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Concrete [`PagingPolicy`] implementations. +//! +//! Today: [`TieredPolicy`], a two-tier byte budget where each Timely worker +//! gets a fixed local budget and falls back to a process-wide shared pool when +//! the local pool is exhausted. + +use std::cell::RefCell; +use std::sync::atomic::{AtomicUsize, Ordering}; + +use mz_ore::pager::Backend; + +use crate::column_pager::{Codec, PageDecision, PageEvent, PageHint, PagingPolicy}; + +/// A two-tier byte budget for resident columns. +/// +/// Each Timely worker thread draws first from a fixed per-worker pool of +/// `per_worker_budget` bytes. When a worker's pool is exhausted, it consults +/// the shared process-wide pool of `shared_budget` bytes (set at construction +/// via [`TieredPolicy::new`]). If both are full, [`PagingPolicy::decide`] +/// returns [`PageDecision::Page`] and the column is paged out via the +/// configured `backend` + `codec`. +/// +/// ## Per-worker state via thread-local storage +/// +/// Worker state lives in a `thread_local!` static, so each OS thread (= each +/// Timely worker, in current Materialize deployments) sees its own +/// `WorkerState`. This means **at most one `TieredPolicy` instance per +/// process** — a second instance would share the same `LOCAL` static and +/// corrupt the first instance's accounting. +/// +/// ## Release order +/// +/// On [`PageEvent::ResidentReleased`], the policy returns budget to the shared +/// pool first (so other workers unblock sooner), then to the local pool. +/// +/// ## Contention +/// +/// The shared pool is a single [`AtomicUsize`]. Only the cold path (local +/// exhausted) touches it. Per-byte CAS is fine at current page granularity; +/// if profiles show contention we can switch to chunk reservations. +pub struct TieredPolicy { + per_worker_budget: usize, + shared: AtomicUsize, + backend: Backend, + codec: Option, +} + +thread_local! { + static LOCAL: RefCell> = const { RefCell::new(None) }; +} + +/// Per-worker state. Initialized lazily on the first `with_local` call so the +/// `thread_local!` static doesn't need to know `per_worker_budget` up front. +#[derive(Debug)] +struct WorkerState { + /// Remaining bytes in the local pool. + remaining: usize, + /// Bytes the worker currently owes back to its local pool. + locally_owed: usize, + /// Bytes the worker currently owes back to the shared pool. + shared_owed: usize, +} + +impl TieredPolicy { + /// Constructs a tiered policy. Total budget is + /// `per_worker_budget * workers + shared_budget`. The first `decide` call + /// from each worker initializes that worker's local pool. + /// + /// `backend` and `codec` are used for the [`PageDecision::Page`] outcome + /// when both pools are exhausted. + pub fn new( + per_worker_budget: usize, + shared_budget: usize, + backend: Backend, + codec: Option, + ) -> Self { + Self { + per_worker_budget, + shared: AtomicUsize::new(shared_budget), + backend, + codec, + } + } + + /// Returns the current shared-pool remaining size in bytes. Useful for + /// metrics or tests. + pub fn shared_remaining(&self) -> usize { + self.shared.load(Ordering::Relaxed) + } + + fn with_local(&self, f: impl FnOnce(&mut WorkerState) -> R) -> R { + LOCAL.with(|cell| { + let mut borrow = cell.borrow_mut(); + let state = borrow.get_or_insert_with(|| WorkerState { + remaining: self.per_worker_budget, + locally_owed: 0, + shared_owed: 0, + }); + f(state) + }) + } +} + +impl PagingPolicy for TieredPolicy { + fn decide(&self, hint: PageHint) -> PageDecision { + self.with_local(|s| { + // Local pool first. + if s.remaining >= hint.len_bytes { + s.remaining -= hint.len_bytes; + s.locally_owed += hint.len_bytes; + return PageDecision::Skip; + } + // Shared pool fallback. + if try_consume(&self.shared, hint.len_bytes) { + s.shared_owed += hint.len_bytes; + return PageDecision::Skip; + } + // Both exhausted — page out. + PageDecision::Page { + backend: self.backend, + codec: self.codec, + } + }) + } + + fn record(&self, event: PageEvent) { + let PageEvent::ResidentReleased { bytes } = event else { + return; + }; + self.with_local(|s| { + // Return to shared first so other workers unblock sooner. + let from_shared = bytes.min(s.shared_owed); + if from_shared > 0 { + s.shared_owed -= from_shared; + self.shared.fetch_add(from_shared, Ordering::Relaxed); + } + let to_local = bytes - from_shared; + if to_local > 0 { + debug_assert!( + s.locally_owed >= to_local, + "release exceeds locally_owed (releasing {to_local}, owed {})", + s.locally_owed, + ); + s.locally_owed -= to_local; + s.remaining += to_local; + } + }); + } +} + +/// Atomically subtracts `want` from `atomic` if at least `want` is available. +/// Returns `true` on success. +fn try_consume(atomic: &AtomicUsize, want: usize) -> bool { + let mut cur = atomic.load(Ordering::Relaxed); + loop { + if cur < want { + return false; + } + match atomic.compare_exchange_weak(cur, cur - want, Ordering::AcqRel, Ordering::Relaxed) { + Ok(_) => return true, + Err(actual) => cur = actual, + } + } +} + +// --------------------------------------------------------------------------- +// Tests +// --------------------------------------------------------------------------- + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use timely::container::PushInto; + + use crate::column_pager::{ColumnPager, PagedColumn}; + use crate::columnar::Column; + + use super::*; + + fn sample(n: i64) -> Column { + let mut c: Column = Default::default(); + for v in 0..n { + c.push_into(v); + } + c + } + + /// Promotes a typed policy `Arc` to `Arc` without + /// triggering `clippy::clone_on_ref_ptr` or `clippy::as_conversions`. + fn as_dyn(p: &Arc) -> Arc { + #[allow(clippy::clone_on_ref_ptr)] + p.clone() + } + + /// All allocations fit in the per-worker pool. + #[mz_ore::test] + fn fits_in_local() { + let policy = Arc::new(TieredPolicy::new(64 * 1024, 0, Backend::Swap, None)); + let cp = ColumnPager::new(as_dyn(&policy)); + let mut col = sample(256); + let p = cp.page(&mut col); + assert!(matches!(p, PagedColumn::Resident(_, _))); + drop(p); // Drop fires ResidentReleased; budget returns. + } + + /// Local pool exhausted, shared pool covers the rest. + #[mz_ore::test] + fn spills_to_shared() { + let policy = Arc::new(TieredPolicy::new(0, 64 * 1024, Backend::Swap, None)); + let cp = ColumnPager::new(as_dyn(&policy)); + let mut col = sample(256); + let before = policy.shared_remaining(); + let p = cp.page(&mut col); + assert!(matches!(p, PagedColumn::Resident(_, _))); + let after = policy.shared_remaining(); + assert!(after < before, "shared pool should be consumed"); + drop(p); + assert_eq!( + policy.shared_remaining(), + before, + "release should refund the shared pool", + ); + } + + /// Both pools exhausted: pageout is forced. + #[mz_ore::test] + fn exhausted_pages_out() { + let policy = Arc::new(TieredPolicy::new(0, 0, Backend::Swap, None)); + let cp = ColumnPager::new(as_dyn(&policy)); + let mut col = sample(256); + let p = cp.page(&mut col); + assert!(matches!(p, PagedColumn::Paged { .. })); + } + + /// Local refill: a held Resident locks budget; dropping it frees space + /// for the next allocation. + #[mz_ore::test] + fn release_refills_local() { + let policy = Arc::new(TieredPolicy::new(4 * 1024, 0, Backend::Swap, None)); + let cp = ColumnPager::new(as_dyn(&policy)); + + // First allocation fits. + let mut col = sample(256); + let p1 = cp.page(&mut col); + assert!(matches!(p1, PagedColumn::Resident(_, _))); + + // Second allocation overflows local (no shared) -> page out. + let mut col2 = sample(256); + let p2 = cp.page(&mut col2); + assert!(matches!(p2, PagedColumn::Paged { .. })); + + // Releasing the first should refill local; a third allocation now + // fits resident again. + drop(p1); + drop(p2); + let mut col3 = sample(256); + let p3 = cp.page(&mut col3); + assert!(matches!(p3, PagedColumn::Resident(_, _))); + } + + #[mz_ore::test] + fn try_consume_atomicity() { + let a = AtomicUsize::new(10); + assert!(try_consume(&a, 4)); + assert_eq!(a.load(Ordering::Relaxed), 6); + assert!(!try_consume(&a, 7)); + assert_eq!(a.load(Ordering::Relaxed), 6); + assert!(try_consume(&a, 6)); + assert_eq!(a.load(Ordering::Relaxed), 0); + assert!(!try_consume(&a, 1)); + } +} diff --git a/src/timely-util/src/lib.rs b/src/timely-util/src/lib.rs index 57474b58cea34..61f5801e86ca7 100644 --- a/src/timely-util/src/lib.rs +++ b/src/timely-util/src/lib.rs @@ -19,6 +19,7 @@ pub mod activator; pub mod antichain; pub mod builder_async; pub mod capture; +pub mod column_pager; pub mod columnar; pub mod columnation; pub mod containers;