Skip to content

Chunk abstraction for collections, merge batching, and batches.#744

Draft
frankmcsherry wants to merge 4 commits into
TimelyDataflow:master-nextfrom
frankmcsherry:chunk_basis
Draft

Chunk abstraction for collections, merge batching, and batches.#744
frankmcsherry wants to merge 4 commits into
TimelyDataflow:master-nextfrom
frankmcsherry:chunk_basis

Conversation

@frankmcsherry

Copy link
Copy Markdown
Member

This PR introduces a trait Chunk that can be used to back collections, merger batchers, and in a list back a whole batch itself. This is meant to streamline implementation, and remove unnecessary distinctions. Also a bit of an experiment, to see just how small these implementations can get. At the moment .. not tiny .. but not enormous either.

frankmcsherry and others added 3 commits June 11, 2026 12:31
col_chunk was an early columnar Chunk mirroring ord_neu; the phase-2 TrieChunk
supersedes it and fixes its recorded limitations (decompress/recompress merge,
the singleton-vs-logical-count prefix re-walk, val-boundary-only cuts).
Carrying it taxed every chunk_basis modification with a re-application to a full
Chunk impl slated for deletion, so it goes early. The chunks example drops its
`colchunk` mode.

Co-Authored-By: Claude Opus 4.8 (1M context) <noreply@anthropic.com>
…y updates

Two merge-batcher / chunker fixes for the bfs/probe regression (plan 1.5, 1.4):

1.5 — VecChunk's SizableContainer absorbs to TARGET updates (len >= TARGET,
ensure_capacity reserves TARGET) instead of timely's byte-derived buffer size,
so chunks arrive pre-graded rather than re-melded downstream.

1.4 — MergeBatcher's geometric ladder weighs chains by summed updates, not
chunk counts: regrading decouples the two, so a trickle of single-update chunks
re-merged the head chain on every insert. A chain is immutable until merged, so
the weight is cached alongside it (chains: Vec<(usize, Vec<Chunk>)>).

Also refocuses the Merger trait: the bundled account() -> (records, size,
capacity, allocations) splits into len() -> usize (update count, drives the
ladder and the logger's records field) and a defaulted allocation() -> (size,
capacity, allocations) for memory telemetry. BatcherEvent's shape is unchanged.
NOTE: breaking change for out-of-tree Merger implementors (e.g. Materialize) —
rename account -> len, optionally override allocation.

chunks 100k/200k (u64 probes): ~127ms -> ~100ms queries-complete.

Co-Authored-By: Claude Opus 4.8 (1M context) <noreply@anthropic.com>
1.2 — invert the merge surface so binary merge_pair is the required primitive:
merge_pair(&mut (usize,Self), &mut (usize,Self), out) is what the harnesses
(merge_chains, the batch merger) drive directly; the k-way `merge` becomes a
provided dispatcher (arity 0 / 2 / unimplemented!); `prune` loses its default
(it was the only arity-1 merge caller). Backends with a genuine k-way merge
(vec_chunk's merge_buf) override `merge`.

1.3 — vec_chunk::merge_pair is a dedicated two-pointer binary merge: one gallop
pins the horizon, disjoint runs bulk-copy via extend_from_slice, only collisions
consolidate element-wise. merge_buf stays as the k-way override and correctness
reference. Chunks example ~100ms -> ~83ms.

Fixes a latent bug in the spike's run-copy: it pinned the horizon at the lesser
last (key,val), which is wrong for multi-chunk chains — a (key,val) group that
straddles a chunk boundary on one side and overlaps times with the other gets
the other side's whole group merged before its straddled continuation arrives,
emitting duplicate, out-of-order (key,val,time) entries. The horizon must be
(key,val,time), as the reference merge_buf already used. The spike's merge tests
are single-chunk and never reached it.

New property test merge_pair_matches_reference: 300 random multi-chunk merges
(tiny chunks force straddling) vs a union-consolidate oracle.

Co-Authored-By: Claude Opus 4.8 (1M context) <noreply@anthropic.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

1 participant