Skip to content

persist: expose API for writing/reading "free-standing" batches #32513

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions src/persist-client/src/internal/encoding.rs
Original file line number Diff line number Diff line change
Expand Up @@ -61,12 +61,17 @@ use crate::internal::trace::{
use crate::read::{LeasedReaderId, READER_LEASE_DURATION};
use crate::{PersistConfig, ShardId, WriterId, cfg};

/// A key and value `Schema` of data written to a batch or shard.
#[derive(Debug)]
pub struct Schemas<K: Codec, V: Codec> {
// TODO: Remove the Option once this finishes rolling out and all shards
// have a registered schema.
/// Id under which this schema is registered in the shard's schema registry,
/// if any.
pub id: Option<SchemaId>,
/// Key `Schema`.
pub key: Arc<K::Schema>,
/// Value `Schema`.
pub val: Arc<V::Schema>,
}

Expand Down
6 changes: 3 additions & 3 deletions src/persist-client/src/internal/state.rs
Original file line number Diff line number Diff line change
Expand Up @@ -869,7 +869,7 @@ impl<T: Ord> Ord for HollowBatch<T> {
}

impl<T: Timestamp + Codec64 + Sync> HollowBatch<T> {
pub fn part_stream<'a>(
pub(crate) fn part_stream<'a>(
&'a self,
shard_id: ShardId,
blob: &'a dyn Blob,
Expand Down Expand Up @@ -973,11 +973,11 @@ impl<T> HollowBatch<T> {
self.parts.iter().map(|x| x.inline_bytes()).sum()
}

pub fn is_empty(&self) -> bool {
pub(crate) fn is_empty(&self) -> bool {
self.parts.is_empty()
}

pub fn part_count(&self) -> usize {
pub(crate) fn part_count(&self) -> usize {
self.parts.len()
}

Expand Down
209 changes: 203 additions & 6 deletions src/persist-client/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,27 +21,39 @@ use std::sync::Arc;

use differential_dataflow::difference::Semigroup;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::trace::Description;
use mz_build_info::{BuildInfo, build_info};
use mz_dyncfg::ConfigSet;
use mz_ore::{instrument, soft_assert_or_log};
use mz_persist::location::{Blob, Consensus, ExternalError};
use mz_persist_types::schema::SchemaId;
use mz_persist_types::{Codec, Codec64, Opaque};
use timely::progress::Timestamp;
use mz_proto::{IntoRustIfSome, ProtoType};
use semver::Version;
use timely::progress::{Antichain, Timestamp};

use crate::async_runtime::IsolatedRuntime;
use crate::batch::{
BATCH_DELETE_ENABLED, BLOB_TARGET_SIZE, Batch, BatchBuilder, BatchBuilderConfig,
BatchBuilderInternal, BatchParts, ProtoBatch,
};
use crate::cache::{PersistClientCache, StateCache};
use crate::cfg::PersistConfig;
use crate::cfg::{COMPACTION_MEMORY_BOUND_BYTES, PersistConfig};
use crate::critical::{CriticalReaderId, SinceHandle};
use crate::error::InvalidUsage;
use crate::fetch::{BatchFetcher, BatchFetcherConfig};
use crate::internal::compact::Compactor;
use crate::internal::encoding::{Schemas, parse_id};
use crate::fetch::{BatchFetcher, BatchFetcherConfig, FetchBatchFilter, Lease};
use crate::internal::compact::{CompactConfig, Compactor};
use crate::internal::encoding::parse_id;
use crate::internal::gc::GarbageCollector;
use crate::internal::machine::{Machine, retry_external};
use crate::internal::state::RunOrder;
use crate::internal::state_versions::StateVersions;
use crate::iter::{Consolidator, StructuredSort};
use crate::metrics::Metrics;
use crate::read::{LeasedReaderId, READER_LEASE_DURATION, ReadHandle};
use crate::read::{
Cursor, CursorConsolidator, LazyPartStats, LeasedReaderId, READER_LEASE_DURATION, ReadHandle,
Since,
};
use crate::rpc::PubSubSender;
use crate::schema::CaESchema;
use crate::write::{WriteHandle, WriterId};
Expand Down Expand Up @@ -121,6 +133,9 @@ pub const BUILD_INFO: BuildInfo = build_info!();
// Re-export for convenience.
pub use mz_persist_types::{PersistLocation, ShardId};

pub use crate::internal::encoding::Schemas;
pub use crate::internal::state::HollowBatch;
Copy link
Contributor Author

Choose a reason for hiding this comment

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

You might object to these being exposed. We only need HollowBatch because we use it to sniff out the size/num_rows from stashed batches without turning them into a Batch and/or reading them.


/// Additional diagnostic information used within Persist
/// e.g. for logging, metric labels, etc.
#[derive(Clone, Debug)]
Expand Down Expand Up @@ -539,6 +554,188 @@ impl PersistClient {
Ok(writer)
}

/// Returns a [BatchBuilder] that can be used to write a batch of updates to
/// blob storage which can then be appended to the given shard using
/// [WriteHandle::compare_and_append_batch] or [WriteHandle::append_batch],
/// or which can be read using [PersistClient::read_batches_consolidated].
///
/// The builder uses a bounded amount of memory, even when the number of
/// updates is very large. Individual records, however, should be small
/// enough that we can reasonably chunk them up: O(KB) is definitely fine,
/// O(MB) come talk to us.
#[instrument(level = "debug", fields(shard = %shard_id))]
pub async fn batch_builder<K, V, T, D>(
Copy link
Contributor Author

Choose a reason for hiding this comment

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

These are basically duplicated code. I could add a WriteHandle::batch_builder_inner that doesn't take a self but all the arguments it needs. And then both WriteHandle and the code here could call that one.

Same for read_batches_consolidated below.

I didn't do this for now because I felt that the method signature was almost the largest part of these methods. But happy to change that!

&self,
shard_id: ShardId,
write_schemas: Schemas<K, V>,
lower: Antichain<T>,
) -> Result<BatchBuilder<K, V, T, D>, InvalidUsage<T>>
where
K: Debug + Codec,
V: Debug + Codec,
T: Timestamp + Lattice + Codec64 + Sync,
D: Semigroup + Ord + Codec64 + Send + Sync,
{
let cfg = CompactConfig::new(&self.cfg, shard_id);
// WIP: Pass this in as an argument?
let shard_metrics = self.metrics.shards.shard(&shard_id, "peek_stash");

let parts = if let Some(max_runs) = cfg.batch.max_runs {
BatchParts::new_compacting::<K, V, D>(
cfg,
Description::new(
lower.clone(),
Antichain::new(),
Antichain::from_elem(T::minimum()),
),
max_runs,
Arc::clone(&self.metrics),
shard_metrics,
shard_id,
Arc::clone(&self.blob),
Arc::clone(&self.isolated_runtime),
&self.metrics.user,
write_schemas.clone(),
)
} else {
BatchParts::new_ordered(
cfg.batch,
RunOrder::Unordered,
Arc::clone(&self.metrics),
shard_metrics,
shard_id,
Arc::clone(&self.blob),
Arc::clone(&self.isolated_runtime),
&self.metrics.user,
)
};
let builder = BatchBuilderInternal::new(
BatchBuilderConfig::new(&self.cfg, shard_id),
parts,
Arc::clone(&self.metrics),
write_schemas.clone(),
Arc::clone(&self.blob),
shard_id,
self.cfg.build_version.clone(),
);
Ok(BatchBuilder::new(
builder,
Description::new(lower, Antichain::new(), Antichain::from_elem(T::minimum())),
))
}

/// Turns the given [`ProtoBatch`] back into a [`Batch`] which can be used
/// to append it to the given shard or to read it via
/// [PersistClient::read_batches_consolidated]
pub fn batch_from_transmittable_batch<K, V, T, D>(
&self,
shard_id: &ShardId,
batch: ProtoBatch,
) -> Batch<K, V, T, D>
where
K: Debug + Codec,
V: Debug + Codec,
T: Timestamp + Lattice + Codec64 + Sync,
D: Semigroup + Ord + Codec64 + Send + Sync,
{
let batch_shard_id: ShardId = batch
.shard_id
.into_rust()
.expect("valid transmittable batch");
assert_eq!(&batch_shard_id, shard_id);

let shard_metrics = self.metrics.shards.shard(shard_id, "peek_stash");

let ret = Batch {
batch_delete_enabled: BATCH_DELETE_ENABLED.get(&self.cfg),
metrics: Arc::clone(&self.metrics),
shard_metrics,
version: Version::parse(&batch.version).expect("valid transmittable batch"),
batch: batch
.batch
.into_rust_if_some("ProtoBatch::batch")
.expect("valid transmittable batch"),
blob: Arc::clone(&self.blob),
_phantom: std::marker::PhantomData,
};

assert_eq!(&ret.shard_id(), shard_id);
ret
}

/// Returns a [Cursor] for reading the given batches. Yielded updates are
/// consolidated if the given batches contain sorted runs, which is true
/// when they have been written using a [BatchBuilder].
///
/// To keep memory usage down when reading a snapshot that consolidates
/// well, this consolidates as it goes. However, note that only the
/// serialized data is consolidated: the deserialized data will only be
/// consolidated if your K/V codecs are one-to-one.
// WIP: Do we want to let callers inject sth like MFP here?
// WIP: This doesn't need async right now, but still might want it in the
// API to have the option in the future?
#[allow(clippy::unused_async)]
pub async fn read_batches_consolidated<K, V, T, D>(
&mut self,
shard_id: ShardId,
as_of: Antichain<T>,
read_schemas: Schemas<K, V>,
batches: &[Batch<K, V, T, D>],
should_fetch_part: impl for<'a> Fn(Option<&'a LazyPartStats>) -> bool,
) -> Result<Cursor<K, V, T, D>, Since<T>>
where
K: Debug + Codec,
V: Debug + Codec,
T: Timestamp + Lattice + Codec64 + Sync,
D: Semigroup + Ord + Codec64 + Send + Sync,
{
let context = format!("{}[as_of={:?}]", shard_id, as_of.elements());
let filter = FetchBatchFilter::Snapshot {
as_of: as_of.clone(),
};

let shard_metrics = self.metrics.shards.shard(&shard_id, "peek_stash");

let consolidator = {
let mut consolidator = Consolidator::new(
context,
shard_id,
StructuredSort::new(read_schemas.clone()),
Arc::clone(&self.blob),
Arc::clone(&self.metrics),
Arc::clone(&shard_metrics),
self.metrics.read.snapshot.clone(),
filter,
COMPACTION_MEMORY_BOUND_BYTES.get(&self.cfg),
);
for batch in batches {
for (meta, run) in batch.batch.runs() {
consolidator.enqueue_run(
&batch.batch.desc,
meta,
run.into_iter()
.filter(|p| should_fetch_part(p.stats()))
.cloned(),
);
}
}
CursorConsolidator::Structured {
consolidator,
// This default may end up consolidating more records than previously
// for cases like fast-path peeks, where only the first few entries are used.
// If this is a noticeable performance impact, thread the max-len in from the caller.
max_len: self.cfg.compaction_yield_after_n_updates,
max_bytes: BLOB_TARGET_SIZE.get(&self.cfg).max(1),
}
};

Ok(Cursor {
consolidator,
_lease: Lease::default(),
read_schemas,
})
}

/// Returns the requested schema, if known at the current state.
pub async fn get_schema<K, V, T, D>(
&self,
Expand Down
8 changes: 4 additions & 4 deletions src/persist-client/src/read.rs
Original file line number Diff line number Diff line change
Expand Up @@ -865,13 +865,13 @@ pub(crate) struct UnexpiredReadHandleState {
/// but it's also free to abandon the instance at any time if it eg. only needs a few entries.
#[derive(Debug)]
pub struct Cursor<K: Codec, V: Codec, T: Timestamp + Codec64, D: Codec64> {
consolidator: CursorConsolidator<K, V, T, D>,
_lease: Lease,
read_schemas: Schemas<K, V>,
pub(crate) consolidator: CursorConsolidator<K, V, T, D>,
pub(crate) _lease: Lease,
pub(crate) read_schemas: Schemas<K, V>,
}

#[derive(Debug)]
enum CursorConsolidator<K: Codec, V: Codec, T: Timestamp + Codec64, D: Codec64> {
pub(crate) enum CursorConsolidator<K: Codec, V: Codec, T: Timestamp + Codec64, D: Codec64> {
Structured {
consolidator: Consolidator<T, D, StructuredSort<K, V, T, D>>,
max_len: usize,
Expand Down