-
Notifications
You must be signed in to change notification settings - Fork 473
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||
---|---|---|---|---|
|
@@ -21,27 +21,33 @@ use std::sync::Arc; | |||
|
||||
use differential_dataflow::difference::Semigroup; | ||||
use differential_dataflow::lattice::Lattice; | ||||
use itertools::Itertools; | ||||
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, Batch, BatchBuilder, ProtoBatch}; | ||||
use crate::cache::{PersistClientCache, StateCache}; | ||||
use crate::cfg::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::internal::encoding::parse_id; | ||||
use crate::internal::gc::GarbageCollector; | ||||
use crate::internal::machine::{Machine, retry_external}; | ||||
use crate::internal::state_versions::StateVersions; | ||||
use crate::metrics::Metrics; | ||||
use crate::read::{LeasedReaderId, READER_LEASE_DURATION, ReadHandle}; | ||||
use crate::read::{ | ||||
Cursor, LazyPartStats, LeasedReaderId, READER_LEASE_DURATION, ReadHandle, Since, | ||||
}; | ||||
use crate::rpc::PubSubSender; | ||||
use crate::schema::CaESchema; | ||||
use crate::write::{WriteHandle, WriterId}; | ||||
|
@@ -121,6 +127,8 @@ pub const BUILD_INFO: BuildInfo = build_info!(); | |||
// Re-export for convenience. | ||||
pub use mz_persist_types::{PersistLocation, ShardId}; | ||||
|
||||
pub use crate::internal::encoding::Schemas; | ||||
|
||||
/// Additional diagnostic information used within Persist | ||||
/// e.g. for logging, metric labels, etc. | ||||
#[derive(Clone, Debug)] | ||||
|
@@ -539,6 +547,133 @@ 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>( | ||||
&self, | ||||
shard_id: ShardId, | ||||
write_schemas: Schemas<K, V>, | ||||
lower: Antichain<T>, | ||||
) -> BatchBuilder<K, V, T, D> | ||||
where | ||||
K: Debug + Codec, | ||||
V: Debug + Codec, | ||||
T: Timestamp + Lattice + Codec64 + Sync, | ||||
D: Semigroup + Ord + Codec64 + Send + Sync, | ||||
{ | ||||
WriteHandle::builder_inner( | ||||
&self.cfg, | ||||
Arc::clone(&self.metrics), | ||||
self.metrics.shards.shard(&shard_id, "peek_stash"), | ||||
&self.metrics.user, | ||||
Arc::clone(&self.isolated_runtime), | ||||
Arc::clone(&self.blob), | ||||
shard_id, | ||||
write_schemas, | ||||
lower, | ||||
) | ||||
} | ||||
|
||||
/// 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] | ||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is somewhat risky API, since it would be pretty easy for us to decode the same Maybe that's necessary, but it feels like it at least warrants a warning in the name or docstring... There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This mirrors materialize/src/persist-client/src/write.rs Line 703 in abd4987
I'll add a warning in the docstring. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, that's fair! The danger of moving code... everything old is under scrutiny again. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Which is good, imo! 😅 |
||||
/// | ||||
/// CAUTION: This API allows turning a [ProtoBatch] into a [Batch] multiple | ||||
/// times, but if a batch is deleted the backing data goes away, so at that | ||||
/// point all in-memory copies of a batch become invalid and cannot be read | ||||
/// anymore. | ||||
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. | ||||
/// | ||||
/// CAUTION: The caller needs to make sure that the given batches are | ||||
/// readable and they have to remain readable for the lifetime of the | ||||
/// returned [Cursor]. The caller is also responsible for the lifecycle of | ||||
/// the batches: once the cursor and the batches are no longer needed you | ||||
/// must call [Cursor::into_lease] to get back the batches and delete them. | ||||
#[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: Vec<Batch<K, V, T, D>>, | ||||
should_fetch_part: impl for<'a> Fn(Option<&'a LazyPartStats>) -> bool, | ||||
) -> Result<Cursor<K, V, T, D, Vec<Batch<K, V, T, D>>>, Since<T>> | ||||
where | ||||
K: Debug + Codec + Ord, | ||||
V: Debug + Codec + Ord, | ||||
T: Timestamp + Lattice + Codec64 + Sync, | ||||
D: Semigroup + Ord + Codec64 + Send + Sync, | ||||
{ | ||||
let shard_metrics = self.metrics.shards.shard(&shard_id, "peek_stash"); | ||||
|
||||
let hollow_batches = batches.iter().map(|b| b.batch.clone()).collect_vec(); | ||||
|
||||
ReadHandle::read_batches_consolidated( | ||||
&self.cfg, | ||||
Arc::clone(&self.metrics), | ||||
shard_metrics, | ||||
self.metrics.read.snapshot.clone(), | ||||
Arc::clone(&self.blob), | ||||
shard_id, | ||||
as_of, | ||||
read_schemas, | ||||
&hollow_batches, | ||||
batches, | ||||
should_fetch_part, | ||||
) | ||||
} | ||||
|
||||
/// Returns the requested schema, if known at the current state. | ||||
pub async fn get_schema<K, V, T, D>( | ||||
&self, | ||||
|
Uh oh!
There was an error while loading. Please reload this page.