From 330db0eeface4435661529ee486fd389c3f59e41 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Tue, 14 Nov 2023 05:58:20 +0000 Subject: [PATCH] all working --- Cargo.lock | 1 + crates/build/src/lib.rs | 28 +- crates/flowctl/Cargo.toml | 1 + crates/flowctl/src/preview/journal_reader.rs | 244 +++++++ crates/flowctl/src/preview/mod.rs | 589 ++++++++--------- crates/flowctl/src/raw/capture.rs | 52 +- crates/journal-client/src/read/uncommitted.rs | 9 +- .../src/read/uncommitted/raw.rs | 2 +- crates/models/src/raw_value.rs | 7 + crates/runtime/src/capture/protocol.rs | 8 +- crates/runtime/src/capture/serve.rs | 11 +- crates/runtime/src/derive/mod.rs | 7 +- crates/runtime/src/derive/protocol.rs | 119 ++-- crates/runtime/src/derive/serve.rs | 25 +- crates/runtime/src/derive/task.rs | 50 +- crates/runtime/src/harness/capture.rs | 374 ++++++----- crates/runtime/src/harness/derive.rs | 237 +++++++ crates/runtime/src/harness/fixture.rs | 3 +- crates/runtime/src/harness/materialize.rs | 265 ++++++++ crates/runtime/src/harness/mod.rs | 53 +- crates/runtime/src/lib.rs | 29 + crates/runtime/src/materialize/mod.rs | 16 +- crates/runtime/src/materialize/protocol.rs | 35 +- crates/runtime/src/materialize/serve.rs | 62 +- crates/runtime/src/materialize/task.rs | 4 +- crates/runtime/src/uuid.rs | 2 +- go/protocols/runtime/runtime.pb.go | 603 +++++++++++++----- 27 files changed, 1983 insertions(+), 853 deletions(-) create mode 100644 crates/flowctl/src/preview/journal_reader.rs create mode 100644 crates/runtime/src/harness/materialize.rs diff --git a/Cargo.lock b/Cargo.lock index 48753e94e5..a6325020a0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1567,6 +1567,7 @@ dependencies = [ "clap 3.2.24", "comfy-table", "connector-init", + "coroutines", "crossterm", "dirs", "doc", diff --git a/crates/build/src/lib.rs b/crates/build/src/lib.rs index 801a5164ff..b9550e372a 100644 --- a/crates/build/src/lib.rs +++ b/crates/build/src/lib.rs @@ -93,22 +93,19 @@ pub async fn load(source: &url::Url, file_root: &Path) -> tables::Sources { /// Perform validations and produce built specifications for `sources`. /// * If `generate_ops_collections` is set, then ops collections are added into `sources`. /// * If any of `noop_*` is true, then validations are skipped for connectors of that type. -pub async fn validate( +pub async fn validate( allow_local: bool, build_id: &str, connector_network: &str, control_plane: &dyn validation::ControlPlane, generate_ops_collections: bool, - log_handler: L, + log_handler: impl runtime::LogHandler, noop_captures: bool, noop_derivations: bool, noop_materializations: bool, project_root: &url::Url, mut sources: tables::Sources, -) -> (tables::Sources, tables::Validations) -where - L: Fn(&ops::Log) + Send + Sync + Clone + 'static, -{ +) -> (tables::Sources, tables::Validations) { // TODO(johnny): We *really* need to kill this, and have ops collections // be injected exclusively from the control-plane. if generate_ops_collections { @@ -168,19 +165,16 @@ where /// This function is used to produce builds by managed control-plane /// components but not the `flowctl` CLI, which requires finer-grain /// control over build behavior. -pub async fn managed_build( +pub async fn managed_build( allow_local: bool, build_id: String, connector_network: String, control_plane: Box, file_root: PathBuf, - log_handler: L, + log_handler: impl runtime::LogHandler, project_root: url::Url, source: url::Url, -) -> Result<(tables::Sources, tables::Validations), tables::Errors> -where - L: Fn(&ops::Log) + Send + Sync + Clone + 'static, -{ +) -> Result<(tables::Sources, tables::Validations), tables::Errors> { let (sources, validations) = validate( allow_local, &build_id, @@ -363,20 +357,14 @@ impl sources::Fetcher for Fetcher { /// Connectors is a general-purpose implementation of validation::Connectors /// that dispatches to its contained runtime::Runtime. -pub struct Connectors -where - L: Fn(&ops::Log) + Send + Sync + Clone + 'static, -{ +pub struct Connectors { noop_captures: bool, noop_derivations: bool, noop_materializations: bool, runtime: runtime::Runtime, } -impl validation::Connectors for Connectors -where - L: Fn(&ops::Log) + Send + Sync + Clone + 'static, -{ +impl validation::Connectors for Connectors { fn validate_capture<'a>( &'a self, request: capture::Request, diff --git a/crates/flowctl/Cargo.toml b/crates/flowctl/Cargo.toml index e017612874..91cb50dc0d 100644 --- a/crates/flowctl/Cargo.toml +++ b/crates/flowctl/Cargo.toml @@ -15,6 +15,7 @@ assemble = { path = "../assemble" } async-process = { path = "../async-process" } build = { path = "../build" } connector-init = { path = "../connector-init" } +coroutines = { path = "../coroutines" } doc = { path = "../doc" } extractors = { path = "../extractors" } journal-client = { path = "../journal-client" } diff --git a/crates/flowctl/src/preview/journal_reader.rs b/crates/flowctl/src/preview/journal_reader.rs new file mode 100644 index 0000000000..cf077b3e27 --- /dev/null +++ b/crates/flowctl/src/preview/journal_reader.rs @@ -0,0 +1,244 @@ +use anyhow::Context; +use futures::{StreamExt, TryFutureExt, TryStreamExt}; +use proto_flow::flow; +use proto_gazette::{broker, consumer}; + +#[derive(Clone)] +pub struct Reader { + pub control_plane: crate::controlplane::Client, + pub delay: std::time::Duration, +} + +impl runtime::harness::Reader for Reader { + type Stream = futures::stream::BoxStream<'static, anyhow::Result>; + + fn start_for_derivation( + self, + derivation: &flow::CollectionSpec, + resume: proto_gazette::consumer::Checkpoint, + ) -> Self::Stream { + let transforms = &derivation.derivation.as_ref().unwrap().transforms; + + let sources = transforms + .iter() + .map(|t| { + let collection = t.collection.as_ref().unwrap(); + Source { + collection: collection.name.clone(), + partition_selector: t.partition_selector.clone().unwrap(), + not_before: t.not_before.clone(), + } + }) + .collect(); + + self.start(sources, resume).boxed() + } + + fn start_for_materialization( + self, + materialization: &flow::MaterializationSpec, + resume: proto_gazette::consumer::Checkpoint, + ) -> Self::Stream { + let sources = materialization + .bindings + .iter() + .map(|b| { + let collection = b.collection.as_ref().unwrap(); + Source { + collection: collection.name.clone(), + partition_selector: b.partition_selector.clone().unwrap(), + not_before: b.not_before.clone(), + } + }) + .collect(); + + self.start(sources, resume).boxed() + } +} + +struct Source { + collection: String, + partition_selector: broker::LabelSelector, + not_before: Option, +} + +impl Reader { + fn start( + self, + sources: Vec, + mut resume: proto_gazette::consumer::Checkpoint, + ) -> impl futures::Stream> { + coroutines::try_coroutine(move |mut co| async move { + // We must be able to access all of its sourced collections. + let access_prefixes = sources + .iter() + .map(|source| source.collection.clone()) + .collect(); + + let data_plane_client = + crate::dataplane::journal_client_for(self.control_plane, access_prefixes).await?; + + // Concurrently list the journals of every Source. + let journals: Vec<(&Source, Vec)> = + futures::future::try_join_all(sources.iter().map(|source| { + Self::list_journals(source, data_plane_client.clone()) + .map_ok(move |l| (source, l)) + })) + .await?; + + // Flatten into (binding, source, journal). + let journals: Vec<(u32, &Source, broker::JournalSpec)> = journals + .into_iter() + .enumerate() + .flat_map(|(binding, (source, journals))| { + journals + .into_iter() + .map(move |journal| (binding as u32, source, journal)) + }) + .collect(); + + // Map into a stream that yields lines from across all journals, as they're ready. + let mut journals = + futures::stream::select_all(journals.iter().map(|(binding, source, journal)| { + Self::read_journal_lines( + *binding, + data_plane_client.clone(), + journal, + &resume, + source, + ) + .boxed() + })); + + // Reset-able timer for delivery of delayed checkpoints. + let deadline = tokio::time::sleep(std::time::Duration::MAX); + tokio::pin!(deadline); + + let mut in_txn = false; // Have we emitted a document that awaits a checkpoint? + + loop { + let step = tokio::select! { + Some(read) = journals.next() => Ok(read?), + () = deadline.as_mut(), if in_txn => Err(()) + }; + + match step { + Ok((binding, doc_json, journal, read_through)) => { + let resume = match resume.sources.get_mut(&journal.name) { + Some(entry) => entry, + None => resume.sources.entry(journal.name.clone()).or_default(), + }; + resume.read_through = read_through; + + () = co + .yield_(runtime::harness::Read::Document { + binding: binding as u32, + doc: doc_json, + }) + .await; + + // If this is the first Read of this transaction, + // schedule when it will Checkpoint. + if !in_txn { + in_txn = true; + deadline + .as_mut() + .reset(tokio::time::Instant::now() + self.delay); + } + } + Err(()) => { + () = co + .yield_(runtime::harness::Read::Checkpoint(resume.clone())) + .await; + in_txn = false; + } + } + } + }) + .boxed() + } + + async fn list_journals( + source: &Source, + mut client: journal_client::Client, + ) -> anyhow::Result> { + let listing = journal_client::list::list_journals(&mut client, &source.partition_selector) + .await + .with_context(|| { + format!( + "failed to list journals for collection {}", + &source.collection + ) + })?; + + if listing.is_empty() { + anyhow::bail!( + "no journals were returned by the selector: {}", + serde_json::to_string_pretty(&source.partition_selector).unwrap() + ); + } + + Ok(listing) + } + + fn read_journal_lines<'s>( + binding: u32, + client: journal_client::Client, + journal: &'s broker::JournalSpec, + resume: &consumer::Checkpoint, + source: &Source, + ) -> impl futures::Stream> + { + use futures::AsyncBufReadExt; + use journal_client::read::uncommitted::{ + ExponentialBackoff, JournalRead, ReadStart, ReadUntil, Reader, + }; + + let offset = resume + .sources + .get(&journal.name) + .map(|s| s.read_through) + .unwrap_or_default(); + + let read = JournalRead::new(journal.name.clone()) + .starting_at(ReadStart::Offset(offset as u64)) + .begin_mod_time( + source + .not_before + .as_ref() + .map(|b| b.seconds) + .unwrap_or_default(), + ) + .read_until(ReadUntil::Forever); + + coroutines::try_coroutine(move |mut co| async move { + let backoff = ExponentialBackoff::new(2); + let reader = Reader::start_read(client, read, backoff); + let mut reader = futures::io::BufReader::new(reader); + + // Fill the buffer and establish the first read byte offset. + let buf_len = reader.fill_buf().await?.len(); + let mut offset = reader.get_ref().current_offset() - buf_len as i64; + + let mut lines = reader.lines(); + + loop { + let Some(doc_json) = lines.try_next().await? else { + break; + }; + // Attempt to keep the offset up to date. + // TODO(johnny): This is subtly broken because it doesn't handle offset jumps. + // Fixing requires a deeper refactor of journal_client::Reader. + offset += doc_json.len() as i64 + 1; + + // TODO(johnny): This is pretty janky. + if doc_json.starts_with("{\"_meta\":{\"ack\":true,") { + continue; + } + + () = co.yield_((binding, doc_json, journal, offset)).await; + } + Ok(()) + }) + } +} diff --git a/crates/flowctl/src/preview/mod.rs b/crates/flowctl/src/preview/mod.rs index 2987e2c484..3216d3903c 100644 --- a/crates/flowctl/src/preview/mod.rs +++ b/crates/flowctl/src/preview/mod.rs @@ -1,12 +1,9 @@ -use crate::{dataplane, local_specs}; +use crate::local_specs; use anyhow::Context; -use doc::shape::schema::to_schema; -use futures::channel::mpsc; -use futures::{SinkExt, StreamExt, TryStreamExt}; -use proto_flow::runtime::derive_request_ext; -use proto_flow::{derive, flow, flow::collection_spec::derivation::Transform}; -use proto_gazette::broker; -use tokio::sync::broadcast; +use futures::TryStreamExt; +use proto_flow::{capture, derive, flow, materialize}; + +mod journal_reader; #[derive(Debug, clap::Args)] #[clap(rename_all = "kebab-case")] @@ -14,337 +11,274 @@ pub struct Preview { /// Path or URL to a Flow specification file. #[clap(long)] source: String, - /// Name of the derived collection to preview within the Flow specification file. - /// Collection is required if there are multiple derivations in --source specifications. + /// Name of the task to preview within the Flow specification file. + /// Required if there are multiple tasks in --source specifications. + #[clap(long)] + name: Option, + /// Optional, artificial delay between transactions to simulate back-pressure + /// and encourage reductions. The default is no delay. + #[clap(long)] + delay: Option, + /// How long can the task produce no data before this command stops? + /// The default is that there is no timeout. #[clap(long)] - collection: Option, - /// When exiting (for example, upon Ctrl-D), should we update the derivation schema - /// based on observed output documents? + timeout: Option, + /// How many connector sessions should be run, and what is the target number + /// of transactions for each session? A session is stopped and the next started + /// upon reaching the target number of transactions, or if the connector exits. + /// The default is a single session with an unbounded number of transactions. + /// For a given session, a value less than zero means "unlimited transactions", + /// though the session will still end upon a connector EOF (when a capture) + /// or due to a timeout. + #[clap(long, value_parser, value_delimiter = ',')] + sessions: Option>, + /// Path to a transactions fixture to use, instead of reading live collections. + /// Fixtures are used only for derivations and materializations. #[clap(long)] - infer_schema: bool, + fixture: Option, + /* /// When previewing a SQLite derivation, the path URI of the database to use. /// This can be useful for debugging the internal state of a database. /// If not specified, an in-memory-only database is used. #[clap(long, default_value = ":memory:")] sqlite_uri: String, - /// How frequently should we close transactions and emit combined documents? - /// If not specified, the default is one second. - #[clap(long)] - interval: Option, + */ + /// Docker network to run connector images. + #[clap(long, default_value = "bridge")] + network: String, } impl Preview { pub async fn run(&self, ctx: &mut crate::CliContext) -> anyhow::Result<()> { let Self { source, - collection, - infer_schema, - sqlite_uri: sqlite_path, - interval: flush_interval, + name, + delay, + timeout, + sessions, + fixture, + network, } = self; - let source = build::arg_source_to_url(source, false)?; - - if self.infer_schema && source.scheme() != "file" { - anyhow::bail!("schema inference can only be used with a local file --source"); - } + let source = build::arg_source_to_url(source, false)?; let client = ctx.controlplane_client().await?; - let (sources, validations) = - local_specs::load_and_validate(client, source.as_str()).await?; - - // Identify the derivation to preview. - let needle = if let Some(needle) = collection { - needle.as_str() - } else if sources.collections.len() == 1 { - sources.collections.first().unwrap().collection.as_str() - } else if sources.collections.is_empty() { - anyhow::bail!("sourced specification files do not contain any derivations"); - } else { - anyhow::bail!("sourced specification files contain multiple derivations. Use --collection to identify a specific one"); - }; - // Resolve the built collection and its contained derivation. - let built_collection = match validations - .built_collections - .binary_search_by_key(&needle, |b| b.collection.as_str()) - { - Ok(index) => &validations.built_collections[index], - Err(_) => anyhow::bail!("could not find the collection {needle}"), - }; - let derivation = built_collection - .spec - .derivation - .as_ref() - .context("collection is not a derivation")?; - - // We must be able to access all of its sourced collections. - let access_prefixes = derivation - .transforms - .iter() - .map(|Transform { collection, .. }| collection.as_ref().unwrap().name.clone()) - .collect(); - - let data_plane_client = - dataplane::journal_client_for(ctx.controlplane_client().await?, access_prefixes) - .await?; - - // Perform a listing of all collection journals to read. - let listings = futures::future::try_join_all(derivation.transforms.iter().enumerate().map( - |( - index, - Transform { - name, - partition_selector, - .. - }, - )| { - let mut data_plane_client = data_plane_client.clone(); - async move { - let listing = journal_client::list::list_journals( - &mut data_plane_client, - partition_selector.as_ref().unwrap(), - ) - .await - .with_context(|| format!("failed to list journal for transform {name}"))?; - - if listing.is_empty() { - anyhow::bail!( - "no journals were returned by the selector: {}", - serde_json::to_string_pretty(partition_selector).unwrap() - ); - } - Result::<_, anyhow::Error>::Ok((index, listing)) - } - }, - )) - .await?; - - // Start derivation connector. - let (mut request_tx, request_rx) = mpsc::channel(64); - - // Remove `uuid_ptr` so that UUID placeholders aren't included in preview output. - let mut spec = built_collection.spec.clone(); - spec.uuid_ptr = String::new(); - - request_tx - .send(Ok(derive::Request { - open: Some(derive::request::Open { - collection: Some(spec), - range: Some(flow::RangeSpec { - key_begin: 0, - key_end: u32::MAX, - r_clock_begin: 0, - r_clock_end: u32::MAX, - }), - version: "local".to_string(), - state_json: "{}".to_string(), - }), - ..Default::default() - } - .with_internal(|internal| { - internal.open = Some(derive_request_ext::Open { - sqlite_vfs_uri: sqlite_path.clone(), - ..Default::default() - }); - }))) - .await?; - - let responses_rx = runtime::Runtime::new( - true, // Allow local connectors. - String::new(), + // TODO(johnny): validate only `name`, if presented. + let (_sources, validations) = + local_specs::load_and_validate_full(client, source.as_str(), &network).await?; + + let runtime = runtime::Runtime::new( + true, // Allow local. + network.clone(), ops::tracing_log_handler, None, "preview".to_string(), - ) - .serve_derive(request_rx); - tokio::pin!(responses_rx); - - let _opened = responses_rx - .next() - .await - .context("expected Opened, not EOF")?? - .opened - .context("expected Opened")?; - - let (cancel_tx, cancel_rx) = broadcast::channel(1); - - // Start reads of all journals. - let reads = listings - .into_iter() - .flat_map(|(transform, journals)| { - journals - .into_iter() - .map(move |journal| (transform, journal)) - }) - .map(|(transform, journal)| { - read_journal( - cancel_rx.resubscribe(), - journal, - transform.clone(), - request_tx.clone(), - data_plane_client.clone(), - ) - }) - .collect::>() - .try_collect(); - - // Future that sends a periodic Flush request. - let flushes = tick_flushes(cancel_rx.resubscribe(), request_tx, flush_interval.as_ref()); - - // Future that emits previewed documents and gathers inference. - let output = output(responses_rx, *infer_schema); + ); + + // Default to no delay. + let delay = delay + .map(|i| i.clone().into()) + .unwrap_or(std::time::Duration::ZERO); + + // Default to no timeout. + let timeout = timeout + .map(|i| i.clone().into()) + .unwrap_or(std::time::Duration::MAX); + + // Negative sessions mean "unlimited transactions", and default to a single unlimited session. + let sessions = if let Some(sessions) = sessions { + sessions + .iter() + .map(|i| usize::try_from(*i).unwrap_or(usize::MAX)) + .collect() + } else { + vec![usize::MAX] + }; - let cancel = async move { - let _cancel_tx = cancel_tx; // Owned and dropped by this future. + // Parse a provided data fixture. + let fixture_reader = if let Some(fixture) = fixture { + let fixture = std::fs::read(fixture).context("couldn't open fixture file")?; + let fixture: runtime::harness::fixture::Fixture = + serde_json::from_slice(&fixture).context("couldn't parse fixture")?; - // Blocking read until stdin is closed. - tokio::io::copy(&mut tokio::io::stdin(), &mut tokio::io::sink()).await?; - Ok::<(), anyhow::Error>(()) + Some(runtime::harness::fixture::Reader(fixture)) + } else { + None + }; + let journal_reader = journal_reader::Reader { + control_plane: ctx.controlplane_client().await?, + delay, }; - let ((), (), (), schema) = futures::try_join!(cancel, reads, flushes, output)?; - - // Update with an inferred schema and write out the updated Flow spec. - if let Some(schema) = schema { - // Reload `sources`, this time without inlining them. - let mut sources = - local_specs::surface_errors(local_specs::load(&source).await.into_result()) - .expect("sources must load a second time"); - - // Find the derivation we just previewed. - let index = sources - .collections - .binary_search_by_key(&needle, |b| b.collection.as_str()) - .unwrap(); - - // Update (just) its schema, making no other changes to loaded `sources`. - // We don't attempt to inline it. - let collection = &mut sources.collections[index].spec; - collection.read_schema = None; - collection.write_schema = None; - collection.schema = Some(models::Schema::new(models::RawValue::from_value(&schema))); - - _ = local_specs::write_resources(sources)?; + let state = models::RawValue::default(); + let state_dir = tempfile::tempdir().unwrap(); + + let num_tasks = validations.built_captures.len() + + validations.built_materializations.len() + + validations + .built_collections + .iter() + .filter(|c| c.spec.derivation.is_some()) + .count(); + + if num_tasks == 0 { + anyhow::bail!("sourced specification files do not contain any tasks (captures, derivations, or materializations)"); + } else if num_tasks > 1 && name.is_none() { + anyhow::bail!("sourced specification files contain multiple tasks (captures, derivations, or materializations). Use --name to identify a specific task"); } - tracing::info!("all done"); - Ok(()) - } -} + for capture in validations.built_captures.iter() { + if !matches!(name, Some(n) if n == &capture.capture) && name.is_some() { + continue; + } + let mut spec = capture.spec.clone(); -async fn read_journal( - mut cancel_rx: broadcast::Receiver<()>, - journal: broker::JournalSpec, - transform: usize, - mut request_tx: mpsc::Sender>, - client: journal_client::Client, -) -> anyhow::Result<()> { - use futures::AsyncBufReadExt; - use journal_client::read::uncommitted::{ - ExponentialBackoff, JournalRead, ReadStart, ReadUntil, Reader, - }; - - tracing::info!(journal = %journal.name, "starting read of journal"); - let read = JournalRead::new(journal.name) - .starting_at(ReadStart::Offset(0)) - .read_until(ReadUntil::Forever); - - let backoff = ExponentialBackoff::new(2); - let reader = Reader::start_read(client, read, backoff); - - // TODO(johnny): Reader should directly implement futures::io::AsyncBufRead. - let mut lines = futures::io::BufReader::new(reader).lines(); - - loop { - let doc_json = tokio::select! { - doc_json = lines.try_next() => match doc_json? { - Some(doc_json) => doc_json, - None => { - return Ok(()) // All done. - } - }, - _ = cancel_rx.recv() => { - return Ok(()) // Cancelled. + // Disable UUID placeholders. + for binding in spec.bindings.iter_mut() { + binding.collection.as_mut().unwrap().uuid_ptr = String::new(); } - }; - // TODO(johnny): This is pretty janky. - if doc_json.starts_with("{\"_meta\":{\"ack\":true,") { - continue; + return preview_capture( + delay, + runtime, + sessions, + spec, + state, + state_dir.path(), + timeout, + ) + .await; } - request_tx - .send(Ok(derive::Request { - read: Some(derive::request::Read { - transform: transform as u32, - doc_json, - // TODO: attach real `shuffle` and `uuid` messages. - ..Default::default() - }), - ..Default::default() - })) - .await?; + for collection in validations.built_collections.iter() { + if !matches!(name, Some(n) if n == collection.collection.as_str()) && name.is_some() { + continue; + } else if collection.spec.derivation.is_none() && name.is_some() { + anyhow::bail!("{} is not a derivation", name.as_ref().unwrap()); + } else if collection.spec.derivation.is_none() { + continue; + } + let mut spec = collection.spec.clone(); + + // Disable UUID placeholders. + spec.uuid_ptr = String::new(); + + if let Some(reader) = fixture_reader { + return preview_derivation( + reader, + runtime, + sessions, + spec, + state, + state_dir.path(), + timeout, + ) + .await; + } else { + return preview_derivation( + journal_reader, + runtime, + sessions, + spec, + state, + state_dir.path(), + timeout, + ) + .await; + } + } + for materialization in validations.built_materializations.iter() { + if !matches!(name, Some(n) if n == materialization.materialization.as_str()) + && name.is_some() + { + continue; + } + let spec = materialization.spec.clone(); + + if let Some(reader) = fixture_reader { + return preview_materialization( + reader, + runtime, + sessions, + spec, + state, + state_dir.path(), + timeout, + ) + .await; + } else { + return preview_materialization( + journal_reader, + runtime, + sessions, + spec, + state, + state_dir.path(), + timeout, + ) + .await; + } + } + anyhow::bail!("could not find task {}", name.as_ref().unwrap()); } } -async fn tick_flushes( - mut cancel_rx: broadcast::Receiver<()>, - mut request_tx: mpsc::Sender>, - flush_interval: Option<&humantime::Duration>, +async fn preview_capture( + delay: std::time::Duration, + runtime: runtime::Runtime, + sessions: Vec, + spec: flow::CaptureSpec, + state: models::RawValue, + state_dir: &std::path::Path, + timeout: std::time::Duration, ) -> anyhow::Result<()> { - let period = flush_interval - .map(|i| i.clone().into()) - .unwrap_or(std::time::Duration::from_secs(1)); - - let mut ticker = tokio::time::interval(period); - loop { - tokio::select! { - _ = ticker.tick() => (), // Fall through. - _ = cancel_rx.recv() => { - return Ok(()) // Cancelled. - } - } + let responses_rx = + runtime::harness::run_capture(delay, runtime, sessions, &spec, state, state_dir, timeout); + tokio::pin!(responses_rx); - if let Err(_) = request_tx - .send_all(&mut futures::stream::iter([ - Ok(Ok(derive::Request { - flush: Some(derive::request::Flush {}), - ..Default::default() - })), - Ok(Ok(derive::Request { - start_commit: Some(derive::request::StartCommit { - runtime_checkpoint: Some(proto_gazette::consumer::Checkpoint { - sources: [( - "a/journal".to_string(), - proto_gazette::consumer::checkpoint::Source { - read_through: 1, - ..Default::default() - }, - )] - .into(), - ..Default::default() - }), - ..Default::default() - }), - ..Default::default() - })), - ])) - .await - { - // We can error only because there's no receiver, - // but that's not an error from this routine's perspective. - return Ok(()); + while let Some(response) = responses_rx.try_next().await? { + let internal = response + .get_internal() + .context("failed to decode internal runtime.CaptureResponseExt")?; + + if let Some(capture::response::Applied { action_description }) = response.applied { + tracing::info!(action_description, "capture was applied"); + } else if let Some(capture::response::Captured { binding, doc_json }) = response.captured { + let proto_flow::runtime::capture_response_ext::Captured { + key_packed, + partitions_packed, + } = internal.captured.unwrap_or_default(); + + tracing::trace!(?key_packed, ?partitions_packed, "captured"); + + let collection = &spec.bindings[binding as usize] + .collection + .as_ref() + .unwrap() + .name; + + print!("[{collection:?},{doc_json}]\n"); + } else if let Some(capture::response::Checkpoint { state }) = response.checkpoint { + let proto_flow::runtime::capture_response_ext::Checkpoint { stats, .. } = + internal.checkpoint.unwrap_or_default(); + tracing::debug!(stats=?ops::DebugJson(stats), state=?ops::DebugJson(state), "checkpoint"); } } + + Ok(()) } -async fn output( - mut responses_rx: R, - infer_schema: bool, -) -> anyhow::Result> -where - R: futures::Stream> + Unpin, -{ - let mut inferred_shape = doc::Shape::nothing(); +async fn preview_derivation( + reader: impl runtime::harness::Reader, + runtime: runtime::Runtime, + sessions: Vec, + spec: flow::CollectionSpec, + state: models::RawValue, + state_dir: &std::path::Path, + timeout: std::time::Duration, +) -> anyhow::Result<()> { + let responses_rx = + runtime::harness::run_derive(reader, runtime, sessions, &spec, state, state_dir, timeout); + tokio::pin!(responses_rx); while let Some(response) = responses_rx.try_next().await? { let internal = response @@ -358,30 +292,51 @@ where partitions_packed, } = internal.published.unwrap_or_default(); - tracing::debug!(?max_clock, ?key_packed, ?partitions_packed, "published"); - - if infer_schema { - let doc: serde_json::Value = - serde_json::from_str(&doc_json).context("failed to parse derived document")?; - - inferred_shape.widen(&doc); - } + tracing::trace!(?max_clock, ?key_packed, ?partitions_packed, "published"); print!("{doc_json}\n"); } else if let Some(derive::response::Flushed {}) = response.flushed { let proto_flow::runtime::derive_response_ext::Flushed { stats } = internal.flushed.unwrap_or_default(); - let stats = serde_json::to_string(&stats).unwrap(); - - tracing::debug!(%stats, "flushed"); + tracing::debug!(stats=?ops::DebugJson(stats), "flushed"); } else if let Some(derive::response::StartedCommit { state }) = response.started_commit { - tracing::debug!(?state, "started commit"); + tracing::debug!(state=?ops::DebugJson(state), "started commit"); + } + } + + Ok(()) +} + +async fn preview_materialization( + reader: impl runtime::harness::Reader, + runtime: runtime::Runtime, + sessions: Vec, + spec: flow::MaterializationSpec, + state: models::RawValue, + state_dir: &std::path::Path, + timeout: std::time::Duration, +) -> anyhow::Result<()> { + let responses_rx = runtime::harness::run_materialize( + reader, runtime, sessions, &spec, state, state_dir, timeout, + ); + tokio::pin!(responses_rx); + + while let Some(response) = responses_rx.try_next().await? { + let internal = response + .get_internal() + .context("failed to decode internal runtime.MaterializeResponseExt")?; + + if let Some(materialize::response::Applied { action_description }) = response.applied { + tracing::info!(action_description, "materialization was applied"); + } else if let Some(materialize::response::Flushed {}) = response.flushed { + let proto_flow::runtime::materialize_response_ext::Flushed { stats } = + internal.flushed.unwrap_or_default(); + tracing::debug!(stats=?ops::DebugJson(stats), "flushed"); + } else if let Some(materialize::response::StartedCommit { state }) = response.started_commit + { + tracing::debug!(state=?ops::DebugJson(state), "started commit"); } } - Ok(if infer_schema { - Some(serde_json::to_value(to_schema(inferred_shape)).unwrap()) - } else { - None - }) + Ok(()) } diff --git a/crates/flowctl/src/raw/capture.rs b/crates/flowctl/src/raw/capture.rs index 65c18a2501..94592c96c6 100644 --- a/crates/flowctl/src/raw/capture.rs +++ b/crates/flowctl/src/raw/capture.rs @@ -14,19 +14,20 @@ pub struct Capture { /// Capture is required if there are multiple captures in --source specifications. #[clap(long)] capture: Option, - /// How long can the capture produce no data before this command stops? - /// The default is to not time out. + /// How long can the task produce no data before this command stops? + /// The default is that there is no timeout. #[clap(long)] timeout: Option, - /// How many times may the capture connector be restarted before this command stops? - /// The default is to not restart the connector if it exits. + /// How many connector sessions should be run, and what is the target number + /// of transactions for each session? A session is stopped and the next started + /// upon reaching the target number of transactions, or if the connector exits. + /// The default is a single session with an unbounded number of transactions. + /// For a given session, the value zero means "unlimited transactions", + /// though the session will still end upon a connector EOF (when a capture) + /// or due to a timeout. #[clap(long)] - max_restarts: Option, - /// How many transactions should capture produce before this command stops? - /// The default is an unlimited number. - #[clap(long)] - max_transactions: Option, - /// Optional, added delay between transactions to simulate back-pressure + sessions: Option>, + /// Optional, artificial delay between transactions to simulate back-pressure /// and encourage reductions. The default is no delay. #[clap(long)] delay: Option, @@ -39,12 +40,11 @@ pub async fn do_capture( ctx: &mut crate::CliContext, Capture { capture, - timeout, - max_restarts, - max_transactions, - network, delay, + network, + sessions, source, + timeout, }: &Capture, ) -> anyhow::Result<()> { let client = ctx.controlplane_client().await?; @@ -62,11 +62,11 @@ pub async fn do_capture( anyhow::bail!("sourced specification files contain multiple captures. Use --capture to identify a specific one"); }; - let (capture, built_capture) = match sources + let built_capture = match sources .captures .binary_search_by_key(&needle, |c| c.capture.as_str()) { - Ok(index) => (&sources.captures[index], &validations.built_captures[index]), + Ok(index) => &validations.built_captures[index], Err(_) => anyhow::bail!("could not find the capture {needle}"), }; @@ -75,27 +75,27 @@ pub async fn do_capture( network.clone(), ops::tracing_log_handler, None, - format!("preview/{}", capture.capture), + format!("preview/{}", built_capture.capture), ); - let idle_timeout = timeout - .map(|i| i.clone().into()) - .unwrap_or(std::time::Duration::MAX); - let poll_delay = delay + let delay = delay .map(|i| i.clone().into()) .unwrap_or(std::time::Duration::ZERO); + let timeout = timeout + .map(|i| i.clone().into()) + .unwrap_or(std::time::Duration::MAX); + let sessions = sessions.clone().unwrap_or(vec![0]); let state_dir = tempfile::tempdir().unwrap(); let response_rx = runtime::harness::run_capture( - idle_timeout, - max_restarts.unwrap_or(usize::MAX), - max_transactions.unwrap_or(usize::MAX), - poll_delay, + delay, runtime, - built_capture.spec.clone(), + sessions, + &built_capture.spec, models::RawValue::default(), state_dir.path(), + timeout, ); tokio::pin!(response_rx); diff --git a/crates/journal-client/src/read/uncommitted.rs b/crates/journal-client/src/read/uncommitted.rs index 7bf92ffb25..6439309bbb 100644 --- a/crates/journal-client/src/read/uncommitted.rs +++ b/crates/journal-client/src/read/uncommitted.rs @@ -83,6 +83,7 @@ pub struct JournalRead { end_offset: i64, fetch_fragments: bool, block: bool, + begin_mod_time: i64, } impl JournalRead { @@ -93,6 +94,7 @@ impl JournalRead { end_offset: 0, fetch_fragments: true, block: false, + begin_mod_time: 0, } } @@ -101,6 +103,11 @@ impl JournalRead { self } + pub fn begin_mod_time(mut self, seconds: i64) -> Self { + self.begin_mod_time = seconds; + self + } + pub fn read_until(mut self, until: ReadUntil) -> Self { self.end_offset = until.to_broker_offset(); self.block = until.is_blocking(); @@ -121,7 +128,7 @@ impl JournalRead { block: self.block, do_not_proxy: false, metadata_only: self.fetch_fragments && !needs_direct_read, - begin_mod_time: 0, + begin_mod_time: self.begin_mod_time, } } } diff --git a/crates/journal-client/src/read/uncommitted/raw.rs b/crates/journal-client/src/read/uncommitted/raw.rs index 05e249b014..40e699224d 100644 --- a/crates/journal-client/src/read/uncommitted/raw.rs +++ b/crates/journal-client/src/read/uncommitted/raw.rs @@ -12,7 +12,7 @@ use tonic::codec::Streaming; pub async fn start_read(client: &mut Client, req: broker::ReadRequest) -> Result { let offset = req.offset; let journal = req.journal.clone(); - tracing::debug!(%journal, %offset, "starting new read of journal"); + tracing::debug!(?req, "starting new read of journal"); let response = client.read(req).await?; // TODO: see if there's anything in the response we should check or log before proceeding to read Ok(Reader::new(journal, offset, response.into_inner())) diff --git a/crates/models/src/raw_value.rs b/crates/models/src/raw_value.rs index 1acc1862ed..e173dbdfb0 100644 --- a/crates/models/src/raw_value.rs +++ b/crates/models/src/raw_value.rs @@ -37,6 +37,13 @@ impl From for Box { } } +impl From for String { + fn from(value: RawValue) -> Self { + let s: Box = value.0.into(); + s.into() + } +} + impl std::ops::Deref for RawValue { type Target = serde_json::value::RawValue; fn deref(&self) -> &Self::Target { diff --git a/crates/runtime/src/capture/protocol.rs b/crates/runtime/src/capture/protocol.rs index 39643ef39e..f2b9fc3846 100644 --- a/crates/runtime/src/capture/protocol.rs +++ b/crates/runtime/src/capture/protocol.rs @@ -261,7 +261,7 @@ pub fn recv_client_start_commit( // Add the runtime checkpoint to our WriteBatch. tracing::debug!( - ?runtime_checkpoint, + checkpoint=?ops::DebugJson(&runtime_checkpoint), "persisting StartCommit.runtime_checkpoint", ); batch.put(RocksDB::CHECKPOINT_KEY, &runtime_checkpoint.encode_to_vec()); @@ -318,8 +318,10 @@ pub fn recv_connector_captured( .with_context(|| "invalid captured binding {binding}")? .document_uuid_ptr; - if let Some(node) = uuid_ptr.create_heap_node(&mut doc, alloc) { - *node = doc::HeapNode::String(doc::BumpStr::from_str(crate::UUID_PLACEHOLDER, alloc)); + if !uuid_ptr.0.is_empty() { + if let Some(node) = uuid_ptr.create_heap_node(&mut doc, alloc) { + *node = doc::HeapNode::String(doc::BumpStr::from_str(crate::UUID_PLACEHOLDER, alloc)); + } } memtable.add(binding, doc, false)?; diff --git a/crates/runtime/src/capture/serve.rs b/crates/runtime/src/capture/serve.rs index 330c8991e4..6b9153ab75 100644 --- a/crates/runtime/src/capture/serve.rs +++ b/crates/runtime/src/capture/serve.rs @@ -1,7 +1,7 @@ use super::{connector, protocol::*, RequestStream, ResponseStream, Task, Transaction}; use crate::{task_state::RocksDB, verify, LogHandler, Runtime}; use anyhow::Context; -use futures::channel::{mpsc, oneshot}; +use futures::channel::oneshot; use futures::future::FusedFuture; use futures::stream::FusedStream; use futures::{FutureExt, SinkExt, StreamExt, TryStreamExt}; @@ -96,8 +96,8 @@ async fn serve_session( super::LONG_POLL_TIMEOUT, yield_rx, )); - let mut last_checkpoints: u32 = 0; // Checkpoints in the last transaction. + let mut last_checkpoints: u32 = 0; // Checkpoints in the last transaction. let mut buf = bytes::BytesMut::new(); loop { // Receive initial request of a transaction: Acknowledge, Open, or EOF. @@ -132,7 +132,12 @@ async fn serve_session( // Acknowledge committed checkpoints to the connector. if let Some(ack) = send_connector_acknowledge(last_checkpoints, &task) { - let _: Result<(), mpsc::SendError> = connector_tx.feed(ack).await; // Best-effort. + tokio::select! { + _ = tokio::time::sleep(std::time::Duration::from_secs(10)) => anyhow::bail!( + "connector requested acknowledgements but is not processing its input", + ), + _ = connector_tx.feed(ack) => (), // We don't error on disconnection, only timeout. + } } // Signal that we're ready for a transaction to yield, and then wait for it. diff --git a/crates/runtime/src/derive/mod.rs b/crates/runtime/src/derive/mod.rs index a26de62282..e70ffa83a5 100644 --- a/crates/runtime/src/derive/mod.rs +++ b/crates/runtime/src/derive/mod.rs @@ -31,12 +31,15 @@ pub struct Task { shard_ref: ops::ShardRef, // Transforms of this task. transforms: Vec, + // Write JSON-Schema of the derivation collection. + write_schema_json: String, } #[derive(Debug)] struct Transform { - collection_name: String, // Source collection. - name: String, // Name of this Transform. + collection_name: String, // Source collection. + name: String, // Name of this Transform. + read_schema_json: String, // Read JSON-Schema of the derivation source collection. } #[derive(Debug)] diff --git a/crates/runtime/src/derive/protocol.rs b/crates/runtime/src/derive/protocol.rs index 6600f08694..209fab1b0e 100644 --- a/crates/runtime/src/derive/protocol.rs +++ b/crates/runtime/src/derive/protocol.rs @@ -45,6 +45,7 @@ pub fn recv_connector_opened( opened: Option, ) -> anyhow::Result<( Task, + Vec, doc::combine::Accumulator, consumer::Checkpoint, Response, @@ -54,6 +55,7 @@ pub fn recv_connector_opened( }; let task = Task::new(&open, &opened)?; + let validators = task.validators()?; let accumulator = doc::combine::Accumulator::new(task.combine_spec()?, tempfile::tempfile()?)?; let mut checkpoint = db @@ -68,7 +70,7 @@ pub fn recv_connector_opened( { checkpoint = connector_checkpoint.clone(); tracing::debug!( - ?checkpoint, + checkpoint=?ops::DebugJson(&checkpoint), "using connector-provided OpenedExt.runtime_checkpoint", ); } else { @@ -76,73 +78,98 @@ pub fn recv_connector_opened( runtime_checkpoint: Some(checkpoint.clone()), }); tracing::debug!( - ?checkpoint, + checkpoint=?ops::DebugJson(&checkpoint), "loaded and attached a persisted OpenedExt.runtime_checkpoint", ); } }); - Ok((task, accumulator, checkpoint, opened)) + Ok((task, validators, accumulator, checkpoint, opened)) } pub fn recv_client_read_or_flush( request: Option, + saw_flush: &mut bool, + task: &Task, txn: &mut Transaction, -) -> anyhow::Result> { - match request { + validators: &mut Vec, +) -> anyhow::Result> { + let read = match request { Some(Request { read: Some(read), .. - }) => { - if let Some(flow::UuidParts { clock, node }) = &read.uuid { - // Filter out message acknowledgements. - if proto_gazette::message_flags::ACK_TXN & node != 0 { - return Ok(None); - } - // Track the largest document clock that we've observed. - if *clock > txn.max_clock { - txn.max_clock = *clock; - } - } - - // Accumulate metrics over reads for our transforms. - let read_stats = &mut txn.read_stats.entry(read.transform).or_default(); - read_stats.docs_total += 1; - read_stats.bytes_total += read.doc_json.len() as u64; - - Ok(Some(( - Request { - read: Some(read), - ..Default::default() - }, - false, - ))) - } + }) => read, Some(Request { - flush: Some(_flush), + flush: Some(request::Flush {}), .. - }) => Ok(Some(( - Request { + }) => { + *saw_flush = true; + + return Ok(Some(Request { flush: Some(request::Flush {}), ..Default::default() - }, - true, - ))), - request => verify("client", "Read or Flush").fail(request), + })); + } + request => return verify("client", "Read or Flush").fail(request), + }; + + let transform = &task.transforms[read.transform as usize]; + + // TODO(johnny): This is transitional, and only run in non-production `flowctl` contexts. + if read.shuffle.is_none() { + () = || -> anyhow::Result<()> { + // TODO: use OwnedArchived or parse into HeapNode. + let doc: serde_json::Value = serde_json::from_str(&read.doc_json)?; + let _valid = validators[read.transform as usize] + .validate(None, &doc)? + .ok()?; + Ok(()) + }() + .with_context(|| { + format!( + "read transform {} collection {} document is invalid", + &transform.name, &transform.collection_name, + ) + })?; + } + + if let Some(flow::UuidParts { clock, node }) = &read.uuid { + // Filter out message acknowledgements. + if proto_gazette::message_flags::ACK_TXN & node != 0 { + return Ok(None); + } + // Track the largest document clock that we've observed. + if *clock > txn.max_clock { + txn.max_clock = *clock; + } } + + // Accumulate metrics over reads for our transforms. + let read_stats = &mut txn.read_stats.entry(read.transform).or_default(); + read_stats.docs_total += 1; + read_stats.bytes_total += read.doc_json.len() as u64; + + Ok(Some(Request { + read: Some(read), + ..Default::default() + })) } pub fn recv_connector_published_or_flushed( accumulator: &mut doc::combine::Accumulator, response: Option, saw_flush: bool, + saw_flushed: &mut bool, task: &Task, txn: &mut Transaction, -) -> anyhow::Result { +) -> anyhow::Result<()> { let response::Published { doc_json } = match response { Some(Response { flushed: Some(response::Flushed {}), .. - }) if saw_flush => return Ok(true), + }) if saw_flush => { + *saw_flushed = true; + return Ok(()); + } Some(Response { flushed: Some(_), .. }) => { @@ -164,15 +191,17 @@ pub fn recv_connector_published_or_flushed( let uuid_ptr = &task.document_uuid_ptr; - if let Some(node) = uuid_ptr.create_heap_node(&mut doc, alloc) { - *node = doc::HeapNode::String(doc::BumpStr::from_str(crate::UUID_PLACEHOLDER, alloc)); + if !uuid_ptr.0.is_empty() { + if let Some(node) = uuid_ptr.create_heap_node(&mut doc, alloc) { + *node = doc::HeapNode::String(doc::BumpStr::from_str(crate::UUID_PLACEHOLDER, alloc)); + } } memtable.add(0, doc, false)?; txn.publish_stats.docs_total += 1; txn.publish_stats.bytes_total += doc_json.len() as u64; - Ok(false) + Ok(()) } pub fn send_client_published( @@ -190,8 +219,8 @@ pub fn send_client_published( let doc_json = serde_json::to_string(&task.ser_policy.on_owned(&root)) .expect("document serialization cannot fail"); - txn.publish_stats.docs_total += 1; - txn.publish_stats.bytes_total += doc_json.len() as u64; + txn.combined_stats.docs_total += 1; + txn.combined_stats.bytes_total += doc_json.len() as u64; if shape.widen_owned(&root) { doc::shape::limits::enforce_shape_complexity_limit( @@ -281,7 +310,7 @@ pub fn recv_client_start_commit( let mut wb = rocksdb::WriteBatch::default(); tracing::debug!( - ?runtime_checkpoint, + checkpoint=?ops::DebugJson(&runtime_checkpoint), "persisting StartCommit.runtime_checkpoint", ); wb.put(RocksDB::CHECKPOINT_KEY, runtime_checkpoint.encode_to_vec()); diff --git a/crates/runtime/src/derive/serve.rs b/crates/runtime/src/derive/serve.rs index dc4606101a..c8d8d931ee 100644 --- a/crates/runtime/src/derive/serve.rs +++ b/crates/runtime/src/derive/serve.rs @@ -80,12 +80,12 @@ async fn serve_session( let (mut connector_tx, mut connector_rx) = connector::start(runtime, open.clone()).await?; let opened = TryStreamExt::try_next(&mut connector_rx).await?; - let (task, mut accumulator, mut last_checkpoint, opened) = + let (task, mut validators, mut accumulator, mut last_checkpoint, opened) = recv_connector_opened(&db, &open, opened)?; () = co.yield_(opened).await; - // Loop over transactions. + let mut buf = bytes::BytesMut::new(); loop { // Loop over EOF, Reset, and Open until an initial Read or Flush. let initial: Request = loop { @@ -95,7 +95,7 @@ async fn serve_session( return Ok(None); } Some(reset @ Request { reset: Some(_), .. }) => { - let _: Result<(), mpsc::SendError> = connector_tx.feed(reset).await; + connector_tx.try_send(reset).expect("sender is empty"); } Some(open @ Request { open: Some(_), .. }) => { drain_connector(connector_tx, connector_rx).await?; @@ -124,19 +124,25 @@ async fn serve_session( let mut step = Step::ClientRx(Some(initial)); // Loop over client requests and connector responses until the transaction has flushed. - while !saw_flush || !saw_flushed { + loop { match step { Step::ClientRx(request) => { - if let Some((send, is_flush)) = recv_client_read_or_flush(request, &mut txn)? { - saw_flush = is_flush; + if let Some(send) = recv_client_read_or_flush( + request, + &mut saw_flush, + &task, + &mut txn, + &mut validators, + )? { send_fut = Some(connector_tx.feed(send)); } } Step::ConnectorRx(response) => { - saw_flushed = recv_connector_published_or_flushed( + recv_connector_published_or_flushed( &mut accumulator, response, saw_flush, + &mut saw_flushed, &task, &mut txn, )?; @@ -149,6 +155,10 @@ async fn serve_session( } } + if saw_flush && saw_flushed { + break; + } + step = if let Some(forward) = &mut send_fut { tokio::select! { result = forward => Step::ConnectorTx(result), @@ -172,7 +182,6 @@ async fn serve_session( let mut drainer = accumulator .into_drainer() .context("preparing to drain combiner")?; - let mut buf = bytes::BytesMut::new(); while let Some(drained) = drainer.drain_next()? { let published = send_client_published(&mut buf, drained, shape, &task, &mut txn); diff --git a/crates/runtime/src/derive/task.rs b/crates/runtime/src/derive/task.rs index 6fd4d48064..b2f3abc5ac 100644 --- a/crates/runtime/src/derive/task.rs +++ b/crates/runtime/src/derive/task.rs @@ -24,7 +24,7 @@ impl Task { projections, read_schema_json: _, uuid_ptr, - write_schema_json: _, + write_schema_json, } = collection.context("missing collection")?; let flow::collection_spec::Derivation { @@ -37,9 +37,7 @@ impl Task { transforms, } = derivation.as_ref().context("missing derivation")?; - if uuid_ptr.is_empty() { - anyhow::bail!("uuid_ptr cannot be empty"); - } else if key.is_empty() { + if key.is_empty() { anyhow::bail!("collection key cannot be empty"); } @@ -72,11 +70,33 @@ impl Task { ser_policy, shard_ref, transforms, + write_schema_json, }) } pub fn combine_spec(&self) -> anyhow::Result { - todo!() + let built_schema = doc::validation::build_bundle(&self.write_schema_json) + .context("collection write_schema_json is not a JSON schema")?; + let validator = doc::Validator::from_schema(built_schema) + .context("could not build a schema validator")?; + + Ok(doc::combine::Spec::with_one_binding( + false, + self.key_extractors.clone(), + None, + validator, + )) + } + + pub fn validators(&self) -> anyhow::Result> { + self.transforms + .iter() + .map(|transform| { + transform.validator().with_context(|| { + format!("failed to build validator for transform {}", transform.name) + }) + }) + .collect::, _>>() } } @@ -105,14 +125,30 @@ impl Transform { partition_fields: _, partition_template: _, projections: _, - read_schema_json: _, + read_schema_json, uuid_ptr: _, - write_schema_json: _, + write_schema_json, } = collection.as_ref().context("missing collection")?; + let read_schema_json = if read_schema_json.is_empty() { + write_schema_json + } else { + read_schema_json + } + .clone(); + Ok(Self { collection_name: collection_name.clone(), name: name.clone(), + read_schema_json, }) } + + pub fn validator(&self) -> anyhow::Result { + let built_schema = doc::validation::build_bundle(&self.read_schema_json) + .context("collection read_schema_json is not a JSON schema")?; + let validator = doc::Validator::from_schema(built_schema) + .context("could not build a schema validator")?; + Ok(validator) + } } diff --git a/crates/runtime/src/harness/capture.rs b/crates/runtime/src/harness/capture.rs index a1ea3ee0d4..8e090a73dc 100644 --- a/crates/runtime/src/harness/capture.rs +++ b/crates/runtime/src/harness/capture.rs @@ -1,7 +1,7 @@ use crate::capture::ResponseStream; use crate::{task_state::RocksDB, verify, LogHandler, Runtime}; use anyhow::Context; -use futures::{channel::mpsc, SinkExt, TryStreamExt}; +use futures::{channel::mpsc, TryStreamExt}; use proto_flow::capture::{request, response, Request, Response}; use proto_flow::flow; use proto_flow::runtime::{ @@ -10,179 +10,229 @@ use proto_flow::runtime::{ CaptureResponseExt, }; use proto_gazette::consumer; +use std::pin::Pin; pub fn run_capture( - // Period of time for which the capture can produce no data before it exits. - timeout: std::time::Duration, - // Maximum number of restarts of the underlying connector (inclusive). - max_restarts: usize, - // Maximum number of transactions before the capture exits (inclusive). - max_transactions: usize, - // Optional delay between transactions to simulate back-pressure and encourage reductions, delay: std::time::Duration, runtime: Runtime, - spec: flow::CaptureSpec, + sessions: Vec, + spec: &flow::CaptureSpec, state: models::RawValue, state_dir: &std::path::Path, + timeout: std::time::Duration, ) -> impl ResponseStream { + let spec = spec.clone(); let state_dir = state_dir.to_owned(); + let mut state: String = state.into(); + + // TODO(johnny): extract from spec? + let version = super::unique_version(); coroutines::try_coroutine(move |mut co| async move { - let (mut tx, rx) = mpsc::channel(crate::CHANNEL_BUFFER); - let rx = runtime.serve_capture(rx); - tokio::pin!(rx); + let (mut request_tx, request_rx) = mpsc::channel(crate::CHANNEL_BUFFER); + let response_rx = runtime.serve_capture(request_rx); + tokio::pin!(response_rx); - let state_dir = state_dir.to_str().context("tempdir is not utf8")?; + // Send Apply. + let apply = Request { + apply: Some(request::Apply { + capture: Some(spec.clone()), + dry_run: false, + version: version.clone(), + }), + ..Default::default() + }; + request_tx.try_send(Ok(apply)).expect("sender is empty"); + + // Receive Applied. + match response_rx.try_next().await? { + Some(applied) if applied.applied.is_some() => { + () = co.yield_(applied).await; + } + response => return verify("runtime", "Applied").fail(response), + } + let state_dir = state_dir.to_str().context("tempdir is not utf8")?; let rocksdb_desc = Some(runtime::RocksDbDescriptor { rocksdb_env_memptr: 0, rocksdb_path: state_dir.to_owned(), }); + let open_ext = capture_request_ext::Open { + rocksdb_descriptor: rocksdb_desc.clone(), + }; - // TODO(johnny): extract from spec? - let version = "test"; + let sessions_len = sessions.len(); + for (index, target_transactions) in sessions.into_iter().enumerate() { + () = run_session( + &mut co, + delay, + index == sessions_len - 1, + &open_ext, + &mut request_tx, + &mut response_rx, + &spec, + &mut state, + target_transactions, + timeout, + &version, + ) + .await?; + } - tx.feed(Ok(Request { - apply: Some(request::Apply { - capture: Some(spec.clone()), - dry_run: false, - version: version.to_string(), + std::mem::drop(request_tx); + verify("runtime", "EOF").is_eof(response_rx.try_next().await?)?; + + // Re-open RocksDB. + let rocksdb = RocksDB::open(rocksdb_desc)?; + + tracing::debug!( + checkpoint = ?::ops::DebugJson(rocksdb.load_checkpoint()?), + "final runtime checkpoint", + ); + + // Extract and yield the final connector state + let state = rocksdb.load_connector_state()?; + () = co + .yield_(Response { + checkpoint: Some(response::Checkpoint { + state: state.map(|updated_json| flow::ConnectorState { + updated_json, + merge_patch: false, + }), + }), + ..Default::default() + }) + .await; + + anyhow::Result::Ok(()) + }) +} + +async fn run_session( + co: &mut coroutines::Suspend, + delay: std::time::Duration, + last_session: bool, + open_ext: &capture_request_ext::Open, + request_tx: &mut mpsc::Sender>, + response_rx: &mut Pin<&mut impl ResponseStream>, + spec: &flow::CaptureSpec, + state: &mut String, + target_transactions: usize, + timeout: std::time::Duration, + version: &str, +) -> anyhow::Result<()> { + // Send Open. + let open = Request { + open: Some(request::Open { + capture: Some(spec.clone()), + version: version.to_string(), + range: Some(flow::RangeSpec { + key_begin: 0, + key_end: u32::MAX, + r_clock_begin: 0, + r_clock_end: u32::MAX, }), - ..Default::default() - })) - .await?; + state_json: std::mem::take(state), + }), + ..Default::default() + } + .with_internal(|internal| { + internal.open = Some(open_ext.clone()); + }); + request_tx.try_send(Ok(open)).expect("sender is empty"); - // Wait for the Capture to be applied. - match rx.try_next().await? { - Some(applied) if applied.applied.is_some() => co.yield_(applied).await, - response => return verify("runtime", "Applied").fail(response), + // Receive Opened. + match response_rx.try_next().await? { + Some(opened) if opened.opened.is_some() => { + () = co.yield_(opened).await; } + response => return verify("runtime", "Opened").fail(response), + } - let mut idle_deadline = tokio::time::sleep(timeout); - let mut restarts = 0; - let mut send_open = true; - let mut transactions = 0; - - while transactions != max_transactions { - if send_open { - tx.feed(Ok(Request { - open: Some(request::Open { - capture: Some(spec.clone()), - version: version.to_string(), - range: Some(flow::RangeSpec { - key_begin: 0, - key_end: u32::MAX, - r_clock_begin: 0, - r_clock_end: u32::MAX, - }), - state_json: state.get().to_string(), - }), - ..Default::default() - } - .with_internal(|internal| { - internal.open = Some(capture_request_ext::Open { - rocksdb_descriptor: rocksdb_desc.clone(), - }); - }))) - .await?; - - match rx.try_next().await? { - Some(opened) if opened.opened.is_some() => { - if restarts == 0 { - () = co.yield_(opened).await; - } - } - response => return verify("runtime", "Opened").fail(response), - } - - idle_deadline = tokio::time::sleep(timeout); - send_open = false; - } + // Reset-able timer for assessing `timeout` between transactions. + let mut deadline = tokio::time::sleep(timeout); + let mut transaction = 0; - // Future which sleeps for `delay` and then sends a poll request. - let send_poll = async { - if !delay.is_zero() { - () = tokio::time::sleep(delay).await; - } - tx.feed(Ok(Request { + while transaction != target_transactions { + // Future which sleeps for `delay` and then sends a poll request. + let send_poll = async { + if !delay.is_zero() { + () = tokio::time::sleep(delay).await; + } + request_tx + .try_send(Ok(Request { acknowledge: Some(request::Acknowledge { checkpoints: 0 }), ..Default::default() })) - .await?; + .expect("sender is empty"); - Ok(()) - }; + Ok(()) + }; - // Join over sending a poll request and reading its result. - let ((), poll_response) = futures::try_join!(send_poll, rx.try_next())?; - - let ready = { - let verify = verify("runtime", "Poll Result"); - let poll_response = verify.not_eof(poll_response)?; - let CaptureResponseExt { - checkpoint: - Some(capture_response_ext::Checkpoint { - stats: None, - poll_result, - }), - .. - } = poll_response.get_internal()? - else { - return verify.fail(poll_response); - }; - - let poll_result = - PollResult::from_i32(poll_result).context("invalid PollResult")?; - tracing::debug!(?poll_result, "polled capture"); - - match poll_result { - PollResult::Invalid => return verify.fail(poll_response), - PollResult::Ready => true, - PollResult::CoolOff | PollResult::Restart if restarts == max_restarts => break, - PollResult::NotReady | PollResult::CoolOff => false, - PollResult::Restart => { - restarts += 1; - send_open = true; - false - } - } + // Join over sending a poll request and reading its result. + let ((), poll_response) = futures::try_join!(send_poll, response_rx.try_next())?; + + let ready = { + let verify = verify("runtime", "Poll Result"); + let poll_response = verify.not_eof(poll_response)?; + let CaptureResponseExt { + checkpoint: + Some(capture_response_ext::Checkpoint { + stats: None, + poll_result, + }), + .. + } = poll_response.get_internal()? + else { + return verify.fail(poll_response); }; - if !ready { - if idle_deadline.is_elapsed() { - break; - } else { - continue; - } - } - transactions += 1; - - let mut done = false; - while !done { - let verify = verify("runtime", "Captured or Checkpoint"); - let response = verify.not_eof(rx.try_next().await?)?; - - done = match &response { - Response { - captured: Some(_), .. - } => false, - Response { - checkpoint: Some(response::Checkpoint { state }), - .. - } => state.is_none(), - _ => return verify.fail(response), - }; - () = co.yield_(response).await; + let poll_result = PollResult::from_i32(poll_result).context("invalid PollResult")?; + tracing::debug!(?poll_result, "polled capture"); + + match poll_result { + PollResult::Invalid => return verify.fail(poll_response), + PollResult::Ready => true, + PollResult::CoolOff if last_session => break, + PollResult::CoolOff | PollResult::NotReady => false, + PollResult::Restart => break, } + }; - // Send a StartCommit with a synthetic checkpoint that reflects our current poll. - tx.feed(Ok(Request::default().with_internal(|internal| { + if !ready && !timeout.is_zero() && deadline.is_elapsed() { + break; + } else if !ready { + continue; // Poll again. + } + + // Receive Captured and Checkpoint. + let mut done = false; + while !done { + let verify = verify("runtime", "Captured or Checkpoint"); + let response = verify.not_eof(response_rx.try_next().await?)?; + + done = match &response { + Response { + captured: Some(_), .. + } => false, + Response { + checkpoint: Some(response::Checkpoint { state }), + .. + } => state.is_none(), // Final Checkpoint (only) has no `state`. + _ => return verify.fail(response), + }; + () = co.yield_(response).await; + } + + // Send a StartCommit with a synthetic checkpoint that reflects our current poll. + request_tx + .try_send(Ok(Request::default().with_internal(|internal| { internal.start_commit = Some(capture_request_ext::StartCommit { runtime_checkpoint: Some(consumer::Checkpoint { sources: [( format!("test/transactions"), consumer::checkpoint::Source { - read_through: transactions as i64, + read_through: 1 + transaction as i64, ..Default::default() }, )] @@ -191,45 +241,23 @@ pub fn run_capture( }), }); }))) - .await?; - - // Receive StartedCommit. - match rx.try_next().await? { - Some(Response { - checkpoint: Some(response::Checkpoint { state: None }), - .. - }) => (), - response => return verify("runtime", "StartedCommit").fail(response), - } + .expect("sender is empty"); - idle_deadline = tokio::time::sleep(timeout); + // Receive StartedCommit. + match response_rx.try_next().await? { + Some(Response { + checkpoint: Some(response::Checkpoint { state: None }), + .. + }) => (), + response => return verify("runtime", "StartedCommit").fail(response), } - std::mem::drop(tx); - verify("runtime", "EOF").is_eof(rx.try_next().await?)?; + transaction += 1; - // Re-open RocksDB. - let rocksdb = RocksDB::open(rocksdb_desc)?; - - tracing::debug!( - checkpoint = ?::ops::DebugJson(rocksdb.load_checkpoint()?), - "final runtime checkpoint", - ); - - // Extract and yield the final connector state - let state = rocksdb.load_connector_state()?; - () = co - .yield_(Response { - checkpoint: Some(response::Checkpoint { - state: state.map(|updated_json| flow::ConnectorState { - updated_json, - merge_patch: false, - }), - }), - ..Default::default() - }) - .await; + if timeout != std::time::Duration::MAX { + deadline = tokio::time::sleep(timeout); + } + } - anyhow::Result::Ok(()) - }) + Ok(()) } diff --git a/crates/runtime/src/harness/derive.rs b/crates/runtime/src/harness/derive.rs index e69de29bb2..ea04f04039 100644 --- a/crates/runtime/src/harness/derive.rs +++ b/crates/runtime/src/harness/derive.rs @@ -0,0 +1,237 @@ +use super::{Read, Reader}; +use crate::derive::ResponseStream; +use crate::{task_state::RocksDB, verify, LogHandler, Runtime}; +use anyhow::Context; +use futures::{channel::mpsc, TryStreamExt}; +use proto_flow::derive::{request, response, Request, Response}; +use proto_flow::flow; +use proto_flow::runtime::{self, derive_request_ext}; +use std::pin::Pin; + +pub fn run_derive( + reader: impl Reader, + runtime: Runtime, + sessions: Vec, + spec: &flow::CollectionSpec, + state: models::RawValue, + state_dir: &std::path::Path, + timeout: std::time::Duration, +) -> impl ResponseStream { + let spec = spec.clone(); + let state_dir = state_dir.to_owned(); + let mut state: String = state.into(); + + // TODO(johnny): extract from spec? + let version = super::unique_version(); + + coroutines::try_coroutine(move |mut co| async move { + let (mut request_tx, request_rx) = mpsc::channel(crate::CHANNEL_BUFFER); + let response_rx = runtime.serve_derive(request_rx); + tokio::pin!(response_rx); + + let state_dir = state_dir.to_str().context("tempdir is not utf8")?; + let rocksdb_desc = Some(runtime::RocksDbDescriptor { + rocksdb_env_memptr: 0, + rocksdb_path: state_dir.to_owned(), + }); + let open_ext = derive_request_ext::Open { + rocksdb_descriptor: rocksdb_desc.clone(), + sqlite_vfs_uri: format!("file://{state_dir}/sqlite.db"), + }; + + for target_transactions in sessions { + () = run_session( + &mut co, + &open_ext, + reader.clone(), + &mut request_tx, + &mut response_rx, + &spec, + &mut state, + target_transactions, + timeout, + &version, + ) + .await?; + } + + std::mem::drop(request_tx); + verify("runtime", "EOF").is_eof(response_rx.try_next().await?)?; + + // Re-open RocksDB. + let rocksdb = RocksDB::open(rocksdb_desc)?; + + tracing::debug!( + checkpoint = ?::ops::DebugJson(rocksdb.load_checkpoint()?), + "final runtime checkpoint", + ); + + // Extract and yield the final connector state + let state = rocksdb.load_connector_state()?; + () = co + .yield_(Response { + started_commit: Some(response::StartedCommit { + state: state.map(|updated_json| flow::ConnectorState { + updated_json, + merge_patch: false, + }), + }), + ..Default::default() + }) + .await; + + Ok(()) + }) +} + +async fn run_session( + co: &mut coroutines::Suspend, + open_ext: &derive_request_ext::Open, + reader: impl Reader, + request_tx: &mut mpsc::Sender>, + response_rx: &mut Pin<&mut impl ResponseStream>, + spec: &flow::CollectionSpec, + state: &mut String, + target_transactions: usize, + timeout: std::time::Duration, + version: &str, +) -> anyhow::Result<()> { + // Send Open. + let open = Request { + open: Some(request::Open { + collection: Some(spec.clone()), + version: version.to_string(), + range: Some(flow::RangeSpec { + key_begin: 0, + key_end: u32::MAX, + r_clock_begin: 0, + r_clock_end: u32::MAX, + }), + state_json: std::mem::take(state), + }), + ..Default::default() + } + .with_internal(|internal| { + internal.open = Some(open_ext.clone()); + }); + request_tx.try_send(Ok(open)).expect("sender is empty"); + + // Receive Opened. + let opened_ext = match response_rx.try_next().await? { + Some(opened) if opened.opened.is_some() && !opened.internal.is_empty() => { + let opened_ext = opened.get_internal()?; + () = co.yield_(opened).await; + opened_ext + } + response => return verify("runtime", "Opened").fail(response), + }; + + let checkpoint = opened_ext + .opened + .context("expected OpenedExt")? + .runtime_checkpoint + .unwrap_or_default(); + + let read_rx = reader.start_for_derivation(&spec, checkpoint); + tokio::pin!(read_rx); + + for _transaction in 0..target_transactions { + let deadline = tokio::time::sleep(timeout); + tokio::pin!(deadline); + + let mut started = false; + + // Read documents until a checkpoint. + let checkpoint = loop { + let read = tokio::select! { + read = read_rx.try_next() => read?, + () = deadline.as_mut(), if !started => { + tracing::info!(?timeout, "session ending upon reaching timeout"); + return Ok(()); + }, + }; + started = true; + + let (forward, checkpoint) = match read { + None => { + tracing::info!("session ending because reader returned EOF"); + return Ok(()); + } + // Forward a Read document to the runtime. + Some(Read::Document { binding, doc }) => ( + Request { + read: Some(request::Read { + doc_json: doc, + transform: binding, + ..Default::default() + }), + ..Default::default() + }, + None, + ), + // Forward a Flush to the runtime, then go on to commit a checkpoint. + Some(Read::Checkpoint(checkpoint)) => ( + Request { + flush: Some(request::Flush {}), + ..Default::default() + }, + Some(checkpoint), + ), + }; + + () = crate::exchange(Ok(forward), request_tx, response_rx) + .try_for_each( + |response| async move { verify("runtime", "no response").fail(response) }, + ) + .await?; + + if let Some(checkpoint) = checkpoint { + break checkpoint; + } + }; + + // Receive Published and then Flushed. + let mut done = false; + while !done { + let verify = verify("runtime", "Published or Flushed"); + let response = verify.not_eof(response_rx.try_next().await?)?; + + done = match &response { + Response { + published: Some(_), .. + } => false, + Response { + flushed: Some(response::Flushed {}), + .. + } => true, + _ => return verify.fail(response), + }; + () = co.yield_(response).await; + } + + // Send StartCommit. + request_tx + .try_send(Ok(Request { + start_commit: Some(request::StartCommit { + runtime_checkpoint: Some(checkpoint), + }), + ..Default::default() + })) + .expect("sender is empty"); + + // Receive StartedCommit. + match response_rx.try_next().await? { + Some( + started_commit @ Response { + started_commit: Some(_), + .. + }, + ) => { + () = co.yield_(started_commit).await; + } + response => return verify("runtime", "StartedCommit").fail(response), + } + } + + Ok(()) +} diff --git a/crates/runtime/src/harness/fixture.rs b/crates/runtime/src/harness/fixture.rs index c81595c7bf..3c468fafd5 100644 --- a/crates/runtime/src/harness/fixture.rs +++ b/crates/runtime/src/harness/fixture.rs @@ -8,6 +8,7 @@ use std::collections::HashMap; // is a Vec of ordered (collection-name, document) instances. pub type Fixture = Vec>; +#[derive(Clone)] pub struct Reader(pub Fixture); impl super::Reader for Reader { @@ -105,7 +106,7 @@ impl Reader { sources: [( "fixture".to_string(), consumer::checkpoint::Source { - read_through: txn as i64, + read_through: 1 + txn as i64, producers: Vec::new(), }, )] diff --git a/crates/runtime/src/harness/materialize.rs b/crates/runtime/src/harness/materialize.rs new file mode 100644 index 0000000000..3d509df52d --- /dev/null +++ b/crates/runtime/src/harness/materialize.rs @@ -0,0 +1,265 @@ +use super::{Read, Reader}; +use crate::materialize::ResponseStream; +use crate::{task_state::RocksDB, verify, LogHandler, Runtime}; +use anyhow::Context; +use futures::{channel::mpsc, TryStreamExt}; +use proto_flow::flow; +use proto_flow::materialize::{request, response, Request, Response}; +use proto_flow::runtime::{self, materialize_request_ext}; +use std::pin::Pin; + +pub fn run_materialize( + reader: impl Reader, + runtime: Runtime, + sessions: Vec, + spec: &flow::MaterializationSpec, + state: models::RawValue, + state_dir: &std::path::Path, + timeout: std::time::Duration, +) -> impl ResponseStream { + let spec = spec.clone(); + let state_dir = state_dir.to_owned(); + let mut state: String = state.into(); + + // TODO(johnny): extract from spec? + let version = super::unique_version(); + + coroutines::try_coroutine(move |mut co| async move { + let (mut request_tx, request_rx) = mpsc::channel(crate::CHANNEL_BUFFER); + let response_rx = runtime.serve_materialize(request_rx); + tokio::pin!(response_rx); + + // Send Apply. + let apply = Request { + apply: Some(request::Apply { + materialization: Some(spec.clone()), + dry_run: false, + version: version.clone(), + }), + ..Default::default() + }; + request_tx.try_send(Ok(apply)).expect("sender is empty"); + + // Receive Applied. + match response_rx.try_next().await? { + Some(applied) if applied.applied.is_some() => { + () = co.yield_(applied).await; + } + response => return verify("runtime", "Applied").fail(response), + } + + let state_dir = state_dir.to_str().context("tempdir is not utf8")?; + let rocksdb_desc = Some(runtime::RocksDbDescriptor { + rocksdb_env_memptr: 0, + rocksdb_path: state_dir.to_owned(), + }); + let open_ext = materialize_request_ext::Open { + rocksdb_descriptor: rocksdb_desc.clone(), + }; + + for target_transactions in sessions { + () = run_session( + &mut co, + &open_ext, + reader.clone(), + &mut request_tx, + &mut response_rx, + &spec, + &mut state, + target_transactions, + timeout, + &version, + ) + .await?; + } + + std::mem::drop(request_tx); + verify("runtime", "EOF").is_eof(response_rx.try_next().await?)?; + + // Re-open RocksDB. + let rocksdb = RocksDB::open(rocksdb_desc)?; + + tracing::debug!( + checkpoint = ?::ops::DebugJson(rocksdb.load_checkpoint()?), + "final runtime checkpoint", + ); + + // Extract and yield the final connector state + let state = rocksdb.load_connector_state()?; + () = co + .yield_(Response { + started_commit: Some(response::StartedCommit { + state: state.map(|updated_json| flow::ConnectorState { + updated_json, + merge_patch: false, + }), + }), + ..Default::default() + }) + .await; + + Ok(()) + }) +} + +async fn run_session( + co: &mut coroutines::Suspend, + open_ext: &materialize_request_ext::Open, + reader: impl Reader, + request_tx: &mut mpsc::Sender>, + response_rx: &mut Pin<&mut impl ResponseStream>, + spec: &flow::MaterializationSpec, + state: &mut String, + target_transactions: usize, + timeout: std::time::Duration, + version: &str, +) -> anyhow::Result<()> { + // Send Open. + let open = Request { + open: Some(request::Open { + materialization: Some(spec.clone()), + version: version.to_string(), + range: Some(flow::RangeSpec { + key_begin: 0, + key_end: u32::MAX, + r_clock_begin: 0, + r_clock_end: u32::MAX, + }), + state_json: std::mem::take(state), + }), + ..Default::default() + } + .with_internal(|internal| { + internal.open = Some(open_ext.clone()); + }); + request_tx.try_send(Ok(open)).expect("sender is empty"); + + // Receive Opened. + let verify_opened = verify("runtime", "Opened"); + let opened = verify_opened.not_eof(response_rx.try_next().await?)?; + let Response { + opened: Some(response::Opened { runtime_checkpoint }), + .. + } = &opened + else { + return verify_opened.fail(opened); + }; + + let checkpoint = runtime_checkpoint.clone().unwrap_or_default(); + () = co.yield_(opened).await; + + // Send initial Acknowledge of the session. + request_tx + .try_send(Ok(Request { + acknowledge: Some(request::Acknowledge {}), + ..Default::default() + })) + .expect("sender is empty"); + + let read_rx = reader.start_for_materialization(&spec, checkpoint); + tokio::pin!(read_rx); + + for _transaction in 0..target_transactions { + let deadline = tokio::time::sleep(timeout); + tokio::pin!(deadline); + + let mut started = false; + let mut saw_acknowledged = false; + + // Read documents until a checkpoint. + let checkpoint = loop { + let read = tokio::select! { + read = read_rx.try_next() => read?, + () = deadline.as_mut(), if !started => { + tracing::info!(?timeout, "session ending upon reaching timeout"); + return Ok(()); + }, + }; + started = true; + + match read { + None => { + tracing::info!("session ending because reader returned EOF"); + return Ok(()); + } + Some(Read::Checkpoint(checkpoint)) => break checkpoint, // Commit below. + Some(Read::Document { binding, doc }) => { + // Forward to the runtime as a Load document. + let request = Request { + load: Some(request::Load { + binding, + key_json: doc, + ..Default::default() + }), + ..Default::default() + }; + + () = crate::exchange(Ok(request), request_tx, response_rx) + .try_for_each(|response| { + futures::future::ready(if response.acknowledged.is_some() { + saw_acknowledged = true; + Ok(()) + } else { + verify("runtime", "Acknowledged").fail(response) + }) + }) + .await?; + + continue; + } + }; + }; + + // Receive Acknowledged, if we haven't already. + if !saw_acknowledged { + match response_rx.try_next().await? { + Some(response) if response.acknowledged.is_some() => (), + response => return verify("runtime", "Acknowledged").fail(response), + } + } + + // Send Flush. + request_tx + .try_send(Ok(Request { + flush: Some(request::Flush {}), + ..Default::default() + })) + .expect("sender is empty"); + + // Receive Flushed. + match response_rx.try_next().await? { + Some(response) if response.flushed.is_some() => { + () = co.yield_(response).await; + } + response => return verify("runtime", "Flushed").fail(response), + } + + // Send StartCommit. + request_tx + .try_send(Ok(Request { + start_commit: Some(request::StartCommit { + runtime_checkpoint: Some(checkpoint), + }), + ..Default::default() + })) + .expect("sender is empty"); + + // Receive StartedCommit. + match response_rx.try_next().await? { + Some(response) if response.started_commit.is_some() => { + () = co.yield_(response).await; + } + response => return verify("runtime", "StartedCommit").fail(response), + } + + // Send Acknowledge. + request_tx + .try_send(Ok(Request { + acknowledge: Some(request::Acknowledge {}), + ..Default::default() + })) + .expect("sender is empty"); + } + + Ok(()) +} diff --git a/crates/runtime/src/harness/mod.rs b/crates/runtime/src/harness/mod.rs index 8d9577202e..f8fc8ee593 100644 --- a/crates/runtime/src/harness/mod.rs +++ b/crates/runtime/src/harness/mod.rs @@ -1,51 +1,22 @@ use proto_flow::flow; use proto_gazette::consumer; -/* -Test Harnesses: - -Capture: -- Takes: CaptureSpec, Count, Interval, State. -- Returns: Stream of Response -- Starts the capture. Upon Opened, it begins a PollInterval ticker. -- It drives the protocol, and will send Acknowledge only on ticker fires. -- It exits when PollCount has elapsed. -- Re-open temp RocksDB to surface connector state. - -Reader Trait: -- Takes collections: [collection-name], checkpoint: Checkpoint -- Returns Stream of Either<(index, raw-document), Checkpoint> - -FixtureReader implementation -- [(collection, document)] or [(collection, [document])] ? - -Derivation: -- Takes: CollectionSpec, Reader, State, SQLiteURI -- Each Checkpoint becomes a Flush. -- Drives it to completion. -- Re-open temp RocksDB to surface connector state. - -Materialization: -- Takes: MaterializationSpec, Reader, State. -- Drives it to completion. -- Each Checkpoint becomes a Flush. -- Optional: perform a final load of every key that's surfaced for snapshot-ing. -- Re-open temp RocksDB to surface connector state. -*/ - mod capture; mod derive; -mod fixture; +pub mod fixture; +mod materialize; pub use capture::run_capture; +pub use derive::run_derive; +pub use materialize::run_materialize; pub enum Read { - Document { binding: u32, doc: bytes::Bytes }, + Document { binding: u32, doc: String }, Checkpoint(consumer::Checkpoint), } -trait Reader { - type Stream: futures::Stream>; +pub trait Reader: Clone + Send + Sync + 'static { + type Stream: futures::Stream> + Send + 'static; fn start_for_derivation( self, @@ -59,3 +30,13 @@ trait Reader { resume: consumer::Checkpoint, ) -> Self::Stream; } + +fn unique_version() -> String { + format!( + "harness-{}", + std::time::SystemTime::now() + .duration_since(std::time::UNIX_EPOCH) + .unwrap() + .as_secs() + ) +} diff --git a/crates/runtime/src/lib.rs b/crates/runtime/src/lib.rs index 501eb1e9a2..bf78256aa0 100644 --- a/crates/runtime/src/lib.rs +++ b/crates/runtime/src/lib.rs @@ -168,3 +168,32 @@ impl Verify { } } } + +fn exchange<'s, Request, Tx, Response, Rx>( + request: Request, + tx: &'s mut Tx, + rx: &'s mut Rx, +) -> impl futures::Stream + 's +where + Request: 'static, + Tx: futures::Sink + Unpin + 's, + Rx: futures::Stream + Unpin + 's, +{ + use futures::{SinkExt, StreamExt}; + + futures::stream::unfold((tx.feed(request), rx), move |(mut feed, rx)| async move { + tokio::select! { + biased; + + // We suppress a `feed` error, which represents a disconnection / reset, + // because a better and causal error will invariably be surfaced by `rx`. + _result = &mut feed => None, + + response = rx.next() => if let Some(response) = response { + Some((response, (feed, rx))) + } else { + None + }, + } + }) +} diff --git a/crates/runtime/src/materialize/mod.rs b/crates/runtime/src/materialize/mod.rs index c09cc37e1a..095669284b 100644 --- a/crates/runtime/src/materialize/mod.rs +++ b/crates/runtime/src/materialize/mod.rs @@ -17,19 +17,21 @@ impl> + Send + 'static> ResponseStream #[derive(Debug)] pub struct Task { + // Bindings of this materialization. bindings: Vec, + // ShardRef of this task. shard_ref: ops::ShardRef, } #[derive(Debug)] struct Binding { - collection_name: String, - delta_updates: bool, - key_extractors: Vec, - read_schema_json: String, - ser_policy: doc::SerPolicy, - store_document: bool, - value_extractors: Vec, + collection_name: String, // Source collection. + delta_updates: bool, // Delta updates, or standard? + key_extractors: Vec, // Key extractors for this collection. + read_schema_json: String, // Read JSON-Schema of collection documents. + ser_policy: doc::SerPolicy, // Serialization policy for this source. + store_document: bool, // Are we storing the root document (often `flow_document`)? + value_extractors: Vec, // Field extractors for this collection. } #[derive(Debug)] diff --git a/crates/runtime/src/materialize/protocol.rs b/crates/runtime/src/materialize/protocol.rs index 45910b8dc1..71543e3cf7 100644 --- a/crates/runtime/src/materialize/protocol.rs +++ b/crates/runtime/src/materialize/protocol.rs @@ -36,7 +36,7 @@ pub fn recv_client_open(open: &mut Request, db: &RocksDB) -> anyhow::Result<()> if let Some(state) = db.load_connector_state()? { let open = open.open.as_mut().unwrap(); open.state_json = state; - tracing::debug!(state=%open.state_json, "loaded and attached a persisted connector Open.state_json"); + tracing::debug!(open=%open.state_json, "loaded and attached a persisted connector Open.state_json"); } else { tracing::debug!("no previously-persisted connector state was found"); } @@ -70,10 +70,6 @@ pub fn recv_connector_opened( .load_checkpoint() .context("failed to load runtime checkpoint from RocksDB")?; - if runtime_checkpoint.is_none() { - } else { - } - if let Some(runtime_checkpoint) = runtime_checkpoint { checkpoint = runtime_checkpoint.clone(); tracing::debug!( @@ -91,12 +87,11 @@ pub fn recv_connector_opened( Ok((task, accumulator, checkpoint, opened)) } -pub fn recv_client_ack_or_load_or_flush( +pub fn recv_client_load_or_flush( accumulator: &mut doc::combine::Accumulator, buf: &mut bytes::BytesMut, load_keys: &mut HashSet<(u32, bytes::Bytes)>, request: Option, - saw_acknowledge: &mut bool, saw_acknowledged: &mut bool, saw_flush: &mut bool, task: &Task, @@ -143,29 +138,12 @@ pub fn recv_client_ack_or_load_or_flush( })) } } - Some(Request { - acknowledge: Some(request::Acknowledge {}), - .. - }) => { - if *saw_acknowledge { - anyhow::bail!("client sent duplicate Acknowledge"); - } - *saw_acknowledge = true; - - Ok(Some(Request { - flush: Some(request::Flush {}), - ..Default::default() - })) - } Some(Request { flush: Some(request::Flush {}), .. }) => { - if !*saw_acknowledge { - anyhow::bail!("client sent Flush before request::Acknowledge"); - } if !*saw_acknowledged { - anyhow::bail!("client sent Flush before response::Acknowledged"); + anyhow::bail!("client sent Flush before Acknowledged"); } *saw_flush = true; @@ -236,10 +214,7 @@ pub fn recv_connector_acked_or_loaded_or_flushed( } *saw_flushed = true; - Ok(Some(Response { - flushed: Some(response::Flushed {}), - ..Default::default() - })) + Ok(None) } request => verify("connector", "Loaded, Acknowledged, or Flushed").fail(request), } @@ -347,7 +322,7 @@ pub fn recv_client_start_commit( let mut wb = rocksdb::WriteBatch::default(); tracing::debug!( - ?runtime_checkpoint, + checkpoint=?ops::DebugJson(&runtime_checkpoint), "persisting StartCommit.runtime_checkpoint", ); wb.put(RocksDB::CHECKPOINT_KEY, runtime_checkpoint.encode_to_vec()); diff --git a/crates/runtime/src/materialize/serve.rs b/crates/runtime/src/materialize/serve.rs index 3f11972b61..3f797b7c02 100644 --- a/crates/runtime/src/materialize/serve.rs +++ b/crates/runtime/src/materialize/serve.rs @@ -36,14 +36,11 @@ impl Runtime { let db = recv_client_first_open(&open)?; - loop { - let Some(next) = serve_session(&mut co, &db, open, &mut request_rx, &self).await? - else { - return Ok(()); - }; - + while let Some(next) = serve_session(&mut co, &db, open, &mut request_rx, &self).await? + { open = next; } + Ok(()) }) } } @@ -86,8 +83,16 @@ async fn serve_session( () = co.yield_(opened).await; - // Loop over transactions. + let mut buf = bytes::BytesMut::new(); loop { + // Read and forward Acknowledge. + match request_rx.try_next().await? { + Some(ack) if ack.acknowledge.is_some() => { + connector_tx.try_send(ack).expect("sender is empty"); + } + request => return verify("client", "Acknowledge").fail(request), + } + // Loop over EOF and Open until an initial Load or Flush. let initial: Request = loop { match request_rx.try_next().await? { @@ -101,19 +106,21 @@ async fn serve_session( } Some(load @ Request { load: Some(_), .. }) => break load, Some(flush @ Request { flush: Some(_), .. }) => break flush, - request => return verify("client", "EOF, Reset, Open, or Read").fail(request), + request => return verify("client", "EOF, Open, Load, or Flush").fail(request), } }; let mut txn = Transaction::new(); txn.started_at = std::time::SystemTime::now(); + // TODO(johnny): Use RocksDB to spill this to disk. + let mut load_keys: HashSet<(u32, bytes::Bytes)> = HashSet::new(); + enum Step { ClientRx(Option), ConnectorRx(Option), ConnectorTx(Result<(), mpsc::SendError>), } - let mut saw_acknowledge = false; let mut saw_acknowledged = false; let mut saw_flush = false; let mut saw_flushed = false; @@ -121,21 +128,15 @@ async fn serve_session( let mut send_fut = None; let mut step = Step::ClientRx(Some(initial)); - let mut buf = bytes::BytesMut::new(); - - // TODO(johnny): Use RocksDB to spill this to disk. - let mut load_keys: HashSet<(u32, bytes::Bytes)> = HashSet::new(); - // Loop over client requests and connector responses until the transaction has flushed. - while !saw_flush || !saw_flushed { + loop { match step { Step::ClientRx(request) => { - if let Some(send) = recv_client_ack_or_load_or_flush( + if let Some(send) = recv_client_load_or_flush( &mut accumulator, &mut buf, &mut load_keys, request, - &mut saw_acknowledge, &mut saw_acknowledged, &mut saw_flush, &task, @@ -164,6 +165,10 @@ async fn serve_session( } } + if saw_flush && saw_flushed { + break; + } + step = if let Some(forward) = &mut send_fut { tokio::select! { result = forward => Step::ConnectorTx(result), @@ -187,14 +192,16 @@ async fn serve_session( let mut drainer = accumulator .into_drainer() .context("preparing to drain combiner")?; - let mut buf = bytes::BytesMut::new(); while let Some(drained) = drainer.drain_next()? { let store = send_connector_store(&mut buf, drained, &task, &mut txn); - if let Err(_send_error) = connector_tx.feed(store).await { - return verify("connector", "no message or EOF during Store phase") - .fail(connector_rx.try_next().await?); + tokio::select! { + biased; // Prefer to feed connector and drive connector only if there's no room. + Ok(()) = connector_tx.feed(store) => (), + response = connector_rx.try_next() => { + return verify("connector", "no response or EOF during Store phase").fail(response?); + } } } () = co.yield_(send_client_flushed(&mut buf, &task, &txn)).await; @@ -202,7 +209,13 @@ async fn serve_session( // Read StartCommit and forward to the connector. let start_commit = request_rx.try_next().await?; let (start_commit, wb) = recv_client_start_commit(last_checkpoint, start_commit, &mut txn)?; - let _: Result<(), mpsc::SendError> = connector_tx.feed(start_commit).await; + + tokio::select! { + Ok(()) = connector_tx.feed(start_commit) => (), + response = connector_rx.try_next() => { + return verify("connector", "no response or EOF during Flush phase").fail(response?); + } + } // Read StartedCommit and forward to the client. let started_commit = connector_rx.try_next().await?; @@ -219,5 +232,10 @@ async fn drain_connector( mut rx: BoxStream<'static, anyhow::Result>, ) -> anyhow::Result<()> { std::mem::drop(tx); + + match rx.try_next().await? { + Some(ack) if ack.acknowledged.is_some() => (), + response => return verify("connector", "Acknowledged").fail(response), + } verify("connector", "EOF").is_eof(rx.try_next().await?) } diff --git a/crates/runtime/src/materialize/task.rs b/crates/runtime/src/materialize/task.rs index f700f5ed3f..15418f597f 100644 --- a/crates/runtime/src/materialize/task.rs +++ b/crates/runtime/src/materialize/task.rs @@ -120,8 +120,8 @@ impl Binding { write_schema_json, } = collection.as_ref().context("missing collection")?; - let key_extractors = extractors::for_key(selected_key, projections, ser_policy)?; - let value_extractors = extractors::for_key(selected_values, projections, ser_policy)?; + let key_extractors = extractors::for_fields(selected_key, projections, ser_policy)?; + let value_extractors = extractors::for_fields(selected_values, projections, ser_policy)?; let read_schema_json = if read_schema_json.is_empty() { write_schema_json diff --git a/crates/runtime/src/uuid.rs b/crates/runtime/src/uuid.rs index f5bed97269..35738c174c 100644 --- a/crates/runtime/src/uuid.rs +++ b/crates/runtime/src/uuid.rs @@ -85,7 +85,7 @@ impl Clock { std::time::UNIX_EPOCH + unix } - const UNIX_EPOCH: Self = Clock::from_unix(0, 0); + pub const UNIX_EPOCH: Self = Clock::from_unix(0, 0); } impl Flags { diff --git a/go/protocols/runtime/runtime.pb.go b/go/protocols/runtime/runtime.pb.go index 36c056484b..827f39038c 100644 --- a/go/protocols/runtime/runtime.pb.go +++ b/go/protocols/runtime/runtime.pb.go @@ -35,6 +35,44 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package +type CaptureResponseExt_PollResult int32 + +const ( + CaptureResponseExt_INVALID CaptureResponseExt_PollResult = 0 + // The poll succeeded and a transaction will be drained. + CaptureResponseExt_READY CaptureResponseExt_PollResult = 1 + // The connector is running but has produced no data. + CaptureResponseExt_NOT_READY CaptureResponseExt_PollResult = 2 + // The connector has exited, but cannot be restarted yet. + CaptureResponseExt_COOL_OFF CaptureResponseExt_PollResult = 3 + // The connector has exited and new session should be started. + CaptureResponseExt_RESTART CaptureResponseExt_PollResult = 4 +) + +var CaptureResponseExt_PollResult_name = map[int32]string{ + 0: "INVALID", + 1: "READY", + 2: "NOT_READY", + 3: "COOL_OFF", + 4: "RESTART", +} + +var CaptureResponseExt_PollResult_value = map[string]int32{ + "INVALID": 0, + "READY": 1, + "NOT_READY": 2, + "COOL_OFF": 3, + "RESTART": 4, +} + +func (x CaptureResponseExt_PollResult) String() string { + return proto.EnumName(CaptureResponseExt_PollResult_name, int32(x)) +} + +func (CaptureResponseExt_PollResult) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{6, 0} +} + type TaskServiceConfig struct { LogFileFd int32 `protobuf:"varint,1,opt,name=log_file_fd,json=logFileFd,proto3" json:"log_file_fd,omitempty"` TaskName string `protobuf:"bytes,2,opt,name=task_name,json=taskName,proto3" json:"task_name,omitempty"` @@ -328,7 +366,7 @@ var xxx_messageInfo_Container proto.InternalMessageInfo type CaptureRequestExt struct { Labels *ops.ShardLabeling `protobuf:"bytes,1,opt,name=labels,proto3" json:"labels,omitempty"` Open *CaptureRequestExt_Open `protobuf:"bytes,2,opt,name=open,proto3" json:"open,omitempty"` - Acknowledge *CaptureRequestExt_Acknowledge `protobuf:"bytes,3,opt,name=acknowledge,proto3" json:"acknowledge,omitempty"` + StartCommit *CaptureRequestExt_StartCommit `protobuf:"bytes,3,opt,name=start_commit,json=startCommit,proto3" json:"start_commit,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -408,27 +446,26 @@ func (m *CaptureRequestExt_Open) XXX_DiscardUnknown() { var xxx_messageInfo_CaptureRequestExt_Open proto.InternalMessageInfo -type CaptureRequestExt_Acknowledge struct { - // This Acknowledge is acting as a StartCommit. +type CaptureRequestExt_StartCommit struct { // Flow runtime checkpoint associated with this transaction. - StartCommitRuntimeCheckpoint *protocol1.Checkpoint `protobuf:"bytes,1,opt,name=start_commit_runtime_checkpoint,json=startCommitRuntimeCheckpoint,proto3" json:"start_commit_runtime_checkpoint,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + RuntimeCheckpoint *protocol1.Checkpoint `protobuf:"bytes,1,opt,name=runtime_checkpoint,json=runtimeCheckpoint,proto3" json:"runtime_checkpoint,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *CaptureRequestExt_Acknowledge) Reset() { *m = CaptureRequestExt_Acknowledge{} } -func (m *CaptureRequestExt_Acknowledge) String() string { return proto.CompactTextString(m) } -func (*CaptureRequestExt_Acknowledge) ProtoMessage() {} -func (*CaptureRequestExt_Acknowledge) Descriptor() ([]byte, []int) { +func (m *CaptureRequestExt_StartCommit) Reset() { *m = CaptureRequestExt_StartCommit{} } +func (m *CaptureRequestExt_StartCommit) String() string { return proto.CompactTextString(m) } +func (*CaptureRequestExt_StartCommit) ProtoMessage() {} +func (*CaptureRequestExt_StartCommit) Descriptor() ([]byte, []int) { return fileDescriptor_73af6e0737ce390c, []int{5, 1} } -func (m *CaptureRequestExt_Acknowledge) XXX_Unmarshal(b []byte) error { +func (m *CaptureRequestExt_StartCommit) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) } -func (m *CaptureRequestExt_Acknowledge) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { +func (m *CaptureRequestExt_StartCommit) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { if deterministic { - return xxx_messageInfo_CaptureRequestExt_Acknowledge.Marshal(b, m, deterministic) + return xxx_messageInfo_CaptureRequestExt_StartCommit.Marshal(b, m, deterministic) } else { b = b[:cap(b)] n, err := m.MarshalToSizedBuffer(b) @@ -438,17 +475,17 @@ func (m *CaptureRequestExt_Acknowledge) XXX_Marshal(b []byte, deterministic bool return b[:n], nil } } -func (m *CaptureRequestExt_Acknowledge) XXX_Merge(src proto.Message) { - xxx_messageInfo_CaptureRequestExt_Acknowledge.Merge(m, src) +func (m *CaptureRequestExt_StartCommit) XXX_Merge(src proto.Message) { + xxx_messageInfo_CaptureRequestExt_StartCommit.Merge(m, src) } -func (m *CaptureRequestExt_Acknowledge) XXX_Size() int { +func (m *CaptureRequestExt_StartCommit) XXX_Size() int { return m.ProtoSize() } -func (m *CaptureRequestExt_Acknowledge) XXX_DiscardUnknown() { - xxx_messageInfo_CaptureRequestExt_Acknowledge.DiscardUnknown(m) +func (m *CaptureRequestExt_StartCommit) XXX_DiscardUnknown() { + xxx_messageInfo_CaptureRequestExt_StartCommit.DiscardUnknown(m) } -var xxx_messageInfo_CaptureRequestExt_Acknowledge proto.InternalMessageInfo +var xxx_messageInfo_CaptureRequestExt_StartCommit proto.InternalMessageInfo type CaptureResponseExt struct { Container *Container `protobuf:"bytes,1,opt,name=container,proto3" json:"container,omitempty"` @@ -536,10 +573,11 @@ func (m *CaptureResponseExt_Captured) XXX_DiscardUnknown() { var xxx_messageInfo_CaptureResponseExt_Captured proto.InternalMessageInfo type CaptureResponseExt_Checkpoint struct { - Stats *ops.Stats `protobuf:"bytes,1,opt,name=stats,proto3" json:"stats,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Stats *ops.Stats `protobuf:"bytes,1,opt,name=stats,proto3" json:"stats,omitempty"` + PollResult CaptureResponseExt_PollResult `protobuf:"varint,2,opt,name=poll_result,json=pollResult,proto3,enum=runtime.CaptureResponseExt_PollResult" json:"poll_result,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *CaptureResponseExt_Checkpoint) Reset() { *m = CaptureResponseExt_Checkpoint{} } @@ -910,10 +948,11 @@ func (m *MaterializeRequestExt_Open) XXX_DiscardUnknown() { var xxx_messageInfo_MaterializeRequestExt_Open proto.InternalMessageInfo type MaterializeResponseExt struct { - Container *Container `protobuf:"bytes,1,opt,name=container,proto3" json:"container,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Container *Container `protobuf:"bytes,1,opt,name=container,proto3" json:"container,omitempty"` + Flushed *MaterializeResponseExt_Flushed `protobuf:"bytes,2,opt,name=flushed,proto3" json:"flushed,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *MaterializeResponseExt) Reset() { *m = MaterializeResponseExt{} } @@ -949,7 +988,48 @@ func (m *MaterializeResponseExt) XXX_DiscardUnknown() { var xxx_messageInfo_MaterializeResponseExt proto.InternalMessageInfo +type MaterializeResponseExt_Flushed struct { + Stats *ops.Stats `protobuf:"bytes,1,opt,name=stats,proto3" json:"stats,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MaterializeResponseExt_Flushed) Reset() { *m = MaterializeResponseExt_Flushed{} } +func (m *MaterializeResponseExt_Flushed) String() string { return proto.CompactTextString(m) } +func (*MaterializeResponseExt_Flushed) ProtoMessage() {} +func (*MaterializeResponseExt_Flushed) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{10, 0} +} +func (m *MaterializeResponseExt_Flushed) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MaterializeResponseExt_Flushed) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_MaterializeResponseExt_Flushed.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *MaterializeResponseExt_Flushed) XXX_Merge(src proto.Message) { + xxx_messageInfo_MaterializeResponseExt_Flushed.Merge(m, src) +} +func (m *MaterializeResponseExt_Flushed) XXX_Size() int { + return m.ProtoSize() +} +func (m *MaterializeResponseExt_Flushed) XXX_DiscardUnknown() { + xxx_messageInfo_MaterializeResponseExt_Flushed.DiscardUnknown(m) +} + +var xxx_messageInfo_MaterializeResponseExt_Flushed proto.InternalMessageInfo + func init() { + proto.RegisterEnum("runtime.CaptureResponseExt_PollResult", CaptureResponseExt_PollResult_name, CaptureResponseExt_PollResult_value) proto.RegisterType((*TaskServiceConfig)(nil), "runtime.TaskServiceConfig") proto.RegisterType((*ShuffleRequest)(nil), "runtime.ShuffleRequest") proto.RegisterType((*ShuffleResponse)(nil), "runtime.ShuffleResponse") @@ -958,7 +1038,7 @@ func init() { proto.RegisterMapType((map[uint32]string)(nil), "runtime.Container.MappedHostPortsEntry") proto.RegisterType((*CaptureRequestExt)(nil), "runtime.CaptureRequestExt") proto.RegisterType((*CaptureRequestExt_Open)(nil), "runtime.CaptureRequestExt.Open") - proto.RegisterType((*CaptureRequestExt_Acknowledge)(nil), "runtime.CaptureRequestExt.Acknowledge") + proto.RegisterType((*CaptureRequestExt_StartCommit)(nil), "runtime.CaptureRequestExt.StartCommit") proto.RegisterType((*CaptureResponseExt)(nil), "runtime.CaptureResponseExt") proto.RegisterType((*CaptureResponseExt_Captured)(nil), "runtime.CaptureResponseExt.Captured") proto.RegisterType((*CaptureResponseExt_Checkpoint)(nil), "runtime.CaptureResponseExt.Checkpoint") @@ -971,6 +1051,7 @@ func init() { proto.RegisterType((*MaterializeRequestExt)(nil), "runtime.MaterializeRequestExt") proto.RegisterType((*MaterializeRequestExt_Open)(nil), "runtime.MaterializeRequestExt.Open") proto.RegisterType((*MaterializeResponseExt)(nil), "runtime.MaterializeResponseExt") + proto.RegisterType((*MaterializeResponseExt_Flushed)(nil), "runtime.MaterializeResponseExt.Flushed") } func init() { @@ -978,102 +1059,107 @@ func init() { } var fileDescriptor_73af6e0737ce390c = []byte{ - // 1514 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0x4b, 0x6f, 0x1b, 0xc9, - 0x11, 0xde, 0x21, 0x29, 0x3e, 0x8a, 0x92, 0x2c, 0x36, 0x94, 0x5d, 0x2e, 0xb3, 0x11, 0xb9, 0x5c, - 0x6f, 0x22, 0xac, 0x9c, 0xa1, 0x23, 0x07, 0x79, 0x21, 0x30, 0x62, 0x52, 0x12, 0x24, 0xc7, 0xb2, - 0xe5, 0xa6, 0xed, 0x43, 0x72, 0x18, 0xb4, 0x66, 0x9a, 0xe4, 0x98, 0xc3, 0xe9, 0x71, 0x4f, 0x8f, - 0x1e, 0xfe, 0x35, 0xf9, 0x1b, 0xf9, 0x07, 0xbe, 0x25, 0xc8, 0x29, 0x27, 0x01, 0x71, 0x8e, 0xb9, - 0x26, 0x87, 0x08, 0x08, 0x10, 0xf4, 0x63, 0x86, 0xa4, 0x5e, 0x50, 0xe4, 0x1c, 0x24, 0x75, 0x7f, - 0x55, 0x5f, 0x55, 0x57, 0xf7, 0xd7, 0xd5, 0x23, 0x68, 0x0f, 0x59, 0x27, 0xe2, 0x4c, 0x30, 0x97, - 0x05, 0x71, 0x87, 0x27, 0xa1, 0xf0, 0x27, 0x34, 0xfd, 0x6b, 0x2b, 0x0b, 0x2a, 0x99, 0x69, 0x63, - 0xed, 0x90, 0xb3, 0x31, 0xe5, 0x19, 0x21, 0x1b, 0x68, 0xc7, 0x46, 0xcb, 0x65, 0x61, 0x9c, 0x4c, - 0x6e, 0xf0, 0xf8, 0x6a, 0x2e, 0xdd, 0x20, 0x60, 0xc7, 0xea, 0x97, 0xb1, 0x36, 0xe6, 0xac, 0x2c, - 0x52, 0x3f, 0xc6, 0xb6, 0x3a, 0x64, 0x43, 0xa6, 0x86, 0x1d, 0x39, 0xd2, 0x68, 0xfb, 0x8f, 0x16, - 0xd4, 0x5e, 0x91, 0x78, 0xdc, 0xa7, 0xfc, 0xc8, 0x77, 0x69, 0x8f, 0x85, 0x03, 0x7f, 0x88, 0xd6, - 0xa0, 0x1a, 0xb0, 0xa1, 0x33, 0xf0, 0x03, 0xea, 0x0c, 0xbc, 0xba, 0xd5, 0xb2, 0xd6, 0x17, 0x70, - 0x25, 0x60, 0xc3, 0x1d, 0x3f, 0xa0, 0x3b, 0x1e, 0xfa, 0x3e, 0x54, 0x04, 0x89, 0xc7, 0x4e, 0x48, - 0x26, 0xb4, 0x9e, 0x6b, 0x59, 0xeb, 0x15, 0x5c, 0x96, 0xc0, 0x73, 0x32, 0xa1, 0xe8, 0x4b, 0x28, - 0x27, 0x5e, 0xec, 0x44, 0x44, 0x8c, 0xea, 0x79, 0x65, 0x2b, 0x25, 0x5e, 0x7c, 0x40, 0xc4, 0x08, - 0x6d, 0x40, 0xcd, 0x65, 0xa1, 0x20, 0x7e, 0x48, 0xb9, 0x13, 0x52, 0x71, 0xcc, 0xf8, 0xb8, 0x5e, - 0x50, 0x3e, 0x2b, 0x99, 0xe1, 0xb9, 0xc6, 0x51, 0x13, 0xaa, 0x24, 0x08, 0xd8, 0xb1, 0x13, 0x30, - 0x97, 0x04, 0xf5, 0x85, 0x96, 0xb5, 0x5e, 0xc6, 0xa0, 0xa0, 0x67, 0x12, 0x69, 0xff, 0xbb, 0x00, - 0xcb, 0xfd, 0x51, 0x32, 0x18, 0x04, 0x14, 0xd3, 0x77, 0x09, 0x8d, 0x05, 0xda, 0x83, 0xd2, 0x5b, - 0x96, 0xf0, 0x90, 0x04, 0x6a, 0xd1, 0x95, 0x6e, 0xe7, 0xfc, 0xac, 0xb9, 0x31, 0x64, 0xf6, 0x90, - 0xbc, 0xa7, 0x42, 0x50, 0xdb, 0xa3, 0x47, 0x1d, 0x97, 0x71, 0xda, 0xb9, 0x70, 0x12, 0xf6, 0x53, - 0x4d, 0xc3, 0x29, 0x1f, 0x7d, 0x0e, 0x45, 0x4e, 0xa3, 0x80, 0x9c, 0xaa, 0x02, 0xcb, 0xd8, 0xcc, - 0x64, 0x79, 0x87, 0x89, 0x1f, 0x78, 0x8e, 0xef, 0xa5, 0xe5, 0xa9, 0xf9, 0x9e, 0x87, 0x76, 0xa0, - 0xc8, 0x06, 0x83, 0x98, 0x0a, 0x55, 0x53, 0xbe, 0x6b, 0x9f, 0x9f, 0x35, 0xbf, 0xbb, 0x4d, 0xf2, - 0x17, 0x8a, 0x85, 0x0d, 0x1b, 0xed, 0x03, 0xd0, 0xd0, 0x73, 0x4c, 0xac, 0x85, 0x3b, 0xc5, 0xaa, - 0xd0, 0xd0, 0xd3, 0x43, 0xb4, 0x01, 0x0b, 0x9c, 0x84, 0x43, 0x5a, 0x2f, 0xb6, 0xac, 0xf5, 0xea, - 0xe6, 0x3d, 0x5b, 0x29, 0x06, 0x4b, 0xa8, 0x1f, 0x51, 0xb7, 0x5b, 0xf8, 0x70, 0xd6, 0xfc, 0x0c, - 0x6b, 0x1f, 0xd4, 0x87, 0xaa, 0xcb, 0x18, 0xf7, 0xfc, 0x90, 0x08, 0xc6, 0xeb, 0x25, 0xb5, 0x8b, - 0x3f, 0x39, 0x3f, 0x6b, 0xfe, 0xf8, 0xaa, 0xe4, 0x97, 0xf4, 0x6a, 0xf7, 0x47, 0x84, 0x7b, 0x7b, - 0x5b, 0x78, 0x36, 0x0a, 0x7a, 0x08, 0xc0, 0x69, 0xcc, 0x82, 0x44, 0xf8, 0x2c, 0xac, 0x97, 0xd5, - 0x32, 0x56, 0xec, 0x8c, 0xb3, 0x4b, 0x89, 0x47, 0x39, 0x9e, 0xf1, 0x41, 0xdf, 0xc0, 0x52, 0xac, - 0x8f, 0xd6, 0xf1, 0x43, 0x8f, 0x9e, 0xd4, 0x2b, 0x2d, 0x6b, 0x7d, 0x09, 0x2f, 0x1a, 0x70, 0x4f, - 0x62, 0xe8, 0xa7, 0x00, 0x1e, 0xe5, 0xfe, 0x11, 0x51, 0x61, 0x41, 0x85, 0x5d, 0xd5, 0xd5, 0xf5, - 0x58, 0x10, 0x50, 0x57, 0xe2, 0xb2, 0x44, 0x3c, 0xe3, 0x87, 0x7a, 0x70, 0x6f, 0x42, 0x04, 0xe5, - 0x3e, 0x09, 0xfc, 0xf7, 0x9a, 0x5a, 0x55, 0xd4, 0x2f, 0x35, 0x75, 0x7f, 0xde, 0xa8, 0xf8, 0x17, - 0x19, 0xed, 0xbf, 0x14, 0xe0, 0x5e, 0xa6, 0xbd, 0x38, 0x62, 0x61, 0x4c, 0xd1, 0x3a, 0x14, 0x63, - 0x41, 0x44, 0x12, 0x2b, 0xed, 0x2d, 0x6f, 0xae, 0xd8, 0xe9, 0xf6, 0xd8, 0x7d, 0x85, 0x63, 0x63, - 0x97, 0x9e, 0x23, 0x55, 0xb3, 0xd2, 0xd6, 0x55, 0x7b, 0x61, 0xec, 0xe8, 0x5b, 0x58, 0x16, 0x94, - 0x4f, 0xfc, 0x90, 0x04, 0x0e, 0xe5, 0x9c, 0x71, 0xa3, 0xb9, 0xa5, 0x14, 0xdd, 0x96, 0x20, 0x7a, - 0x09, 0x8b, 0x9c, 0x12, 0xcf, 0x11, 0x23, 0xce, 0x92, 0xe1, 0xe8, 0x8e, 0xfa, 0xab, 0xca, 0x18, - 0xaf, 0x74, 0x08, 0x29, 0xc2, 0x63, 0xee, 0x0b, 0xea, 0xc8, 0x95, 0xdc, 0x55, 0x84, 0x2a, 0x82, - 0x2c, 0x09, 0xed, 0xc1, 0x02, 0xe1, 0x34, 0x24, 0x4a, 0x84, 0x8b, 0xdd, 0x47, 0xe7, 0x67, 0xcd, - 0xce, 0xd0, 0x17, 0xa3, 0xe4, 0xd0, 0x76, 0xd9, 0xa4, 0x43, 0x63, 0x91, 0x10, 0x7e, 0xaa, 0x3b, - 0xda, 0xa5, 0x1e, 0x67, 0x3f, 0x91, 0x54, 0xac, 0x23, 0xa0, 0x6f, 0xa1, 0xe0, 0x31, 0x37, 0xae, - 0x97, 0x5a, 0xf9, 0xf5, 0xea, 0x66, 0x55, 0x9f, 0x5a, 0x3f, 0xf0, 0x5d, 0x6a, 0xa4, 0xac, 0xcc, - 0x68, 0x17, 0x4a, 0xfa, 0x06, 0xc5, 0xf5, 0x72, 0x2b, 0x7f, 0x87, 0xd5, 0xa7, 0x74, 0xa9, 0xb3, - 0x24, 0xf1, 0x3d, 0x27, 0x22, 0x5c, 0xc4, 0xf5, 0x8a, 0x4a, 0x6b, 0x6e, 0xd1, 0xeb, 0xd7, 0x7b, - 0x5b, 0x07, 0x12, 0x36, 0xa9, 0x2b, 0xd2, 0x51, 0x01, 0x52, 0xf4, 0x11, 0x71, 0xc7, 0xd4, 0x73, - 0xc6, 0xf4, 0xb4, 0x0e, 0xd7, 0x2d, 0xb6, 0xa2, 0x9d, 0x7e, 0x4b, 0x4f, 0xdb, 0x1e, 0xd4, 0x30, - 0x73, 0xc7, 0xf1, 0x56, 0x77, 0x8b, 0xc6, 0x2e, 0xf7, 0x23, 0x79, 0x77, 0x1e, 0x00, 0xe2, 0x12, - 0xf4, 0x0e, 0x1d, 0x1a, 0x1e, 0x39, 0x13, 0x3a, 0x89, 0x04, 0x57, 0x0a, 0x2b, 0xe2, 0x15, 0x63, - 0xd9, 0x0e, 0x8f, 0xf6, 0x15, 0x8e, 0xbe, 0x86, 0xc5, 0xd4, 0x5b, 0x35, 0x60, 0xdd, 0x9c, 0xab, - 0x06, 0x93, 0x4d, 0xb8, 0xfd, 0x2f, 0x0b, 0x2a, 0xbd, 0xb4, 0xd9, 0xa2, 0x2f, 0xa0, 0xe4, 0x47, - 0x0e, 0xf1, 0x3c, 0x1d, 0xb3, 0x82, 0x8b, 0x7e, 0xf4, 0xc4, 0xf3, 0x38, 0xfa, 0x19, 0x2c, 0x99, - 0x0e, 0xed, 0x44, 0x4c, 0xd6, 0x9d, 0x53, 0x15, 0xd4, 0x74, 0x05, 0xa6, 0x49, 0x1f, 0x30, 0x2e, - 0xf0, 0x62, 0x38, 0x9d, 0xc4, 0xa8, 0x0f, 0xb5, 0x09, 0x89, 0x22, 0xea, 0x39, 0x23, 0x16, 0x0b, - 0xc3, 0xcd, 0x2b, 0xee, 0x8f, 0xec, 0xf4, 0x5d, 0xcc, 0xf2, 0xdb, 0xfb, 0xca, 0x77, 0x97, 0xc5, - 0x42, 0xd1, 0xb7, 0x43, 0xc1, 0x4f, 0xe5, 0x75, 0x9b, 0x43, 0x1b, 0x5d, 0x58, 0xbd, 0xca, 0x11, - 0xad, 0x40, 0x5e, 0x6e, 0xae, 0xa5, 0x9a, 0x83, 0x1c, 0xa2, 0x55, 0x58, 0x38, 0x22, 0x41, 0x92, - 0x3e, 0x4b, 0x7a, 0xf2, 0xab, 0xdc, 0x2f, 0xac, 0xf6, 0x7f, 0x72, 0x50, 0xeb, 0x91, 0x48, 0x24, - 0x3c, 0x7d, 0x2e, 0xb6, 0x4f, 0x04, 0xfa, 0x0e, 0x8a, 0x01, 0x39, 0xa4, 0x81, 0xbe, 0xb4, 0xd5, - 0x4d, 0x64, 0xcb, 0x27, 0x53, 0x75, 0xb1, 0x67, 0x12, 0xf7, 0xc3, 0x21, 0x36, 0x1e, 0xe8, 0x11, - 0x14, 0x58, 0x44, 0x43, 0x73, 0x69, 0x9b, 0xd3, 0x6a, 0x2e, 0x46, 0xb5, 0x5f, 0x44, 0x34, 0xc4, - 0xca, 0x19, 0xed, 0x42, 0x95, 0xb8, 0xe3, 0x90, 0x1d, 0x07, 0xd4, 0x1b, 0x52, 0x75, 0x7d, 0xab, - 0x9b, 0x3f, 0xbc, 0x81, 0xfb, 0x64, 0xea, 0x8d, 0x67, 0xa9, 0x8d, 0x97, 0x50, 0x90, 0x71, 0xd1, - 0xde, 0x54, 0x11, 0x5e, 0xa6, 0x13, 0xb3, 0xfc, 0x46, 0x16, 0xf8, 0x92, 0x92, 0x70, 0xcd, 0xb0, - 0xa6, 0x50, 0xe3, 0x2d, 0x54, 0x67, 0xd2, 0xa1, 0xdf, 0x43, 0x33, 0x16, 0x84, 0x0b, 0xc7, 0x65, - 0x93, 0x89, 0x2f, 0x1c, 0x13, 0xcb, 0x71, 0x47, 0xd4, 0x1d, 0x47, 0xcc, 0x0f, 0x85, 0x49, 0xb3, - 0x3a, 0x6d, 0x6d, 0xbd, 0xcc, 0x86, 0xbf, 0x52, 0xe4, 0x9e, 0xe2, 0x62, 0x4d, 0x9d, 0x5a, 0xdb, - 0x7f, 0xcd, 0x01, 0xca, 0xaa, 0xd5, 0x2d, 0x53, 0x1e, 0xc0, 0x43, 0xa8, 0x64, 0x4f, 0x7f, 0x76, - 0x06, 0x97, 0x74, 0x82, 0xa7, 0x4e, 0xe8, 0x37, 0x50, 0x76, 0x75, 0x1c, 0xcf, 0x1c, 0xc5, 0xfd, - 0xcb, 0xdb, 0x99, 0x25, 0x48, 0x21, 0x0f, 0x67, 0x2c, 0xb4, 0x03, 0x30, 0x53, 0x52, 0xe1, 0xba, - 0x23, 0x99, 0x89, 0x31, 0x2d, 0x72, 0x86, 0xd9, 0x78, 0x03, 0xe5, 0x34, 0x3a, 0xfa, 0x01, 0xc0, - 0x98, 0x9e, 0x3a, 0xfa, 0x36, 0xab, 0x42, 0x16, 0x71, 0x65, 0x4c, 0x4f, 0x0f, 0x14, 0x20, 0x3f, - 0x7d, 0x64, 0xfb, 0xf0, 0xe5, 0xeb, 0x11, 0xa7, 0x5e, 0x39, 0xe5, 0xb5, 0x32, 0x35, 0x68, 0xe7, - 0x86, 0x0d, 0x30, 0xcd, 0x88, 0x5a, 0xb0, 0x20, 0xdf, 0x8d, 0x54, 0xa1, 0xa0, 0x15, 0x2a, 0x11, - 0xac, 0x0d, 0xed, 0x7f, 0x58, 0xb0, 0xb2, 0x25, 0x5f, 0xb8, 0xbb, 0x2a, 0x7b, 0x73, 0x4e, 0xd9, - 0x6b, 0xd9, 0x56, 0x5c, 0x0c, 0x3a, 0x23, 0xec, 0xc6, 0xb1, 0x91, 0xe3, 0x7d, 0x58, 0x8e, 0xdf, - 0x05, 0xf2, 0xa5, 0x38, 0x1a, 0xc4, 0x4e, 0xc2, 0x7d, 0xd3, 0x48, 0x16, 0x35, 0xfa, 0x66, 0x10, - 0xbf, 0xe6, 0xfe, 0x35, 0xa2, 0xcd, 0xdd, 0x41, 0xb4, 0xed, 0x7f, 0xe6, 0xa1, 0x96, 0x2e, 0xec, - 0x53, 0x74, 0xf4, 0x4b, 0x28, 0xca, 0x42, 0x32, 0x15, 0x7d, 0x7d, 0xa9, 0xec, 0xa9, 0x00, 0x5e, - 0x28, 0x47, 0x6c, 0x08, 0xa8, 0x0b, 0x95, 0x28, 0x39, 0x0c, 0xfc, 0x78, 0x44, 0x3d, 0x73, 0xa5, - 0xef, 0xdf, 0xc0, 0x3e, 0x48, 0x7d, 0xf1, 0x94, 0x86, 0x7e, 0x0d, 0xa5, 0x41, 0x90, 0xa8, 0x08, - 0x5a, 0x81, 0xed, 0x1b, 0x22, 0xec, 0x68, 0x4f, 0x9c, 0x52, 0x1a, 0xfb, 0x50, 0xd4, 0x6b, 0x42, - 0x3d, 0x40, 0xff, 0xe3, 0x3d, 0xad, 0xf1, 0x8b, 0x97, 0xb3, 0x21, 0xa0, 0x92, 0x2d, 0x52, 0x7e, - 0xde, 0x4f, 0xc8, 0x89, 0xe3, 0x06, 0xcc, 0x1d, 0x9b, 0x97, 0xa6, 0x3c, 0x21, 0x27, 0x3d, 0x39, - 0xbf, 0xa0, 0xf3, 0xdc, 0xad, 0x74, 0x9e, 0xbf, 0x46, 0xe7, 0x1b, 0x50, 0x32, 0x85, 0xdd, 0x42, - 0xe4, 0x7f, 0xb2, 0xe0, 0x7b, 0xd3, 0x6f, 0xb3, 0xbb, 0x2a, 0xfd, 0xe7, 0x73, 0x4a, 0xff, 0x26, - 0xdb, 0xf2, 0x2b, 0x23, 0xcf, 0xca, 0xfd, 0xff, 0xdf, 0x7d, 0xdb, 0x4f, 0xe1, 0xf3, 0xb9, 0xb4, - 0x9f, 0x20, 0xe6, 0xcd, 0xa7, 0x50, 0x36, 0xdf, 0xa3, 0x1c, 0x3d, 0x86, 0x92, 0x19, 0xa3, 0x2f, - 0x32, 0xd6, 0xfc, 0x7f, 0x4a, 0x8d, 0xfa, 0x65, 0x83, 0x4e, 0xff, 0xd0, 0xea, 0x3e, 0xfe, 0xf0, - 0xb7, 0xb5, 0xcf, 0x3e, 0x7c, 0x5c, 0xb3, 0xfe, 0xfc, 0x71, 0xcd, 0xfa, 0xc3, 0xdf, 0xd7, 0xac, - 0xdf, 0x3d, 0xb8, 0xd5, 0xa7, 0x9a, 0x89, 0x78, 0x58, 0x54, 0xd0, 0xa3, 0xff, 0x06, 0x00, 0x00, - 0xff, 0xff, 0x23, 0xe5, 0x45, 0x58, 0x1c, 0x0f, 0x00, 0x00, + // 1599 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0xdd, 0x6e, 0x1b, 0xc7, + 0x15, 0xf6, 0x8a, 0x14, 0xc9, 0x3d, 0xa4, 0x64, 0x72, 0xa0, 0x26, 0x0c, 0xdb, 0x4a, 0x0c, 0xe3, + 0x34, 0x42, 0x9c, 0x52, 0xae, 0x5c, 0xf4, 0x0f, 0x45, 0x50, 0x89, 0x92, 0x6a, 0xa6, 0xd6, 0x4f, + 0x86, 0xb2, 0x81, 0xf6, 0x66, 0x31, 0xda, 0x1d, 0x92, 0x5b, 0x0e, 0x77, 0x36, 0xb3, 0xb3, 0x92, + 0x95, 0xa7, 0xe9, 0x6b, 0xb4, 0x4f, 0xe0, 0xbb, 0x16, 0x7d, 0x00, 0x01, 0x4d, 0x81, 0xde, 0xf4, + 0xaa, 0x40, 0x7b, 0x51, 0x5f, 0x15, 0xf3, 0xb3, 0x4b, 0xea, 0xb7, 0x8e, 0x92, 0x0b, 0x5b, 0x33, + 0xdf, 0x39, 0xdf, 0x99, 0x39, 0x33, 0xdf, 0x39, 0xb3, 0x84, 0xce, 0x88, 0x6f, 0xc4, 0x82, 0x4b, + 0xee, 0x73, 0x96, 0x6c, 0x88, 0x34, 0x92, 0xe1, 0x94, 0x66, 0x7f, 0xbb, 0xda, 0x82, 0xca, 0x76, + 0xda, 0x5a, 0x3d, 0x11, 0x7c, 0x42, 0x45, 0x4e, 0xc8, 0x07, 0xc6, 0xb1, 0xd5, 0xf6, 0x79, 0x94, + 0xa4, 0xd3, 0x3b, 0x3c, 0xbe, 0x77, 0x69, 0xb9, 0x21, 0xe3, 0x67, 0xfa, 0x3f, 0x6b, 0x6d, 0x5d, + 0xb2, 0xf2, 0x58, 0xff, 0xb3, 0xb6, 0x95, 0x11, 0x1f, 0x71, 0x3d, 0xdc, 0x50, 0x23, 0x83, 0x76, + 0xfe, 0xe8, 0x40, 0xe3, 0x98, 0x24, 0x93, 0x01, 0x15, 0xa7, 0xa1, 0x4f, 0x7b, 0x3c, 0x1a, 0x86, + 0x23, 0xb4, 0x0a, 0x55, 0xc6, 0x47, 0xde, 0x30, 0x64, 0xd4, 0x1b, 0x06, 0x4d, 0xa7, 0xed, 0xac, + 0x2f, 0x62, 0x97, 0xf1, 0xd1, 0x5e, 0xc8, 0xe8, 0x5e, 0x80, 0xbe, 0x0b, 0xae, 0x24, 0xc9, 0xc4, + 0x8b, 0xc8, 0x94, 0x36, 0x17, 0xda, 0xce, 0xba, 0x8b, 0x2b, 0x0a, 0x38, 0x20, 0x53, 0x8a, 0xde, + 0x83, 0x4a, 0x1a, 0x24, 0x5e, 0x4c, 0xe4, 0xb8, 0x59, 0xd0, 0xb6, 0x72, 0x1a, 0x24, 0x47, 0x44, + 0x8e, 0xd1, 0x63, 0x68, 0xf8, 0x3c, 0x92, 0x24, 0x8c, 0xa8, 0xf0, 0x22, 0x2a, 0xcf, 0xb8, 0x98, + 0x34, 0x8b, 0xda, 0xa7, 0x9e, 0x1b, 0x0e, 0x0c, 0x8e, 0xd6, 0xa0, 0x4a, 0x18, 0xe3, 0x67, 0x1e, + 0xe3, 0x3e, 0x61, 0xcd, 0xc5, 0xb6, 0xb3, 0x5e, 0xc1, 0xa0, 0xa1, 0xe7, 0x0a, 0xe9, 0xfc, 0xb7, + 0x08, 0xcb, 0x83, 0x71, 0x3a, 0x1c, 0x32, 0x8a, 0xe9, 0x17, 0x29, 0x4d, 0x24, 0xea, 0x43, 0xf9, + 0xf7, 0x3c, 0x15, 0x11, 0x61, 0x7a, 0xd3, 0xee, 0xf6, 0xc6, 0x9b, 0x8b, 0xb5, 0xc7, 0x23, 0xde, + 0x1d, 0x91, 0x2f, 0xa9, 0x94, 0xb4, 0x1b, 0xd0, 0xd3, 0x0d, 0x9f, 0x0b, 0xba, 0x71, 0xe5, 0x26, + 0xba, 0x9f, 0x19, 0x1a, 0xce, 0xf8, 0xe8, 0x1d, 0x28, 0x09, 0x1a, 0x33, 0x72, 0xae, 0x13, 0xac, + 0x60, 0x3b, 0x53, 0xe9, 0x9d, 0xa4, 0x21, 0x0b, 0xbc, 0x30, 0xc8, 0xd2, 0xd3, 0xf3, 0x7e, 0x80, + 0xf6, 0xa0, 0xc4, 0x87, 0xc3, 0x84, 0x4a, 0x9d, 0x53, 0x61, 0xbb, 0xfb, 0xe6, 0x62, 0xed, 0xe3, + 0xb7, 0x59, 0xfc, 0x50, 0xb3, 0xb0, 0x65, 0xa3, 0x7d, 0x00, 0x1a, 0x05, 0x9e, 0x8d, 0xb5, 0x78, + 0xaf, 0x58, 0x2e, 0x8d, 0x02, 0x33, 0x44, 0x8f, 0x61, 0x51, 0x90, 0x68, 0x44, 0x9b, 0xa5, 0xb6, + 0xb3, 0x5e, 0xdd, 0x7c, 0xd8, 0xd5, 0x8a, 0xc1, 0x0a, 0x1a, 0xc4, 0xd4, 0xdf, 0x2e, 0xbe, 0xbe, + 0x58, 0x7b, 0x80, 0x8d, 0x0f, 0x1a, 0x40, 0xd5, 0xe7, 0x5c, 0x04, 0x61, 0x44, 0x24, 0x17, 0xcd, + 0xb2, 0x3e, 0xc5, 0x1f, 0xbd, 0xb9, 0x58, 0xfb, 0xe1, 0x4d, 0x8b, 0x5f, 0xd3, 0x6b, 0x77, 0x30, + 0x26, 0x22, 0xe8, 0xef, 0xe0, 0xf9, 0x28, 0xe8, 0x09, 0x80, 0xa0, 0x09, 0x67, 0xa9, 0x0c, 0x79, + 0xd4, 0xac, 0xe8, 0x6d, 0xd4, 0xbb, 0x39, 0xe7, 0x19, 0x25, 0x01, 0x15, 0x78, 0xce, 0x07, 0x7d, + 0x00, 0x4b, 0x89, 0xb9, 0x5a, 0x2f, 0x8c, 0x02, 0xfa, 0xaa, 0xe9, 0xb6, 0x9d, 0xf5, 0x25, 0x5c, + 0xb3, 0x60, 0x5f, 0x61, 0xe8, 0xc7, 0x00, 0x01, 0x15, 0xe1, 0x29, 0xd1, 0x61, 0x41, 0x87, 0x5d, + 0x31, 0xd9, 0xf5, 0x38, 0x63, 0xd4, 0x57, 0xb8, 0x4a, 0x11, 0xcf, 0xf9, 0xa1, 0x1e, 0x3c, 0x9c, + 0x12, 0x49, 0x45, 0x48, 0x58, 0xf8, 0xa5, 0xa1, 0x56, 0x35, 0xf5, 0x3d, 0x43, 0xdd, 0xbf, 0x6c, + 0xd4, 0xfc, 0xab, 0x8c, 0xce, 0x5f, 0x8b, 0xf0, 0x30, 0xd7, 0x5e, 0x12, 0xf3, 0x28, 0xa1, 0x68, + 0x1d, 0x4a, 0x89, 0x24, 0x32, 0x4d, 0xb4, 0xf6, 0x96, 0x37, 0xeb, 0xdd, 0xec, 0x78, 0xba, 0x03, + 0x8d, 0x63, 0x6b, 0x57, 0x9e, 0x63, 0x9d, 0xb3, 0xd6, 0xd6, 0x4d, 0x67, 0x61, 0xed, 0xe8, 0x43, + 0x58, 0x96, 0x54, 0x4c, 0xc3, 0x88, 0x30, 0x8f, 0x0a, 0xc1, 0x85, 0xd5, 0xdc, 0x52, 0x86, 0xee, + 0x2a, 0x10, 0x7d, 0x0e, 0x35, 0x41, 0x49, 0xe0, 0xc9, 0xb1, 0xe0, 0xe9, 0x68, 0x7c, 0x4f, 0xfd, + 0x55, 0x55, 0x8c, 0x63, 0x13, 0x42, 0x89, 0xf0, 0x4c, 0x84, 0x92, 0x7a, 0x6a, 0x27, 0xf7, 0x15, + 0xa1, 0x8e, 0xa0, 0x52, 0x42, 0x7d, 0x58, 0x24, 0x82, 0x46, 0x44, 0x8b, 0xb0, 0xb6, 0xfd, 0xf4, + 0xcd, 0xc5, 0xda, 0xc6, 0x28, 0x94, 0xe3, 0xf4, 0xa4, 0xeb, 0xf3, 0xe9, 0x06, 0x4d, 0x64, 0x4a, + 0xc4, 0xb9, 0xe9, 0x68, 0xd7, 0x7a, 0x5c, 0x77, 0x4b, 0x51, 0xb1, 0x89, 0x80, 0x3e, 0x84, 0x62, + 0xc0, 0xfd, 0xa4, 0x59, 0x6e, 0x17, 0xd6, 0xab, 0x9b, 0x55, 0x73, 0x6b, 0x03, 0x16, 0xfa, 0xd4, + 0x4a, 0x59, 0x9b, 0xd1, 0x33, 0x28, 0x9b, 0x0a, 0x4a, 0x9a, 0x95, 0x76, 0xe1, 0x1e, 0xbb, 0xcf, + 0xe8, 0x4a, 0x67, 0x69, 0x1a, 0x06, 0x5e, 0x4c, 0x84, 0x4c, 0x9a, 0xae, 0x5e, 0xd6, 0x56, 0xd1, + 0x8b, 0x17, 0xfd, 0x9d, 0x23, 0x05, 0xdb, 0xa5, 0x5d, 0xe5, 0xa8, 0x01, 0x25, 0xfa, 0x98, 0xf8, + 0x13, 0x1a, 0x78, 0x13, 0x7a, 0xde, 0x84, 0xdb, 0x36, 0xeb, 0x1a, 0xa7, 0xdf, 0xd0, 0xf3, 0x4e, + 0x00, 0x0d, 0xcc, 0xfd, 0x49, 0xb2, 0xb3, 0xbd, 0x43, 0x13, 0x5f, 0x84, 0xb1, 0xaa, 0x9d, 0x4f, + 0x00, 0x09, 0x05, 0x06, 0x27, 0x1e, 0x8d, 0x4e, 0xbd, 0x29, 0x9d, 0xc6, 0x52, 0x68, 0x85, 0x95, + 0x70, 0xdd, 0x5a, 0x76, 0xa3, 0xd3, 0x7d, 0x8d, 0xa3, 0xf7, 0xa1, 0x96, 0x79, 0xeb, 0x06, 0x6c, + 0x9a, 0x73, 0xd5, 0x62, 0xaa, 0x09, 0x77, 0xfe, 0xe3, 0x80, 0xdb, 0xcb, 0x9a, 0x2d, 0x7a, 0x17, + 0xca, 0x61, 0xec, 0x91, 0x20, 0x30, 0x31, 0x5d, 0x5c, 0x0a, 0xe3, 0xad, 0x20, 0x10, 0xe8, 0x27, + 0xb0, 0x64, 0x3b, 0xb4, 0x17, 0x73, 0x95, 0xf7, 0x82, 0xce, 0xa0, 0x61, 0x32, 0xb0, 0x4d, 0xfa, + 0x88, 0x0b, 0x89, 0x6b, 0xd1, 0x6c, 0x92, 0xa0, 0x01, 0x34, 0xa6, 0x24, 0x8e, 0x69, 0xe0, 0x8d, + 0x79, 0x22, 0x2d, 0xb7, 0xa0, 0xb9, 0x1f, 0x75, 0xb3, 0x77, 0x31, 0x5f, 0xbf, 0xbb, 0xaf, 0x7d, + 0x9f, 0xf1, 0x44, 0x6a, 0xfa, 0x6e, 0x24, 0xc5, 0xb9, 0x2a, 0xb7, 0x4b, 0x68, 0x6b, 0x1b, 0x56, + 0x6e, 0x72, 0x44, 0x75, 0x28, 0xa8, 0xc3, 0x75, 0x74, 0x73, 0x50, 0x43, 0xb4, 0x02, 0x8b, 0xa7, + 0x84, 0xa5, 0xd9, 0xb3, 0x64, 0x26, 0xbf, 0x58, 0xf8, 0x99, 0xd3, 0xf9, 0xc7, 0x02, 0x34, 0x7a, + 0x24, 0x96, 0xa9, 0xc8, 0x9e, 0x8b, 0xdd, 0x57, 0x12, 0x7d, 0x0c, 0x25, 0x46, 0x4e, 0x28, 0x33, + 0x45, 0x5b, 0xdd, 0x44, 0x5d, 0xf5, 0x64, 0xea, 0x2e, 0xf6, 0x5c, 0xe1, 0x61, 0x34, 0xc2, 0xd6, + 0x03, 0x3d, 0x85, 0x22, 0x8f, 0x69, 0x64, 0x8b, 0x76, 0x6d, 0x96, 0xcd, 0xd5, 0xa8, 0xdd, 0xc3, + 0x98, 0x46, 0x58, 0x3b, 0xa3, 0x3e, 0xd4, 0x12, 0x49, 0x84, 0xf4, 0x7c, 0x3e, 0x9d, 0x86, 0x52, + 0xd7, 0x6f, 0x75, 0xf3, 0x07, 0x77, 0x90, 0x07, 0xca, 0xbd, 0xa7, 0xbd, 0x71, 0x35, 0x99, 0x4d, + 0x5a, 0x9f, 0x43, 0xf1, 0xd0, 0x84, 0xcc, 0x25, 0x11, 0xe4, 0x42, 0xb1, 0xfb, 0x6f, 0xe5, 0x81, + 0xaf, 0x49, 0x09, 0x37, 0x2c, 0x6b, 0x06, 0xb5, 0x30, 0x54, 0xe7, 0x96, 0x43, 0x3d, 0x40, 0x96, + 0xee, 0xf9, 0x63, 0xea, 0x4f, 0x62, 0x1e, 0x46, 0xd2, 0x46, 0x5e, 0x99, 0xb5, 0xb3, 0x5e, 0x6e, + 0xc3, 0x0d, 0xeb, 0x3f, 0x83, 0x3a, 0xff, 0x2a, 0x00, 0xca, 0xb3, 0x32, 0xbd, 0x51, 0x9d, 0xf4, + 0x13, 0x70, 0xf3, 0x37, 0x3e, 0x3f, 0xec, 0x6b, 0x82, 0xc0, 0x33, 0x27, 0xf4, 0x2b, 0xa8, 0xf8, + 0x26, 0x4e, 0x60, 0xcf, 0xfc, 0xd1, 0xf5, 0x63, 0xcb, 0x17, 0xc8, 0xa0, 0x00, 0xe7, 0x2c, 0xb4, + 0x07, 0x30, 0x97, 0x47, 0xf1, 0xb6, 0xa3, 0x9f, 0x8b, 0x31, 0xcb, 0x6c, 0x8e, 0xd9, 0x7a, 0x09, + 0x95, 0x2c, 0x3a, 0xfa, 0x3e, 0xc0, 0x84, 0x9e, 0x7b, 0xa6, 0x6c, 0x75, 0x22, 0x35, 0xec, 0x4e, + 0xe8, 0xf9, 0x91, 0x06, 0xd4, 0x37, 0x8e, 0xea, 0x13, 0xa1, 0x7a, 0x26, 0x92, 0xcc, 0x6b, 0x41, + 0x7b, 0xd5, 0x67, 0x06, 0xe3, 0xdc, 0x3a, 0x03, 0x98, 0xad, 0x88, 0xda, 0xb0, 0xa8, 0x1e, 0x88, + 0x4c, 0x8a, 0x60, 0xa4, 0xa8, 0x10, 0x6c, 0x0c, 0xe8, 0xd7, 0x50, 0x8d, 0x39, 0x63, 0x9e, 0xa0, + 0x49, 0xca, 0xa4, 0x0e, 0xbb, 0x7c, 0x77, 0x42, 0x47, 0x9c, 0x31, 0xac, 0xbd, 0x31, 0xc4, 0xf9, + 0xb8, 0x73, 0x00, 0x30, 0xb3, 0xa0, 0x2a, 0x94, 0xfb, 0x07, 0x2f, 0xb7, 0x9e, 0xf7, 0x77, 0xea, + 0x0f, 0x90, 0x0b, 0x8b, 0x78, 0x77, 0x6b, 0xe7, 0xb7, 0x75, 0x07, 0x2d, 0x81, 0x7b, 0x70, 0x78, + 0xec, 0x99, 0xe9, 0x02, 0xaa, 0x41, 0xa5, 0x77, 0x78, 0xf8, 0xdc, 0x3b, 0xdc, 0xdb, 0xab, 0x17, + 0x14, 0x09, 0xef, 0x0e, 0x8e, 0xb7, 0xf0, 0x71, 0xbd, 0xd8, 0xf9, 0xa7, 0x03, 0xf5, 0x1d, 0xf5, + 0xc6, 0xde, 0xb7, 0xb6, 0x36, 0x2f, 0xd5, 0xd6, 0x6a, 0x9e, 0xd2, 0xd5, 0xa0, 0x73, 0xa5, 0xd5, + 0x3a, 0xb3, 0xf5, 0xf0, 0x08, 0x96, 0x93, 0x2f, 0x98, 0x7a, 0xab, 0x4e, 0x87, 0x89, 0x97, 0x8a, + 0xd0, 0xb6, 0xb2, 0x9a, 0x41, 0x5f, 0x0e, 0x93, 0x17, 0x22, 0xbc, 0xa5, 0x6a, 0x16, 0xee, 0x51, + 0x35, 0x9d, 0x7f, 0x17, 0xa0, 0x91, 0x6d, 0xec, 0x9b, 0x08, 0xfc, 0xe7, 0x50, 0x52, 0x89, 0xe4, + 0xf2, 0x7e, 0xff, 0x5a, 0xda, 0xb3, 0x8b, 0x3c, 0xd4, 0x8e, 0xd8, 0x12, 0xd0, 0x36, 0xb8, 0x71, + 0x7a, 0xc2, 0xc2, 0x64, 0x4c, 0x03, 0xdb, 0x53, 0x1e, 0xdd, 0xc1, 0x3e, 0xca, 0x7c, 0xf1, 0x8c, + 0x86, 0x7e, 0x09, 0xe5, 0x21, 0x4b, 0x75, 0x04, 0x53, 0x1a, 0x9d, 0x3b, 0x22, 0xec, 0x19, 0x4f, + 0x9c, 0x51, 0x5a, 0xfb, 0x50, 0x32, 0x7b, 0xfa, 0x56, 0xba, 0x46, 0x4b, 0x82, 0x9b, 0x6f, 0x52, + 0xfd, 0xc0, 0x98, 0x92, 0x57, 0x9e, 0xcf, 0xb8, 0x3f, 0xb1, 0x6f, 0x5d, 0x65, 0x4a, 0x5e, 0xf5, + 0xd4, 0xfc, 0x4a, 0x01, 0x2e, 0xbc, 0x55, 0x01, 0x16, 0x6e, 0x29, 0xc0, 0xc7, 0x50, 0xb6, 0x89, + 0xfd, 0xff, 0xea, 0xeb, 0xfc, 0xd9, 0x81, 0xef, 0xcc, 0xbe, 0x0e, 0xef, 0xab, 0xf4, 0x9f, 0x5e, + 0x52, 0xfa, 0x07, 0xf9, 0x91, 0xdf, 0x18, 0x79, 0x5e, 0xee, 0xdf, 0x7e, 0xfb, 0xef, 0xfc, 0xc9, + 0x81, 0x77, 0x2e, 0xad, 0xfb, 0x4d, 0xd4, 0xbc, 0x35, 0x93, 0x93, 0xc9, 0xed, 0xa3, 0x9b, 0x73, + 0xbb, 0x43, 0x53, 0x5f, 0xe7, 0x3a, 0x36, 0x3f, 0x83, 0x8a, 0xfd, 0x04, 0x17, 0xe8, 0x53, 0x28, + 0xdb, 0x31, 0x7a, 0x37, 0x5f, 0xf5, 0xf2, 0x8f, 0xc3, 0x56, 0xf3, 0xba, 0xc1, 0x6c, 0xe5, 0x89, + 0xb3, 0xfd, 0xe9, 0xeb, 0xbf, 0xad, 0x3e, 0x78, 0xfd, 0xd5, 0xaa, 0xf3, 0x97, 0xaf, 0x56, 0x9d, + 0x3f, 0xfc, 0x7d, 0xd5, 0xf9, 0xdd, 0x27, 0x6f, 0xf5, 0x75, 0x6a, 0x23, 0x9e, 0x94, 0x34, 0xf4, + 0xf4, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xe8, 0x26, 0xf2, 0x60, 0x0f, 0x10, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -1623,9 +1709,9 @@ func (m *CaptureRequestExt) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } - if m.Acknowledge != nil { + if m.StartCommit != nil { { - size, err := m.Acknowledge.MarshalToSizedBuffer(dAtA[:i]) + size, err := m.StartCommit.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -1701,7 +1787,7 @@ func (m *CaptureRequestExt_Open) MarshalToSizedBuffer(dAtA []byte) (int, error) return len(dAtA) - i, nil } -func (m *CaptureRequestExt_Acknowledge) Marshal() (dAtA []byte, err error) { +func (m *CaptureRequestExt_StartCommit) Marshal() (dAtA []byte, err error) { size := m.ProtoSize() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -1711,12 +1797,12 @@ func (m *CaptureRequestExt_Acknowledge) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *CaptureRequestExt_Acknowledge) MarshalTo(dAtA []byte) (int, error) { +func (m *CaptureRequestExt_StartCommit) MarshalTo(dAtA []byte) (int, error) { size := m.ProtoSize() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *CaptureRequestExt_Acknowledge) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *CaptureRequestExt_StartCommit) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -1725,9 +1811,9 @@ func (m *CaptureRequestExt_Acknowledge) MarshalToSizedBuffer(dAtA []byte) (int, i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } - if m.StartCommitRuntimeCheckpoint != nil { + if m.RuntimeCheckpoint != nil { { - size, err := m.StartCommitRuntimeCheckpoint.MarshalToSizedBuffer(dAtA[:i]) + size, err := m.RuntimeCheckpoint.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -1868,6 +1954,11 @@ func (m *CaptureResponseExt_Checkpoint) MarshalToSizedBuffer(dAtA []byte) (int, i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.PollResult != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.PollResult)) + i-- + dAtA[i] = 0x10 + } if m.Stats != nil { { size, err := m.Stats.MarshalToSizedBuffer(dAtA[:i]) @@ -2294,6 +2385,18 @@ func (m *MaterializeResponseExt) MarshalToSizedBuffer(dAtA []byte) (int, error) i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.Flushed != nil { + { + size, err := m.Flushed.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } if m.Container != nil { { size, err := m.Container.MarshalToSizedBuffer(dAtA[:i]) @@ -2309,6 +2412,45 @@ func (m *MaterializeResponseExt) MarshalToSizedBuffer(dAtA []byte) (int, error) return len(dAtA) - i, nil } +func (m *MaterializeResponseExt_Flushed) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *MaterializeResponseExt_Flushed) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *MaterializeResponseExt_Flushed) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Stats != nil { + { + size, err := m.Stats.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func encodeVarintRuntime(dAtA []byte, offset int, v uint64) int { offset -= sovRuntime(v) base := offset @@ -2521,8 +2663,8 @@ func (m *CaptureRequestExt) ProtoSize() (n int) { l = m.Open.ProtoSize() n += 1 + l + sovRuntime(uint64(l)) } - if m.Acknowledge != nil { - l = m.Acknowledge.ProtoSize() + if m.StartCommit != nil { + l = m.StartCommit.ProtoSize() n += 1 + l + sovRuntime(uint64(l)) } if m.XXX_unrecognized != nil { @@ -2547,14 +2689,14 @@ func (m *CaptureRequestExt_Open) ProtoSize() (n int) { return n } -func (m *CaptureRequestExt_Acknowledge) ProtoSize() (n int) { +func (m *CaptureRequestExt_StartCommit) ProtoSize() (n int) { if m == nil { return 0 } var l int _ = l - if m.StartCommitRuntimeCheckpoint != nil { - l = m.StartCommitRuntimeCheckpoint.ProtoSize() + if m.RuntimeCheckpoint != nil { + l = m.RuntimeCheckpoint.ProtoSize() n += 1 + l + sovRuntime(uint64(l)) } if m.XXX_unrecognized != nil { @@ -2617,6 +2759,9 @@ func (m *CaptureResponseExt_Checkpoint) ProtoSize() (n int) { l = m.Stats.ProtoSize() n += 1 + l + sovRuntime(uint64(l)) } + if m.PollResult != 0 { + n += 1 + sovRuntime(uint64(m.PollResult)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -2792,6 +2937,26 @@ func (m *MaterializeResponseExt) ProtoSize() (n int) { l = m.Container.ProtoSize() n += 1 + l + sovRuntime(uint64(l)) } + if m.Flushed != nil { + l = m.Flushed.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *MaterializeResponseExt_Flushed) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Stats != nil { + l = m.Stats.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -4169,7 +4334,7 @@ func (m *CaptureRequestExt) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Acknowledge", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartCommit", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -4196,10 +4361,10 @@ func (m *CaptureRequestExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Acknowledge == nil { - m.Acknowledge = &CaptureRequestExt_Acknowledge{} + if m.StartCommit == nil { + m.StartCommit = &CaptureRequestExt_StartCommit{} } - if err := m.Acknowledge.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.StartCommit.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -4312,7 +4477,7 @@ func (m *CaptureRequestExt_Open) Unmarshal(dAtA []byte) error { } return nil } -func (m *CaptureRequestExt_Acknowledge) Unmarshal(dAtA []byte) error { +func (m *CaptureRequestExt_StartCommit) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -4335,15 +4500,15 @@ func (m *CaptureRequestExt_Acknowledge) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Acknowledge: wiretype end group for non-group") + return fmt.Errorf("proto: StartCommit: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Acknowledge: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StartCommit: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field StartCommitRuntimeCheckpoint", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RuntimeCheckpoint", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -4370,10 +4535,10 @@ func (m *CaptureRequestExt_Acknowledge) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.StartCommitRuntimeCheckpoint == nil { - m.StartCommitRuntimeCheckpoint = &protocol1.Checkpoint{} + if m.RuntimeCheckpoint == nil { + m.RuntimeCheckpoint = &protocol1.Checkpoint{} } - if err := m.StartCommitRuntimeCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.RuntimeCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -4742,6 +4907,25 @@ func (m *CaptureResponseExt_Checkpoint) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PollResult", wireType) + } + m.PollResult = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PollResult |= CaptureResponseExt_PollResult(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipRuntime(dAtA[iNdEx:]) @@ -5779,6 +5963,129 @@ func (m *MaterializeResponseExt) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Flushed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Flushed == nil { + m.Flushed = &MaterializeResponseExt_Flushed{} + } + if err := m.Flushed.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthRuntime + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *MaterializeResponseExt_Flushed) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Flushed: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Flushed: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Stats", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Stats == nil { + m.Stats = &ops.Stats{} + } + if err := m.Stats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRuntime(dAtA[iNdEx:])