diff --git a/AGENTS.md b/AGENTS.md new file mode 100644 index 00000000000..9a4daaf374c --- /dev/null +++ b/AGENTS.md @@ -0,0 +1,133 @@ +# CLAUDE.md + +Estuary is a real-time data platform with: +- Control plane: user-facing catalog management APIs +- Data planes: distributed runtime execution +- Connectors: OCI images integrating external systems + +This repo lives at `https://github.com/estuary/flow` + +## Repository Overview + +Estuary is built with: +- **Rust** (primary language) + - Third-party sources under `~/.cargo/registry/src/` +- **Go** - integration glue with the Gazette consumer framework + - Third-party sources under `~/go/pkg/mod/` +- **Protobuf** - communication between control plane, data planes, and connectors +- **Supabase** - migrations are under `supabase/migrations/` + - pgTAP tests under `supabase/tests/` +- **Docs** - external user-facing product documentation under `site/` (Docusaurus) + +## Essential Commands + +### Build & Test + +Use regular `cargo` and `go` tools to build and test crates. + +```bash +# libsqlite3 tag is required for `bindings` and `flowctl-go` packages. +go build -tags libsqlite3 ./go/bindings + +# Regenerate checked-in protobuf (required after .proto changes) +mise run build:go-protobufs +mise run build:rust-protobufs + +# Run pgTAP SQL Tests +mise run ci:sql-tap + +# E2E tests over derivation examples (SLOW) +mise run ci:catalog-test +``` + +### Development + +A development Supabase instance is available: +```bash +# Reset with current migrations as needed +supabase db reset + +# Interact directly with dev DB +psql postgresql://postgres:postgres@localhost:5432/postgres -c 'SELECT 1;' +``` + +## Architecture Overview + +### Core Concepts + +Users interact with the control plane to manage a catalog of: +- **Captures**: tasks which capture from a user endpoint into target collections +- **Collections**: collections of data with enforced JSON Schema +- **Derivations**: both a collection and a task - the task builds its collection through transformation of other collections +- **Materializations**: tasks which maintain materialized views of source collections in an endpoint +- **Tests**: fixtures of source collection inputs and expected derivation outputs + +Collections and tasks have a declarative (JSON/YAML) **model**. +Users refine model changes in **drafts**, which are **published** +to the control plane for verification and testing. +The control plane compiles the user's catalog model into +**built specs** that have extra specifics required by the runtime, +and activates specs into their associated data plane. + +### Control-plane components +- **Supabase**: catalog and platform config DB +- **Agent**: APIs and background automation +- **Data-plane controller**: provisions data planes + +### Data-plane components +- **Gazette**: brokers serve the journals that back collections +- **Reactors**: runtime written to Gazette consumer framework; + executes tasks and runs connectors as sidecars over gRPC +- **Etcd**: config for gazette and reactors + +### Protocols + +- `go/protocols/flow/flow.proto` - core types and built specs +- `go/protocols/capture/capture.proto` - protocol for capture tasks +- `go/protocols/derive/derive.proto` - for derivation tasks +- `go/protocols/materialize/materialize.proto` - for materialization tasks + +## README.md + +Every crate/module should have a README.md with essential context: +- Purpose and fit within the project +- Key types and entry points +- Brief architecture and non-obvious details + +A README.md is ONLY a roadmap for expert developers, +orienting them where to look next. + +Keep READMEs current - update with code changes. + +## Development Guidelines + +### Implementation +- Use `var myVar = ...` in Go. Do NOT use `myVar := ...` (unless required due to shadowing) +- Write comments that document "why" - rationale, broader context, and non-obvious detail +- Do NOT write comments which describe the obvious behavior of code. + Don't write `// Get credentials` before a call `getCredentials()` +- Use early-return over nested conditionals +- Use at least one level of name qualification for third-party types and functions. + For example, `axum::Router::new()` instead of `use axum::Router; Router::new()`. + Types / functions should be unqualified ONLY if they're in the current module. +- Prefer pure functions that take and act over POD states. + AVOID structures that mix complex state and impl behaviors, where possible. + The exception is state machines: structs and enums that encapsulate fine-grain + POD state into higher-order transitions that are easier to reason about. + DO seek to decompose problems into state machines. +- Avoid trivial impl routines which could be inlined by the caller. + Indirection is harder to read; routines must buy us something. +- Decompose IO and POD processing into separate routines where possible. + Routines should gravitate towards IO or processing, and not mix both. + +### Testing +- Prefer snapshots over fine-grain assertions (`insta` / `cupaloy`) + +### Errors +- Wrap errors with context (`anyhow::Context` / `fmt.Errorf`) +- Return errors up the stack rather than logging +- Panic on impossible states (do NOT add spurious error handling) + +### Logging +- Structured logging with context (`tracing` / `logrus`) +- Avoid verbose logging in hot paths diff --git a/CLAUDE.md b/CLAUDE.md index 7c3d9a2157a..9a4daaf374c 100644 --- a/CLAUDE.md +++ b/CLAUDE.md @@ -106,11 +106,19 @@ Keep READMEs current - update with code changes. - Write comments that document "why" - rationale, broader context, and non-obvious detail - Do NOT write comments which describe the obvious behavior of code. Don't write `// Get credentials` before a call `getCredentials()` -- Prefer functional approaches. Try to avoid mutation. - Use early-return over nested conditionals - Use at least one level of name qualification for third-party types and functions. For example, `axum::Router::new()` instead of `use axum::Router; Router::new()`. Types / functions should be unqualified ONLY if they're in the current module. +- Prefer pure functions that take and act over POD states. + AVOID structures that mix complex state and impl behaviors, where possible. + The exception is state machines: structs and enums that encapsulate fine-grain + POD state into higher-order transitions that are easier to reason about. + DO seek to decompose problems into state machines. +- Avoid trivial impl routines which could be inlined by the caller. + Indirection is harder to read; routines must buy us something. +- Decompose IO and POD processing into separate routines where possible. + Routines should gravitate towards IO or processing, and not mix both. ### Testing - Prefer snapshots over fine-grain assertions (`insta` / `cupaloy`) diff --git a/Cargo.lock b/Cargo.lock index ca83e959596..f1c5d715e39 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3348,9 +3348,11 @@ dependencies = [ "proto-flow", "proto-gazette", "proto-grpc", + "publisher", "rand 0.9.2", "reqwest 0.12.23", "runtime", + "runtime-next", "rusqlite", "rustls 0.23.32", "rustyline", @@ -7080,6 +7082,71 @@ dependencies = [ "zeroize", ] +[[package]] +name = "runtime-next" +version = "0.0.0" +dependencies = [ + "anyhow", + "assemble", + "async-process", + "aws-config", + "aws-credential-types", + "aws-sdk-sts", + "axum", + "base64 0.22.1", + "bytes", + "chrono", + "connector-init", + "coroutines", + "dekaf-connector", + "doc", + "extractors", + "flow-client-next", + "futures", + "gazette", + "handlebars 6.3.2", + "iam-auth", + "insta", + "json", + "json-patch", + "jsonwebtoken", + "labels", + "librocksdb-sys", + "locate-bin", + "models", + "ops", + "pbjson-types", + "prost", + "proto-flow", + "proto-gazette", + "proto-grpc", + "publisher", + "quickcheck", + "rand 0.9.2", + "reqwest 0.12.23", + "rocksdb", + "serde", + "serde_json", + "shuffle", + "simd-doc", + "tables", + "tempfile", + "thiserror 2.0.17", + "time", + "tokens", + "tokio", + "tokio-stream", + "tonic", + "tracing", + "tracing-subscriber", + "tuple", + "unseal", + "url", + "uuid", + "xxhash-rust", + "zeroize", +] + [[package]] name = "rusqlite" version = "0.32.1" diff --git a/crates/doc/src/combine/mod.rs b/crates/doc/src/combine/mod.rs index 5a8134011e8..7d2a0cada57 100644 --- a/crates/doc/src/combine/mod.rs +++ b/crates/doc/src/combine/mod.rs @@ -159,6 +159,13 @@ impl Accumulator { Ok(self.memtable.as_ref().unwrap()) } + /// Segment ranges of the spill file written by this Accumulator so far. + /// Ranges are append-only, non-overlapping, and monotonic within an + /// Accumulator's lifetime. + pub fn ranges(&self) -> &[std::ops::Range] { + self.spill.segment_ranges() + } + /// Map this combine Accumulator into a Drainer, which will drain directly /// from the inner MemTable (if no spill occurred) or from an inner SpillDrainer. pub fn into_drainer(self) -> Result { diff --git a/crates/flow-client-next/src/workflows/user_collection_auth.rs b/crates/flow-client-next/src/workflows/user_collection_auth.rs index 80de14a147d..733fb672458 100644 --- a/crates/flow-client-next/src/workflows/user_collection_auth.rs +++ b/crates/flow-client-next/src/workflows/user_collection_auth.rs @@ -44,8 +44,6 @@ pub fn new_journal_client( /// /// The partition template prefix of certain legacy collections were created /// without a "/{generation_id}/" suffix -- these are supported as well. -/// -/// Panics if the partition template prefix is not in the expected format. pub fn new_journal_client_factory( api_client: crate::rest::Client, capability: models::Capability, diff --git a/crates/flowctl/Cargo.toml b/crates/flowctl/Cargo.toml index 1b9c2891e0b..6b997abb0c3 100644 --- a/crates/flowctl/Cargo.toml +++ b/crates/flowctl/Cargo.toml @@ -33,7 +33,9 @@ proto-grpc = { path = "../proto-grpc", features = [ "capture_client", "shuffle_client", ] } +publisher = { path = "../publisher" } runtime = { path = "../runtime" } +runtime-next = { path = "../runtime-next" } shuffle = { path = "../shuffle" } simd-doc = { path = "../simd-doc" } sources = { path = "../sources" } diff --git a/crates/flowctl/src/raw/materialize_fixture.rs b/crates/flowctl/src/raw/materialize_fixture.rs index 2b03dae0451..ac5bfb9fa3d 100644 --- a/crates/flowctl/src/raw/materialize_fixture.rs +++ b/crates/flowctl/src/raw/materialize_fixture.rs @@ -76,7 +76,7 @@ pub async fn do_materialize_fixture( ..Default::default() }); emit(Request { - acknowledge: Some(request::Acknowledge {}), + acknowledge: Some(request::Acknowledge::default()), ..Default::default() }); @@ -147,7 +147,7 @@ pub async fn do_materialize_fixture( emit(load) } emit(Request { - flush: Some(request::Flush {}), + flush: Some(request::Flush::default()), ..Default::default() }); for store in stores { @@ -171,7 +171,7 @@ pub async fn do_materialize_fixture( ..Default::default() }); emit(Request { - acknowledge: Some(request::Acknowledge {}), + acknowledge: Some(request::Acknowledge::default()), ..Default::default() }); } diff --git a/crates/flowctl/src/raw/mod.rs b/crates/flowctl/src/raw/mod.rs index be5ffebe20e..0f3d1f3b82b 100644 --- a/crates/flowctl/src/raw/mod.rs +++ b/crates/flowctl/src/raw/mod.rs @@ -15,8 +15,8 @@ mod alerts; mod discover; mod materialize_fixture; mod oauth; +mod preview_next; mod shards; -mod shuffle; mod spec; #[derive(Debug, clap::Args)] @@ -75,9 +75,8 @@ pub enum Command { /// Print environment variables for working with a given data-plane /// and prefix using Gazette's `gazctl`. GazctlEnv(GazctlEnv), - - #[clap(hide = true)] - Shuffle(shuffle::Shuffle), + /// Locally run and preview a materialization using the V2 runtime. + PreviewNext(preview_next::Preview), } #[derive(Debug, clap::Args)] @@ -241,7 +240,7 @@ impl Advanced { Command::BearerLogs(bearer_logs) => bearer_logs.run(ctx).await, Command::ListShards(selector) => shards::do_list_shards(ctx, selector).await, Command::GazctlEnv(gazctl_env) => gazctl_env.run(ctx).await, - Command::Shuffle(shuffle) => shuffle.run(ctx).await, + Command::PreviewNext(preview) => preview.run(ctx).await, } } } diff --git a/crates/flowctl/src/raw/preview_next/driver.rs b/crates/flowctl/src/raw/preview_next/driver.rs new file mode 100644 index 00000000000..93188122a71 --- /dev/null +++ b/crates/flowctl/src/raw/preview_next/driver.rs @@ -0,0 +1,222 @@ +//! Run-scoped driver: spawns N shard tasks via +//! `runtime_next::shard::Service::spawn_materialize`, synthesizing the +//! SessionLoop/Join/Task envelopes the controller (Go in production) would +//! normally send. + +use crate::raw::preview_next::services::Run; +use prost::Message; +use proto_flow::{flow, runtime as cruntime}; +use runtime_next::proto; +use tokio::sync::mpsc; +use tokio_stream::wrappers::UnboundedReceiverStream; +use tokio_util::sync::CancellationToken; + +/// Run preview sessions against the prepared topology. Sessions are +/// numbered `1..` for log context, and all run over the same per-shard +/// SessionLoop streams. +pub async fn run_sessions( + run: &Run, + spec: &flow::MaterializationSpec, + session_targets: Vec, + stop_token: CancellationToken, +) -> anyhow::Result<()> { + let join_shards = crate::raw::preview_next::shards::build_join_shards(run.n_shards, spec)?; + + let mut handles = Vec::with_capacity(run.n_shards as usize); + for i in 0..run.n_shards { + let run_handle = RunHandle { + peer_endpoint: run.peer_endpoint.clone(), + shuffle_log_dir: run.shuffle_log_dir.clone(), + rocksdb_path: run.rocksdb_path.clone(), + network: run.network.clone(), + log_handler: run.log_handler, + }; + let spec = spec.clone(); + let join_shards = join_shards.clone(); + let session_targets = session_targets.clone(); + let stop_token = stop_token.clone(); + + handles.push(tokio::spawn(async move { + drive_one_shard( + run_handle, + spec, + i, + join_shards, + session_targets, + stop_token, + ) + .await + })); + } + + // Await all shard drivers. The first error surfaces; remaining drivers + // observe their request channel dropping (when their handle drops) and + // tear down naturally. + let mut first_err: Option = None; + for handle in handles { + match handle.await { + Ok(Ok(())) => (), + Ok(Err(e)) if first_err.is_none() => first_err = Some(e), + Ok(Err(e)) => tracing::warn!(error = ?e, "secondary shard driver error"), + Err(panic) => { + if first_err.is_none() { + first_err = Some(anyhow::anyhow!("driver panic: {panic}")); + } else { + tracing::warn!(?panic, "secondary driver panic"); + } + } + } + } + if let Some(e) = first_err { + return Err(e); + } + Ok(()) +} + +/// `Run` fields a single shard driver needs. Cheaper to clone than `&Run` +/// so we can hand it into a spawned task without lifetime gymnastics. +struct RunHandle { + peer_endpoint: String, + shuffle_log_dir: String, + rocksdb_path: String, + network: String, + log_handler: fn(&::ops::Log), +} + +async fn drive_one_shard( + run: RunHandle, + spec: flow::MaterializationSpec, + shard_index: u32, + join_shards: Vec, + session_targets: Vec, + stop_token: CancellationToken, +) -> anyhow::Result<()> { + let (request_tx, request_rx) = mpsc::unbounded_channel::>(); + + let task_name = format!("preview-shard-{shard_index:03}"); + + let publisher_factory: gazette::journal::ClientFactory = std::sync::Arc::new({ + move |_authz_sub: String, _authz_obj: String| -> gazette::journal::Client { + unreachable!("live Publisher is not used by preview ({_authz_sub}, {_authz_obj})") + } + }); + + let shard_svc = runtime_next::shard::Service::new( + cruntime::Plane::Local, + run.network.clone(), + run.log_handler, + None, + task_name, + publisher_factory, + ); + + let mut response_rx = shard_svc.spawn_materialize(UnboundedReceiverStream::new(request_rx)); + let spec_bytes: bytes::Bytes = spec.encode_to_vec().into(); + + // Open the SessionLoop once. `runtime-next` opens RocksDB here and keeps + // the handle live across the repeated Join/Task sessions below. + let rocksdb_descriptor = if shard_index == 0 { + Some(cruntime::RocksDbDescriptor { + rocksdb_path: run.rocksdb_path.clone(), + rocksdb_env_memptr: 0, + }) + } else { + None + }; + request_tx + .send(Ok(proto::Materialize { + session_loop: Some(proto::SessionLoop { rocksdb_descriptor }), + ..Default::default() + })) + .map_err(|_| anyhow::anyhow!("serve task closed before SessionLoop"))?; + + for (idx, target_txns) in session_targets.into_iter().enumerate() { + if stop_token.is_cancelled() { + break; + } + let session_index = idx + 1; + + request_tx + .send(Ok(proto::Materialize { + join: Some(proto::Join { + etcd_mod_revision: session_index as i64, + shards: join_shards.clone(), + shard_index, + shuffle_directory: run.shuffle_log_dir.clone(), + shuffle_endpoint: run.peer_endpoint.clone(), + leader_endpoint: run.peer_endpoint.clone(), + }), + ..Default::default() + })) + .map_err(|_| anyhow::anyhow!("serve task closed before Join"))?; + + if shard_index == 0 { + tracing::info!( + session = session_index, + target_txns, + "starting preview session", + ); + + request_tx + .send(Ok(proto::Materialize { + task: Some(proto::Task { + spec: spec_bytes.clone(), + ops_stats_journal: String::new(), + ops_stats_spec: None, + preview: true, + max_transactions: target_txns, + }), + ..Default::default() + })) + .map_err(|_| anyhow::anyhow!("serve task closed before Task"))?; + } + + drive_session_responses(&request_tx, &mut response_rx, session_index, &stop_token).await?; + } + + drop(request_tx); + while let Some(msg) = response_rx.recv().await { + let _msg = msg.map_err(runtime_next::status_to_anyhow)?; + } + + Ok(()) +} + +async fn drive_session_responses( + request_tx: &mpsc::UnboundedSender>, + response_rx: &mut mpsc::UnboundedReceiver>, + session_index: usize, + stop_token: &CancellationToken, +) -> anyhow::Result<()> { + let verify = runtime_next::verify("Materialize", "Joined, Opened, or Stopped", "shard"); + + let mut requested_stop = false; + loop { + tokio::select! { + biased; + + _ = stop_token.cancelled(), if !requested_stop => { + requested_stop = true; + _ = request_tx + .send(Ok(proto::Materialize { + stop: Some(proto::Stop {}), + ..Default::default() + })); + } + msg = response_rx.recv() => { + let msg = verify.not_eof(msg)?; + + if let Some(proto::Joined { max_etcd_revision }) = msg.joined { + tracing::debug!(session_index, max_etcd_revision, "session joined"); + } else if let Some(proto::materialize::Opened { container, .. }) = &msg.opened { + tracing::debug!(session_index, ?container, "session opened"); + } else if let Some(proto::Stopped {}) = msg.stopped { + tracing::debug!(session_index, "session stopped"); + return Ok(()); + } else { + return Err(verify.fail_msg(msg)); + } + } + } + } +} diff --git a/crates/flowctl/src/raw/preview_next/mod.rs b/crates/flowctl/src/raw/preview_next/mod.rs new file mode 100644 index 00000000000..1acb919d35e --- /dev/null +++ b/crates/flowctl/src/raw/preview_next/mod.rs @@ -0,0 +1,207 @@ +//! `flowctl preview` on top of the runtime-next + shuffle stack. +//! +//! Spawns one in-process tonic server hosting both `runtime_next::Service` +//! and `shuffle::Service` on a single ephemeral 127.0.0.1 port, then runs +//! N synthetic shards as tokio tasks each driving one long-lived SessionLoop. +//! Source documents come from real Gazette journal reads (authed via the +//! user's flowctl token); endpoint mutations go to the connector container +//! as in production. +use crate::local_specs; +use anyhow::Context; + +mod driver; +mod services; +mod shards; + +#[derive(Debug, clap::Args)] +#[clap(rename_all = "kebab-case")] +pub struct Preview { + /// Path or URL to a Flow specification file. + #[clap(long)] + source: String, + /// Name of the task to preview within the Flow specification file. + /// Required if there are multiple tasks in --source specifications. + #[clap(long)] + name: Option, + /// Number of synthetic shards to drive in parallel. Default 1. + #[clap(long, default_value = "1")] + shards: u32, + /// How long should preview run before gracefully stopping? + /// The default is no timeout. Note that the task may finish active + /// transaction activity even after this timeout is reached. + #[clap(long)] + timeout: Option, + /// How many connector sessions should be run, and what is the target number + /// of transactions for each session? + /// + /// Sessions are specified as a comma-separated list of the number of + /// transactions for the ordered session. For a given session, a value less + /// than zero means "unlimited transactions", though the session will still + /// end upon a connector exit / EOF (when a capture) or timeout. + /// + /// For example, to run three sessions consisting of two transactions, + /// then one transaction, and then unlimited transactions, + /// use argument `--sessions 2,1,-1`. + /// + /// A session is stopped and the next started upon reaching the target number + /// of transactions, or upon a timeout, or if the connector exits. + /// + /// The default is a single session with an unbounded number of transactions. + #[clap(long, value_parser, value_delimiter = ',')] + sessions: Option>, + /// Docker network to run connector images. + #[clap(long, default_value = "bridge")] + network: String, + /// Output task logs in JSON format to stderr. + #[clap(long, action)] + log_json: bool, +} + +impl Preview { + pub async fn run(&self, ctx: &mut crate::CliContext) -> anyhow::Result<()> { + let Self { + source, + name, + shards, + timeout, + sessions, + network, + log_json, + } = self; + + let source_url = build::arg_source_to_url(source, false)?; + + let log_handler: fn(&::ops::Log) = if *log_json { + ::ops::stderr_log_handler + } else { + ::ops::tracing_log_handler + }; + + let (_sources, _live, validations) = local_specs::load_and_validate_full( + &ctx.client, + source_url.as_str(), + network, + log_handler, + ) + .await?; + + let spec = resolve_materialization(&validations, name.as_deref())?; + + let timeout = timeout.map(|i| i.into()); + + let session_targets: Vec = if let Some(s) = sessions { + s.iter() + .map(|i| { + if *i < 0 { + Ok(0) + } else { + u32::try_from(*i).context("--sessions values must fit in uint32") + } + }) + .collect::>()? + } else { + vec![0] + }; + + let run = services::Run::start(ctx, network.clone(), *log_json, *shards).await?; + let stop_token = tokio_util::sync::CancellationToken::new(); + + let session_loop = driver::run_sessions(&run, &spec, session_targets, stop_token.clone()); + tokio::pin!(session_loop); + + if let Some(timeout) = timeout { + tokio::select! { + result = &mut session_loop => result?, + _ = tokio::signal::ctrl_c() => { + tracing::info!("Ctrl-C received; aborting in-flight session"); + } + _ = tokio::time::sleep(timeout) => { + tracing::info!(?timeout, "preview --timeout reached; stopping active session"); + stop_token.cancel(); + session_loop.await?; + } + } + } else { + tokio::select! { + result = &mut session_loop => result?, + _ = tokio::signal::ctrl_c() => { + tracing::info!("Ctrl-C received; aborting in-flight session"); + } + } + } + + // `run` drops here, aborting the tonic server and removing the + // RocksDB / shuffle-log tempdirs. + Ok(()) + } +} + +fn resolve_materialization( + validations: &tables::Validations, + name: Option<&str>, +) -> anyhow::Result { + let num_tasks = validations.built_captures.len() + + validations.built_materializations.len() + + validations + .built_collections + .iter() + .filter(|c| { + c.spec + .as_ref() + .map(|s| s.derivation.is_some()) + .unwrap_or_default() + }) + .count(); + + if num_tasks == 0 { + anyhow::bail!( + "sourced specification files do not contain any tasks (captures, derivations, or materializations)", + ); + } + if num_tasks > 1 && name.is_none() { + anyhow::bail!( + "sourced specification files contain multiple tasks; use --name to identify a materialization", + ); + } + + // Fail fast if the named target is a capture or a derivation. + if let Some(target) = name { + if validations + .built_captures + .iter() + .any(|c| c.capture.as_str() == target) + { + anyhow::bail!( + "runtime-next preview supports materializations only; capture and derivation will be re-added before upstream merge", + ); + } + if validations.built_collections.iter().any(|c| { + c.collection.as_str() == target + && c.spec + .as_ref() + .map(|s| s.derivation.is_some()) + .unwrap_or(false) + }) { + anyhow::bail!( + "runtime-next preview supports materializations only; capture and derivation will be re-added before upstream merge", + ); + } + } + + for row in validations.built_materializations.iter() { + if let Some(target) = name { + if row.materialization.as_str() != target { + continue; + } + } + let Some(spec) = &row.spec else { continue }; + return Ok(spec.clone()); + } + + if let Some(target) = name { + anyhow::bail!("could not find materialization {target}"); + } + anyhow::bail!( + "no materialization in source; runtime-next preview supports materializations only", + ); +} diff --git a/crates/flowctl/src/raw/preview_next/services.rs b/crates/flowctl/src/raw/preview_next/services.rs new file mode 100644 index 00000000000..eeed25d8234 --- /dev/null +++ b/crates/flowctl/src/raw/preview_next/services.rs @@ -0,0 +1,105 @@ +//! Run-scoped state for `flowctl preview`: one tonic server hosting both +//! `runtime_next::Service` and `shuffle::Service` on a single ephemeral +//! 127.0.0.1 port, plus tempdirs for shard-zero RocksDB and shuffle log +//! segments. The preview driver keeps SessionLoop streams open across +//! `--sessions` iterations so RocksDB is reused without closing. +use anyhow::Context; +use tokio_stream::wrappers::TcpListenerStream; + +/// Run-scoped resources for one `flowctl preview` invocation. Field order +/// matters for `Drop`: the server task aborts before the tempdirs disappear +/// out from under any in-flight handler. +pub struct Run { + _server_task: tokio::task::JoinHandle>, + _rocksdb_tmp: tempfile::TempDir, + _shuffle_log_tmp: tempfile::TempDir, + pub peer_endpoint: String, + pub log_handler: fn(&::ops::Log), + pub network: String, + pub rocksdb_path: String, + pub shuffle_log_dir: String, + pub n_shards: u32, +} + +impl Run { + pub async fn start( + ctx: &mut crate::CliContext, + network: String, + log_json: bool, + n_shards: u32, + ) -> anyhow::Result { + let log_handler: fn(&::ops::Log) = if log_json { + ::ops::stderr_log_handler + } else { + ::ops::tracing_log_handler + }; + + let access_token = ctx + .config + .user_access_token + .clone() + .context("you must be logged in to preview. Try `flowctl auth login`")?; + + // TODO(johnny): handle refresh rotation. + let user_tokens = tokens::fixed(Ok(flow_client_next::user_auth::UserToken { + access_token: Some(access_token), + refresh_token: None, + })); + + let listener = tokio::net::TcpListener::bind("127.0.0.1:0") + .await + .context("binding ephemeral preview listener")?; + let local_addr = listener.local_addr()?; + let peer_endpoint = format!("http://{local_addr}"); + + let factory = flow_client_next::workflows::user_collection_auth::new_journal_client_factory( + flow_client_next::rest::Client::new(ctx.config.get_agent_url(), "flowctl"), + models::Capability::Read, + gazette::Router::new("local"), + user_tokens, + ); + // 2 GiB matches the runtime-next default for sidecar-hosted shuffle services. + let shuffle_svc = + shuffle::Service::new(peer_endpoint.clone(), factory, 2 * 1024 * 1024 * 1024); + + let publisher_factory: gazette::journal::ClientFactory = std::sync::Arc::new({ + move |_authz_sub: String, _authz_obj: String| -> gazette::journal::Client { + unreachable!("live Publisher is not used by preview ({_authz_sub}, {_authz_obj})") + } + }); + let runtime_svc = runtime_next::Service::new(shuffle_svc.clone(), publisher_factory); + + let router = tonic::transport::Server::builder() + .add_service(runtime_svc.into_tonic_service()) + .add_service(shuffle_svc.into_tonic_service()); + + let _server_task = + tokio::spawn(router.serve_with_incoming(TcpListenerStream::new(listener))); + + let _rocksdb_tmp = tempfile::tempdir().context("creating preview RocksDB tempdir")?; + let _shuffle_log_tmp = + tempfile::tempdir().context("creating preview shuffle-log tempdir")?; + let rocksdb_path = _rocksdb_tmp.path().to_string_lossy().into_owned(); + let shuffle_log_dir = _shuffle_log_tmp.path().to_string_lossy().into_owned(); + + tracing::info!( + %peer_endpoint, + %rocksdb_path, + %shuffle_log_dir, + n_shards, + "preview services started", + ); + + Ok(Self { + _server_task, + _rocksdb_tmp, + _shuffle_log_tmp, + peer_endpoint, + log_handler, + network, + rocksdb_path, + shuffle_log_dir, + n_shards, + }) + } +} diff --git a/crates/flowctl/src/raw/preview_next/shards.rs b/crates/flowctl/src/raw/preview_next/shards.rs new file mode 100644 index 00000000000..8495eea5a2e --- /dev/null +++ b/crates/flowctl/src/raw/preview_next/shards.rs @@ -0,0 +1,72 @@ +//! Synthetic shard topology for `flowctl preview`. The two shard lists +//! (the leader's `proto::join::Shard` and the shuffle service's +//! `shuffle::proto::Shard`) carry overlapping but distinct fields, both +//! built here off a shared even key-space split. All shards point at the +//! single ephemeral preview endpoint. + +use proto_flow::flow; +use runtime_next::proto; + +/// Compute `[key_begin, key_end]` for shard `i` of `count` over the full +/// `u32` key space. +fn key_range(i: u32, count: u32) -> (u32, u32) { + let begin = if i == 0 { + 0 + } else { + ((i as u64 * (u32::MAX as u64 + 1)) / count as u64) as u32 + }; + let end = if i == count - 1 { + u32::MAX + } else { + (((i + 1) as u64 * (u32::MAX as u64 + 1)) / count as u64 - 1) as u32 + }; + (begin, end) +} + +/// Build the per-shard `proto::join::Shard` list used in `Join` messages. +/// `id`, `reactor`, and `etcd_create_revision` are synthesized; the +/// `labeling.range` matches the shuffle topology for the same index so the +/// shuffle Session sees a consistent 2D tiling. +pub fn build_join_shards( + count: u32, + spec: &flow::MaterializationSpec, +) -> anyhow::Result> { + let build = build_label(spec)?; + + Ok((0..count) + .map(|i| { + let (key_begin, key_end) = key_range(i, count); + proto::join::Shard { + id: format!("preview-shard-{i:03}"), + labeling: Some(::ops::ShardLabeling { + task_name: spec.name.to_string(), + range: Some(flow::RangeSpec { + key_begin, + key_end, + r_clock_begin: 0, + r_clock_end: u32::MAX, + }), + build: build.clone(), + ..Default::default() + }), + reactor: Some(proto_gazette::broker::process_spec::Id { + zone: "local".to_string(), + suffix: format!("preview-{i:03}"), + }), + etcd_create_revision: 1, + } + }) + .collect()) +} + +fn build_label(spec: &flow::MaterializationSpec) -> anyhow::Result { + let labels = spec + .shard_template + .as_ref() + .and_then(|template| template.labels.as_ref()) + .ok_or_else(|| anyhow::anyhow!("built materialization is missing shard labels"))?; + + Ok(labels::expect_one(labels, labels::BUILD) + .map_err(anyhow::Error::from)? + .to_string()) +} diff --git a/crates/flowctl/src/raw/shuffle.rs b/crates/flowctl/src/raw/shuffle.rs deleted file mode 100644 index 61fe1e71978..00000000000 --- a/crates/flowctl/src/raw/shuffle.rs +++ /dev/null @@ -1,481 +0,0 @@ -use anyhow::Context; -use doc::combine; -use proto_flow::flow; - -#[derive(Debug, clap::Args)] -#[clap(rename_all = "kebab-case")] -pub struct Shuffle { - /// Name of the catalog task (materialization, derivation, or collection). - #[clap(long)] - name: String, - /// Port to run the shuffle server on. - #[clap(long, default_value = "9876")] - port: u16, - /// Number of shards in the shuffle topology. - #[clap(long, default_value = "1")] - shards: u32, - /// Directory for log segment files. If omitted, a temporary directory is used. - #[clap(long)] - directory: Option, - /// Minimum interval between transaction checkpoints, in milliseconds. - #[clap(long, default_value = "500")] - interval: u64, - /// Number of checkpoints to process before exiting. - #[clap(long, default_value = "20")] - checkpoints: usize, - /// Disk backlog threshold in Mebibytes before engaging back-pressure. - #[clap(long, default_value = "1024")] - disk_backlog_mib: u64, -} - -impl Shuffle { - pub async fn run(&self, ctx: &mut crate::CliContext) -> anyhow::Result<()> { - let Self { - name, - port, - shards: shard_count, - directory, - interval, - checkpoints, - disk_backlog_mib, - } = self; - - // Fetch the task spec from the control plane. - let task = fetch_task_spec(ctx, name, *disk_backlog_mib * (1024 * 1024)).await?; - - // Start two shuffle servers on adjacent ports. - // Even-indexed shards use the first server, odd-indexed use the second. - let bind_addr_even = format!("127.0.0.1:{port}"); - let bind_addr_odd = format!("127.0.0.1:{}", port + 1); - let peer_addr_even = format!("http://{bind_addr_even}"); - let peer_addr_odd = format!("http://{bind_addr_odd}"); - - // TODO(johnny): handle refresh rotation. - let user_tokens = tokens::fixed(Ok(flow_client_next::user_auth::UserToken { - access_token: ctx.config.user_access_token.clone(), - refresh_token: None, - })); - - let factory = flow_client_next::workflows::user_collection_auth::new_journal_client_factory( - flow_client_next::rest::Client::new(ctx.config.get_agent_url(), "flowctl"), - models::Capability::Read, - gazette::Router::new("local"), - user_tokens, - ); - let service_even = shuffle::Service::new(peer_addr_even.clone(), factory.clone()); - let service_odd = shuffle::Service::new(peer_addr_odd.clone(), factory.clone()); - - let server_even = service_even.clone().build_tonic_server(); - let server_odd = service_odd.build_tonic_server(); - - let listener_even = tokio::net::TcpListener::bind(&bind_addr_even) - .await - .with_context(|| format!("binding to {bind_addr_even}"))?; - let listener_odd = tokio::net::TcpListener::bind(&bind_addr_odd) - .await - .with_context(|| format!("binding to {bind_addr_odd}"))?; - - tracing::info!( - bind_addr_even, - bind_addr_odd, - shard_count, - "starting shuffle servers" - ); - - let server_handle_even = tokio::spawn(async move { - let incoming = tokio_stream::wrappers::TcpListenerStream::new(listener_even); - server_even - .serve_with_incoming(incoming) - .await - .context("shuffle server (even) error") - }); - - let server_handle_odd = tokio::spawn(async move { - let incoming = tokio_stream::wrappers::TcpListenerStream::new(listener_odd); - server_odd - .serve_with_incoming(incoming) - .await - .context("shuffle server (odd) error") - }); - - // Use the provided directory or create a temporary one for log segment files. - let _tmp_dir; // Hold the TempDir so it isn't dropped until the session ends. - let log_dir_str = if let Some(dir) = directory { - std::fs::create_dir_all(dir) - .with_context(|| format!("creating log directory {dir:?}"))?; - _tmp_dir = None; - dir.to_string_lossy().into_owned() - } else { - let td = tempfile::tempdir().context("creating temp directory for log segments")?; - let s = td.path().to_string_lossy().into_owned(); - _tmp_dir = Some(td); - s - }; - - tracing::info!(log_dir = %log_dir_str, "using log directory"); - let combine_spec = build_combine_spec(&task).context("building combine spec")?; - - // Create shard topology: even-indexed shards use even server, odd use odd. - let shards = - build_shard_topology(*shard_count, &peer_addr_even, &peer_addr_odd, &log_dir_str); - - tracing::info!( - addr_even = %peer_addr_even, - addr_odd = %peer_addr_odd, - shard_count, - "opening session" - ); - - let mut client = - shuffle::SessionClient::open(&service_even, task, shards, shuffle::Frontier::default()) - .await - .context("opening session")?; - - tracing::info!("session opened, requesting checkpoints"); - - // Per-shard reader state, taken during each scan and returned after. - let log_dir = std::path::Path::new(&log_dir_str); - let mut shard_state: Vec> = (0..*shard_count) - .map(|i| { - Some(( - shuffle::log::reader::Reader::new(log_dir, i), - std::collections::VecDeque::new(), - )) - }) - .collect(); - - let mut accumulator = combine::Accumulator::new( - combine_spec, - tempfile::tempfile().context("opening combiner spill file")?, - )?; - - let mut total_read_docs: usize = 0; - let mut total_read_bytes: u64 = 0; - let mut bytes_behind: i64 = 0; - - let interval = std::time::Duration::from_millis(*interval); - let start = std::time::Instant::now(); - let mut next_txn_time = start + interval; - - for i in 0..*checkpoints { - // Run no more frequently than once every interval. - tokio::time::sleep(next_txn_time.saturating_duration_since(std::time::Instant::now())) - .await; - next_txn_time = std::time::Instant::now() + interval; - - tracing::debug!(i, "requesting NextCheckpoint"); - - let mut frontier = client - .next_checkpoint() - .await - .context("requesting next checkpoint")?; - - while frontier.unresolved_hints != 0 { - // Reducing intermediate peeks is unnecessary here because the - // eventual ready frontier is a full restatement. - frontier = client - .next_checkpoint() - .await - .context("requesting next checkpoint follow-up")?; - } - - // Scan committed entries from each shard's log, - // pushing documents into the combiner. - let scan_frontier = frontier.clone(); - let (next_shard_state, next_accumulator, read_docs, read_bytes) = - tokio::task::spawn_blocking(move || { - scan_frontier_shards(scan_frontier, shard_state, accumulator) - }) - .await - .context("joining scan task")??; - - shard_state = next_shard_state; - accumulator = next_accumulator; - - bytes_behind += frontier - .journals - .iter() - .map(|jf| jf.bytes_behind_delta) - .sum::(); - - total_read_docs += read_docs; - total_read_bytes += read_bytes; - - let (next_accumulator, combined_docs, combined_bytes) = - tokio::task::spawn_blocking(move || drain_accumulator_to_stdout(accumulator)) - .await - .context("joining drain task")??; - - accumulator = next_accumulator; - - tracing::info!( - i, - journals = frontier.journals.len(), - total_producers = frontier - .journals - .iter() - .map(|j| j.producers.len()) - .sum::(), - flushed_lsn = ?frontier.flushed_lsn, - read_docs, - read_mib = read_bytes / (1024 * 1024), - combined_docs, - combined_mib = combined_bytes / (1024 * 1024), - mib_behind = bytes_behind / (1024 * 1024), - "scanned and combined checkpoint" - ); - } - let elapsed = start.elapsed(); - - client.close().await.context("closing session")?; - - tracing::info!( - total_read_docs, - total_read_mib = total_read_bytes / (1024 * 1024), - avg_doc_rate = total_read_docs as f64 / elapsed.as_secs_f64(), - avg_mib_rate = total_read_bytes as f64 / (elapsed.as_secs_f64() * 1024f64 * 1024f64), - mib_behind = bytes_behind / (1024 * 1024), - "shuffle test completed successfully" - ); - - server_handle_even.abort(); - server_handle_odd.abort(); - - Ok(()) - } -} - -type ShardState = ( - shuffle::log::reader::Reader, - std::collections::VecDeque, -); - -fn scan_frontier_shards( - frontier: shuffle::Frontier, - mut shard_state: Vec>, - mut accumulator: combine::Accumulator, -) -> anyhow::Result<(Vec>, combine::Accumulator, usize, u64)> { - let mut read_docs: usize = 0; - let mut read_bytes: u64 = 0; - - for (shard_index, state_slot) in shard_state.iter_mut().enumerate() { - let (reader, remainders) = state_slot.take().expect("shard state must be present"); - - let mut scan = - shuffle::log::reader::FrontierScan::new(frontier.clone(), reader, remainders) - .with_context(|| format!("creating FrontierScan for shard {shard_index}"))?; - - while scan - .advance_block() - .with_context(|| format!("advancing block for shard {shard_index}"))? - { - let memtable = accumulator.memtable()?; - - for entry in scan.block_iter() { - memtable.add_embedded( - entry.meta.binding.to_native(), - &entry.doc.packed_key_prefix, - entry.doc.doc.to_heap(memtable.alloc()), - false, - entry.meta.flags & shuffle::FLAGS_SCHEMA_VALID != 0, - )?; - read_docs += 1; - read_bytes += entry.doc.source_byte_length.to_native() as u64; - } - } - - let (_, reader, remainders) = scan.into_parts(); - *state_slot = Some((reader, remainders)); - } - - Ok((shard_state, accumulator, read_docs, read_bytes)) -} - -fn drain_accumulator_to_stdout( - accumulator: combine::Accumulator, -) -> anyhow::Result<(combine::Accumulator, usize, usize)> { - let ser_policy = doc::SerPolicy::noop(); - let mut stdout = std::io::stdout(); - let mut combined_docs: usize = 0; - let mut combined_bytes: usize = 0; - let mut buf = Vec::::new(); - - let mut drainer = accumulator.into_drainer()?; - while let Some(drained) = drainer.next() { - let drained = drained.context("draining combined document")?; - serde_json::to_writer(&mut buf, &ser_policy.on_owned(&drained.root)) - .context("writing NDJSON")?; - buf.push(b'\n'); - combined_docs += 1; - combined_bytes += buf.len(); - - std::io::Write::write_all(&mut stdout, &buf).context("flushing NDJSON to stdout")?; - buf.clear(); - } - - let accumulator = drainer - .into_new_accumulator() - .context("recycling accumulator")?; - - Ok((accumulator, combined_docs, combined_bytes)) -} - -async fn fetch_task_spec( - ctx: &mut crate::CliContext, - name: &str, - disk_backlog_threshold: u64, -) -> anyhow::Result { - let builder = ctx - .client - .from("live_specs") - .select("spec_type,built_spec") - .eq("catalog_name", name); - - #[derive(serde::Deserialize)] - struct Row { - spec_type: String, - built_spec: serde_json::Value, - } - - let rows: Vec = crate::api_exec(builder).await?; - - let row = rows - .into_iter() - .next() - .with_context(|| format!("task '{name}' not found"))?; - - let task = match row.spec_type.as_str() { - "materialization" => { - let spec: flow::MaterializationSpec = serde_json::from_value(row.built_spec)?; - tracing::info!(name = spec.name, "fetched materialization"); - shuffle::proto::Task { - task: Some(shuffle::proto::task::Task::Materialization(spec)), - } - } - "collection" => { - let spec: flow::CollectionSpec = serde_json::from_value(row.built_spec)?; - - if spec.derivation.is_some() { - tracing::info!(name = spec.name, "fetched derivation"); - shuffle::proto::Task { - task: Some(shuffle::proto::task::Task::Derivation(spec)), - } - } else { - tracing::info!(name = spec.name, "fetched collection"); - let partition_selector = Some(assemble::journal_selector(&spec, None)); - - shuffle::proto::Task { - task: Some(shuffle::proto::task::Task::CollectionPartitions( - shuffle::proto::CollectionPartitions { - collection: Some(spec), - partition_selector, - disk_backlog_threshold, - }, - )), - } - } - } - other => anyhow::bail!("unsupported spec_type: {other}"), - }; - - Ok(task) -} - -/// Build a `combine::Spec` from the task's collection specs. -/// -/// Each binding corresponds to a source collection, and uses the collection's -/// own key extractors for reduction (not the shuffle key, which may differ -/// for derivation transforms with explicit shuffle keys). -fn build_combine_spec(task: &shuffle::proto::Task) -> anyhow::Result { - let collection_specs: Vec<&flow::CollectionSpec> = match &task.task { - Some(shuffle::proto::task::Task::CollectionPartitions(cp)) => { - vec![cp.collection.as_ref().context("missing collection spec")?] - } - Some(shuffle::proto::task::Task::Materialization(mat)) => mat - .bindings - .iter() - .map(|b| b.collection.as_ref().context("missing collection spec")) - .collect::>()?, - Some(shuffle::proto::task::Task::Derivation(col)) => { - let derivation = col.derivation.as_ref().context("missing derivation")?; - derivation - .transforms - .iter() - .map(|t| t.collection.as_ref().context("missing collection spec")) - .collect::>()? - } - None => anyhow::bail!("missing task variant"), - }; - - let bindings = collection_specs.into_iter().map(|spec| { - let (validator, shape) = - shuffle::binding::build_schema(&spec.read_schema_json, &spec.write_schema_json)?; - let key_extractors = shuffle::binding::build_key_extractors(&spec.key, &shape); - - Ok::<_, anyhow::Error>((true, key_extractors, spec.name.clone(), validator)) - }); - - // Collect to surface errors before passing to with_bindings. - let bindings: Vec<_> = bindings.collect::>()?; - - Ok(combine::Spec::with_bindings(bindings, Vec::new())) -} - -/// Build a shard topology with `count` shards, splitting the key space evenly. -/// Even-indexed shards use `addr_even`, odd-indexed shards use `addr_odd`. -fn build_shard_topology( - count: u32, - addr_even: &str, - addr_odd: &str, - directory: &str, -) -> Vec { - let mut shards = Vec::with_capacity(count as usize); - - for i in 0..count { - // Split key space evenly across shards. - let key_begin = if i == 0 { - 0 - } else { - ((i as u64 * (u32::MAX as u64 + 1)) / count as u64) as u32 - }; - let key_end = if i == count - 1 { - u32::MAX - } else { - (((i + 1) as u64 * (u32::MAX as u64 + 1)) / count as u64 - 1) as u32 - }; - - let endpoint = if i % 2 == 0 { addr_even } else { addr_odd }; - - shards.push(shuffle::proto::Shard { - range: Some(flow::RangeSpec { - key_begin, - key_end, - r_clock_begin: 0, - r_clock_end: u32::MAX, - }), - endpoint: endpoint.to_string(), - directory: directory.to_string(), - }); - } - - tracing::info!( - count, - "built shard topology: {:?}", - shards - .iter() - .enumerate() - .map(|(i, m)| { - let r = m.range.as_ref().unwrap(); - format!( - "{}:[{:#x}, {:#x}]@{}", - i, - r.key_begin, - r.key_end, - if i % 2 == 0 { "even" } else { "odd" } - ) - }) - .collect::>() - ); - - shards -} diff --git a/crates/ops/src/lib.rs b/crates/ops/src/lib.rs index e1154ff0311..55381320f41 100644 --- a/crates/ops/src/lib.rs +++ b/crates/ops/src/lib.rs @@ -7,10 +7,14 @@ pub mod tracing; // Re-export many types from proto_flow::ops, so that users of this crate // don't also have to use that module. +// DEPRECATED. Prefer `proto` instead. pub use proto_flow::ops::{ Log, Meta, ShardLabeling, ShardRef, Stats, TaskType, log::Level as LogLevel, stats, }; +// Re-export proto_flow::ops as proto (use as ops::proto::Foo). +pub use proto_flow::ops as proto; + #[derive(Serialize, Deserialize, Clone, Debug)] #[serde(rename_all = "camelCase")] pub struct Shard { diff --git a/crates/proto-flow/build.rs b/crates/proto-flow/build.rs index e4f8eeebc3c..e22c5f8c3b0 100644 --- a/crates/proto-flow/build.rs +++ b/crates/proto-flow/build.rs @@ -28,6 +28,7 @@ fn main() { ".derive", ".materialize", ".ops", + ".runtime", ".shuffle", ]) .expect("building pbjson"); @@ -70,6 +71,7 @@ fn main() { "./flow.serde.rs", "./materialize.serde.rs", "./ops.serde.rs", + "./runtime.serde.rs", ] { let root = &b.src_dir; let mut buf = std::fs::read_to_string(&root.join(path)).unwrap(); diff --git a/crates/proto-flow/src/lib.rs b/crates/proto-flow/src/lib.rs index ec28cec6ff5..22fd799eb76 100644 --- a/crates/proto-flow/src/lib.rs +++ b/crates/proto-flow/src/lib.rs @@ -10,6 +10,10 @@ pub mod ops; pub mod runtime; pub mod shuffle; +// Re-export so pbjson-generated `super::consumer::...` paths in +// `runtime.serde.rs` resolve. +pub use proto_gazette::consumer; + /// An enum representing any one of the types of built specs. #[derive(Clone, Debug, serde::Serialize)] pub enum AnyBuiltSpec { @@ -126,8 +130,10 @@ mod serde_shuffle { use crate::shuffle::*; include!("shuffle.serde.rs"); } -// We don't generate serde support for the `runtime` protobuf package, -// as it's not intended for JSON serialization. +mod serde_runtime { + use crate::runtime::*; + include!("runtime.serde.rs"); +} /// Capability bit-mask values which scope Estuary platform operations. /// These values share the Gazette capability namespace. diff --git a/crates/proto-flow/src/materialize.rs b/crates/proto-flow/src/materialize.rs index cf86d1a0900..a68e97c8598 100644 --- a/crates/proto-flow/src/materialize.rs +++ b/crates/proto-flow/src/materialize.rs @@ -179,8 +179,18 @@ pub mod request { /// Flush loads. No further Loads will be sent in this transaction, /// and the runtime will await the connectors's remaining Loaded /// responses followed by one Flushed response. - #[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] - pub struct Flush {} + #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] + pub struct Flush { + /// Aggregated connector state patches from all shards' + /// prior-transaction C:Acknowledged responses, in State Update Wire + /// Format. Includes this shard's own patches (the runtime feeds the + /// shard's contribution back to it for symmetry with the scaled-out + /// case). Connectors participating in cooperative multi-shard + /// strategies use this to observe peers' state. The runtime forwards + /// it verbatim from the leader's L:Flush. + #[prost(bytes = "bytes", tag = "1")] + pub connector_state_patches_json: ::prost::bytes::Bytes, + } /// Store documents updated by the current transaction. #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] pub struct Store { @@ -218,11 +228,29 @@ pub mod request { /// Flow runtime checkpoint to commit with this transaction. #[prost(message, optional, tag = "1")] pub runtime_checkpoint: ::core::option::Option<::proto_gazette::consumer::Checkpoint>, + /// Aggregated connector state patches from all shards' current-transaction + /// C:Flushed responses, in State Update Wire Format. Includes this + /// shard's own patches (the runtime feeds the shard's contribution back + /// to it for symmetry with the scaled-out case). Connectors use this + /// to observe peers' Flushed state for cooperative strategies like + /// parallel file staging. The runtime forwards it verbatim from the + /// leader's L:StartCommit. + #[prost(bytes = "bytes", tag = "2")] + pub connector_state_patches_json: ::prost::bytes::Bytes, } /// Acknowledge to the connector that the previous transaction /// has committed to the Flow runtime's recovery log. - #[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] - pub struct Acknowledge {} + #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] + pub struct Acknowledge { + /// Aggregated connector state patches from all shards' just-committed + /// C:StartedCommit responses, in State Update Wire Format. Includes + /// this shard's own patches (the runtime feeds the shard's contribution + /// back to it for symmetry with the scaled-out case). Connectors use + /// this to observe peers' StartedCommit state. The runtime forwards + /// it verbatim from the leader's L:Acknowledge. + #[prost(bytes = "bytes", tag = "1")] + pub connector_state_patches_json: ::prost::bytes::Bytes, + } } #[derive(Clone, PartialEq, ::prost::Message)] pub struct Response { diff --git a/crates/proto-flow/src/materialize.serde.rs b/crates/proto-flow/src/materialize.serde.rs index f2417f2384f..5d8685e9ee5 100644 --- a/crates/proto-flow/src/materialize.serde.rs +++ b/crates/proto-flow/src/materialize.serde.rs @@ -546,8 +546,16 @@ impl serde::Serialize for request::Acknowledge { S: serde::Serializer, { use serde::ser::SerializeStruct; - let len = 0; - let struct_ser = serializer.serialize_struct("materialize.Request.Acknowledge", len)?; + let mut len = 0; + if !self.connector_state_patches_json.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("materialize.Request.Acknowledge", len)?; + if !self.connector_state_patches_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorStatePatches", &crate::as_raw_json(&self.connector_state_patches_json)?)?; + } struct_ser.end() } } @@ -558,10 +566,13 @@ impl<'de> serde::Deserialize<'de> for request::Acknowledge { D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ + "connector_state_patches_json", + "connectorStatePatches", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { + ConnectorStatePatchesJson, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -582,7 +593,10 @@ impl<'de> serde::Deserialize<'de> for request::Acknowledge { where E: serde::de::Error, { - Err(serde::de::Error::unknown_field(value, FIELDS)) + match value { + "connectorStatePatches" | "connector_state_patches_json" => Ok(GeneratedField::ConnectorStatePatchesJson), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } } } deserializer.deserialize_identifier(GeneratedVisitor) @@ -600,10 +614,21 @@ impl<'de> serde::Deserialize<'de> for request::Acknowledge { where V: serde::de::MapAccess<'de>, { - while map_.next_key::()?.is_some() { - let _ = map_.next_value::()?; + let mut connector_state_patches_json__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::ConnectorStatePatchesJson => { + if connector_state_patches_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorStatePatches")); + } + connector_state_patches_json__ = + Some(map_.next_value::()?.0) + ; + } + } } Ok(request::Acknowledge { + connector_state_patches_json: connector_state_patches_json__.unwrap_or_default(), }) } } @@ -783,8 +808,16 @@ impl serde::Serialize for request::Flush { S: serde::Serializer, { use serde::ser::SerializeStruct; - let len = 0; - let struct_ser = serializer.serialize_struct("materialize.Request.Flush", len)?; + let mut len = 0; + if !self.connector_state_patches_json.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("materialize.Request.Flush", len)?; + if !self.connector_state_patches_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorStatePatches", &crate::as_raw_json(&self.connector_state_patches_json)?)?; + } struct_ser.end() } } @@ -795,10 +828,13 @@ impl<'de> serde::Deserialize<'de> for request::Flush { D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ + "connector_state_patches_json", + "connectorStatePatches", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { + ConnectorStatePatchesJson, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -819,7 +855,10 @@ impl<'de> serde::Deserialize<'de> for request::Flush { where E: serde::de::Error, { - Err(serde::de::Error::unknown_field(value, FIELDS)) + match value { + "connectorStatePatches" | "connector_state_patches_json" => Ok(GeneratedField::ConnectorStatePatchesJson), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } } } deserializer.deserialize_identifier(GeneratedVisitor) @@ -837,10 +876,21 @@ impl<'de> serde::Deserialize<'de> for request::Flush { where V: serde::de::MapAccess<'de>, { - while map_.next_key::()?.is_some() { - let _ = map_.next_value::()?; + let mut connector_state_patches_json__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::ConnectorStatePatchesJson => { + if connector_state_patches_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorStatePatches")); + } + connector_state_patches_json__ = + Some(map_.next_value::()?.0) + ; + } + } } Ok(request::Flush { + connector_state_patches_json: connector_state_patches_json__.unwrap_or_default(), }) } } @@ -1258,10 +1308,18 @@ impl serde::Serialize for request::StartCommit { if self.runtime_checkpoint.is_some() { len += 1; } + if !self.connector_state_patches_json.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("materialize.Request.StartCommit", len)?; if let Some(v) = self.runtime_checkpoint.as_ref() { struct_ser.serialize_field("runtimeCheckpoint", v)?; } + if !self.connector_state_patches_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorStatePatches", &crate::as_raw_json(&self.connector_state_patches_json)?)?; + } struct_ser.end() } } @@ -1274,11 +1332,14 @@ impl<'de> serde::Deserialize<'de> for request::StartCommit { const FIELDS: &[&str] = &[ "runtime_checkpoint", "runtimeCheckpoint", + "connector_state_patches_json", + "connectorStatePatches", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { RuntimeCheckpoint, + ConnectorStatePatchesJson, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -1301,6 +1362,7 @@ impl<'de> serde::Deserialize<'de> for request::StartCommit { { match value { "runtimeCheckpoint" | "runtime_checkpoint" => Ok(GeneratedField::RuntimeCheckpoint), + "connectorStatePatches" | "connector_state_patches_json" => Ok(GeneratedField::ConnectorStatePatchesJson), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -1321,6 +1383,7 @@ impl<'de> serde::Deserialize<'de> for request::StartCommit { V: serde::de::MapAccess<'de>, { let mut runtime_checkpoint__ = None; + let mut connector_state_patches_json__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::RuntimeCheckpoint => { @@ -1329,10 +1392,19 @@ impl<'de> serde::Deserialize<'de> for request::StartCommit { } runtime_checkpoint__ = map_.next_value()?; } + GeneratedField::ConnectorStatePatchesJson => { + if connector_state_patches_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorStatePatches")); + } + connector_state_patches_json__ = + Some(map_.next_value::()?.0) + ; + } } } Ok(request::StartCommit { runtime_checkpoint: runtime_checkpoint__, + connector_state_patches_json: connector_state_patches_json__.unwrap_or_default(), }) } } diff --git a/crates/proto-flow/src/runtime.rs b/crates/proto-flow/src/runtime.rs index ef660f0c7d0..b4abfa0eebf 100644 --- a/crates/proto-flow/src/runtime.rs +++ b/crates/proto-flow/src/runtime.rs @@ -419,6 +419,585 @@ pub struct ConnectorProxyResponse { #[prost(message, optional, tag = "3")] pub log: ::core::option::Option, } +/// Join is sent by each shard to the leader at session start. The leader +/// gathers all expected Joins, verifies `shards` are exactly equal, +/// and responds with Joined on consensus or disagreement. +/// +/// Index 0 in `shards` is "shard zero": its reactor is co-located +/// with the Shuffle Leader sidecar that owns this session, and it hosts +/// the task's recovery log. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Join { + /// Etcd mod revision at which `shards` was observed. + /// On disagreement all participants await the highest revision, then re-try. + #[prost(int64, tag = "1")] + pub etcd_mod_revision: i64, + /// Current shard topology as observed by the sender. + #[prost(message, repeated, tag = "2")] + pub shards: ::prost::alloc::vec::Vec, + /// This shard's index within `shards`. + #[prost(uint32, tag = "3")] + pub shard_index: u32, + /// Client-owned local directory where this shuffle Log segments should be written. + #[prost(string, tag = "4")] + pub shuffle_directory: ::prost::alloc::string::String, + /// Dial-able gRPC endpoint for reaching this shard's shuffle service. + #[prost(string, tag = "5")] + pub shuffle_endpoint: ::prost::alloc::string::String, + /// Dial-able gRPC endpoint for reaching the shuffle Leader's shuffle service. + #[prost(string, tag = "6")] + pub leader_endpoint: ::prost::alloc::string::String, +} +/// Nested message and enum types in `Join`. +pub mod join { + /// Shard describes one shard and the reactor it is assigned to. + /// + /// Shards are ordered on ascending `shard.range.key_begin` and, + /// within a key range, on ascending `shard.range.r_clock_begin`. The + /// tiling must be complete and non-overlapping in 2D over + /// \[key_begin, key_end\] and \[r_clock_begin, r_clock_end\]. + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct Shard { + #[prost(string, tag = "1")] + pub id: ::prost::alloc::string::String, + /// Labeling that captures all shard metadata. + #[prost(message, optional, tag = "2")] + pub labeling: ::core::option::Option, + /// Hosting reactor ID. + #[prost(message, optional, tag = "3")] + pub reactor: ::core::option::Option<::proto_gazette::broker::process_spec::Id>, + /// Creation revision of the Etcd key representing the shard's + /// reactor assignment. Represents assignment identity: + /// each re-assignment has a unique, monotonic revision. + #[prost(int64, tag = "4")] + pub etcd_create_revision: i64, + } +} +/// Joined is the leader's response to Join, fanned out to every joining shard. +/// Each shard then forwards Joined to its controller. +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct Joined { + /// Zero: consensus reached (success); the session proceeds. + /// Non-zero: topology disagreement. Await this revision, then retry. + #[prost(int64, tag = "1")] + pub max_etcd_revision: i64, +} +/// Task which is being processed by the runtime. +/// Sent from Controller to Shard, and from Shard zero (only) to Leader +/// after Joined. Other shards do not forward Task. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Task { + /// Task specification (protobuf-encoded bytes). + #[prost(bytes = "bytes", tag = "1")] + pub spec: ::prost::bytes::Bytes, + /// Collection journal partition to which task states are written. + #[prost(string, tag = "2")] + pub ops_stats_journal: ::prost::alloc::string::String, + /// Collection to which task stats are written. + #[prost(message, optional, tag = "3")] + pub ops_stats_spec: ::core::option::Option, + /// When true, documents and stats are written to output and not directed to collections. + #[prost(bool, tag = "4")] + pub preview: bool, + /// Preview / harness control. Zero means unlimited. + #[prost(uint32, tag = "5")] + pub max_transactions: u32, +} +/// Recover is sent by each shard to the leader after Joined, and carries +/// state recovered from the shard's RocksDB. +/// +/// Shard zero populates Recover from its actual RocksDB scan. Non-zero +/// shards have no recovery log: they MUST send `Recover{}` (proto default) +/// or the leader fails the session. This is the migration guard: a non-zero +/// shard reporting state is evidence of stale per-shard data left over +/// from before consolidation, and we'd rather fail loudly than silently +/// merge it. +/// +/// Hinted vs committed. `committed_*` is durable in the recovery log. +/// `hinted_*` was *intended* to commit; it is durable on its last write +/// but the corresponding transaction may or may not have completed. +/// Remote-authoritative connectors may commit a transaction in the +/// *endpoint* (during StartCommit/StartedCommit) ahead of the recovery +/// log; the leader detects this at startup by comparing the close-clock +/// embedded in `C:Opened`'s checkpoint against `hinted_close_clock`, +/// and promotes the hinted Frontier to committed when they agree. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Recover { + /// Last-persisted ACK intents. + #[prost(btree_map = "string, bytes", tag = "1")] + pub ack_intents: ::prost::alloc::collections::BTreeMap< + ::prost::alloc::string::String, + ::prost::bytes::Bytes, + >, + /// Clock at which the last-committed transaction closed, + /// or zero if never committed. + #[prost(fixed64, tag = "2")] + pub committed_close_clock: u64, + /// Committed Frontier entries. Not a delta. + #[prost(message, optional, tag = "3")] + pub committed_frontier: ::core::option::Option, + /// Persisted connector state. + #[prost(bytes = "bytes", tag = "4")] + pub connector_state_json: ::prost::bytes::Bytes, + /// Clock at which the last hinted transaction closed, or zero if never + /// hinted. If equal to `committed_close_clock` the hinted transaction + /// committed normally; if ahead of `committed_close_clock` the hinted + /// transaction was started but did not commit in the recovery log + /// (and may or may not have committed in the connector's endpoint). + #[prost(fixed64, tag = "5")] + pub hinted_close_clock: u64, + /// Persisted hinted Frontier entries (FH: range). + #[prost(message, optional, tag = "6")] + pub hinted_frontier: ::core::option::Option, + /// Last-applied task specification (protobuf-encoded bytes), or empty. + #[prost(bytes = "bytes", tag = "7")] + pub last_applied: ::prost::bytes::Bytes, + /// Legacy Checkpoint persisted in RocksDB for roll-forward / roll-back + /// capability with the V1 runtime. This will be phased out as we fully + /// cut over to the `committed_frontier` representation. + /// + /// Distinct from the checkpoint returned by C:Opened (see the + /// `Recover` message comment above). The C:Opened checkpoint stays; + /// only this `legacy_checkpoint` is on the path to removal. + #[prost(message, optional, tag = "8")] + pub legacy_checkpoint: ::core::option::Option<::proto_gazette::consumer::Checkpoint>, + /// Per-binding max-key entries. + /// Key: binding index; Value: packed composite key tuple. + #[prost(btree_map = "uint32, bytes", tag = "9")] + pub max_keys: ::prost::alloc::collections::BTreeMap, + /// Persisted trigger parameters (materialize only), or empty. + #[prost(bytes = "bytes", tag = "10")] + pub trigger_params_json: ::prost::bytes::Bytes, +} +/// Persist is sent by the leader to shard zero when state must be durably +/// written. Each field maps to a contractual WriteBatch effect on shard +/// zero's RocksDB. Absent fields are inert. +/// +/// All fields of a Persist land together in a single WriteBatch. +/// `nonce` is echoed back by the shard's Persisted response, allowing +/// the leader to match a Persisted response to its originating request. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Persist { + /// Request ID picked by the leader and echoed verbatim in the shard's + /// `Persisted` response. The leader chooses any value and the shard + /// does not interpret it. + #[prost(uint64, tag = "1")] + pub nonce: u64, + /// Delete previously-persisted ACK intents. Applies ahead of `ack_intents`. + /// Effect: DeleteRange("AI:") + #[prost(bool, tag = "2")] + pub delete_ack_intents: bool, + /// ACK intent entries. + /// Key: journal name; Value: raw journal content to write. + /// Effect: Put under "AI:{journal}". + #[prost(btree_map = "string, bytes", tag = "3")] + pub ack_intents: ::prost::alloc::collections::BTreeMap< + ::prost::alloc::string::String, + ::prost::bytes::Bytes, + >, + /// Clock at which the last-committed transaction closed. + /// Effect: Put under "committed-close-clock". + #[prost(fixed64, tag = "4")] + pub committed_close_clock: u64, + /// Committed Frontier entries. + /// Effect: Put under "FC:..." keys. + #[prost(message, optional, tag = "5")] + pub committed_frontier: ::core::option::Option, + /// Connector state patches. State Update Wire Format. + /// Effect: Merge each patch under "connector-state". + #[prost(bytes = "bytes", tag = "6")] + pub connector_patches_json: ::prost::bytes::Bytes, + /// Clock at which the hinted transaction closed. + /// Effect: Put under "hinted-close-clock". + #[prost(fixed64, tag = "7")] + pub hinted_close_clock: u64, + /// Delete a previously-persisted hinted frontier. Applies ahead of `hinted_frontier`. + /// Effect: DeleteRange("FH:") + #[prost(bool, tag = "8")] + pub delete_hinted_frontier: bool, + /// Hinted Frontier entries. + /// Effect: Put under "FH:" keys. + #[prost(message, optional, tag = "9")] + pub hinted_frontier: ::core::option::Option, + /// Last-applied task specification (protobuf-encoded bytes), or empty. + /// Effect: Put under "last-applied" key. + #[prost(bytes = "bytes", tag = "10")] + pub last_applied: ::prost::bytes::Bytes, + /// Legacy checkpoint, required for rollback to legacy runtime. + /// Effect: Put under "checkpoint" key. + #[prost(message, optional, tag = "11")] + pub legacy_checkpoint: ::core::option::Option<::proto_gazette::consumer::Checkpoint>, + /// Per-binding max-key updates, reduced to per-binding maximum across shards. + /// Key: binding index; Value: packed composite key tuple. + /// Effect: Put value under "MK-v2:{state_key}" (state_key resolved by the encoder). + #[prost(btree_map = "uint32, bytes", tag = "12")] + pub max_keys: ::prost::alloc::collections::BTreeMap, + /// Delete previously-persisted trigger parameters. Applies ahead of `trigger_params_json`. + /// Effect: Delete the "trigger-params" key. + #[prost(bool, tag = "13")] + pub delete_trigger_params: bool, + /// Materialization trigger parameters. + /// Effect: Put under "trigger-params" key. + #[prost(bytes = "bytes", tag = "14")] + pub trigger_params_json: ::prost::bytes::Bytes, +} +/// Persisted is sent by shard zero to the leader after the state is durable +/// in the recovery log. +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct Persisted { + /// Echoed back from the originating `Persist.nonce` request. + #[prost(uint64, tag = "1")] + pub nonce: u64, +} +/// Apply asks shard zero to invoke its connector's Apply action, both for +/// the initial application of a new spec and for re-application after a +/// connector returned state patches that the leader has now persisted. +/// +/// The leader's apply loop iterates Apply / Applied / (Persist / +/// Persisted) until the connector returns no further patches; on the +/// terminal iteration it Persists `last_applied = spec` and returns to +/// the rest of session startup. +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct Apply { + /// Task specification to be applied (protobuf-encoded bytes). + #[prost(bytes = "bytes", tag = "1")] + pub spec: ::prost::bytes::Bytes, + /// Version of the specification being applied. + #[prost(string, tag = "2")] + pub version: ::prost::alloc::string::String, + /// Last specification which was successfully applied, or empty. + #[prost(bytes = "bytes", tag = "4")] + pub last_spec: ::prost::bytes::Bytes, + /// Version of the last applied specification, or empty. + #[prost(string, tag = "5")] + pub last_version: ::prost::alloc::string::String, + /// Reduced connector state observed by this Apply iteration. + #[prost(bytes = "bytes", tag = "6")] + pub connector_state_json: ::prost::bytes::Bytes, +} +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct Applied { + /// Human-readable description of the action that the connector took. + /// If empty, this Apply is to be considered a "no-op". + #[prost(string, tag = "1")] + pub action_description: ::prost::alloc::string::String, + /// Applied connector state patches. State Update Wire Format. + #[prost(bytes = "bytes", tag = "2")] + pub connector_patches_json: ::prost::bytes::Bytes, +} +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Open { + /// Task specification to be opened (protobuf-encoded bytes). + #[prost(bytes = "bytes", tag = "1")] + pub spec: ::prost::bytes::Bytes, + /// Version of the specification being opened. + #[prost(string, tag = "2")] + pub version: ::prost::alloc::string::String, + /// Range of documents to be processed by this session. + #[prost(message, optional, tag = "3")] + pub range: ::core::option::Option, + /// Last-persisted connector state, post-Apply, from `Recover`. + #[prost(bytes = "bytes", tag = "4")] + pub connector_state_json: ::prost::bytes::Bytes, + /// Materializations only: per-binding maximum keys. + /// Key: binding index; Value: packed composite key tuple. + #[prost(btree_map = "uint32, bytes", tag = "5")] + pub max_keys: ::prost::alloc::collections::BTreeMap, +} +/// CloseNow is sent Controller → Shard → Leader, as a request to +/// immediately close a transaction being held open by policy. +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct CloseNow {} +/// Stop is sent Controller → Shard → Leader to request graceful shutdown. +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct Stop {} +/// Stopped confirms the session has shut down. The leader sends Stopped +/// to each shard followed by EOF; each shard then forwards Stopped to its +/// controller and EOFs. +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct Stopped {} +/// SessionLoop is sent as the first message of a session-loop stream on +/// the Shard service. It carries process-level configuration that +/// outlives the cycle of leader sessions on this stream. The Leader +/// service never sees SessionLoop. +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct SessionLoop { + /// RocksDB context opened for this Shard stream and reused across all + /// of its leader sessions. Absent for non-zero materialize/derive + /// shards, which don't host a recovery log. + #[prost(message, optional, tag = "1")] + pub rocksdb_descriptor: ::core::option::Option, +} +/// Materialize is the bidirectional message type for materialization +/// sessions. Exactly one field is set per message. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Materialize { + /// Controller → Shard. Unary request outside of a SessionLoop. + #[prost(message, optional, tag = "1")] + pub spec: ::core::option::Option, + /// Shard → Controller. Connector's reply to `spec`. + #[prost(message, optional, tag = "2")] + pub spec_response: ::core::option::Option, + /// Controller → Shard. Unary request outside of a SessionLoop. + #[prost(message, optional, tag = "3")] + pub validate: ::core::option::Option, + /// Shard → Controller. Connector's reply to `validate`. + #[prost(message, optional, tag = "4")] + pub validated: ::core::option::Option, + /// Controller → Shard. First message of a session-loop stream; + /// never sent to the Leader. + #[prost(message, optional, tag = "20")] + pub session_loop: ::core::option::Option, + /// Controller → Shard → Leader. Session initiation with topology. + #[prost(message, optional, tag = "21")] + pub join: ::core::option::Option, + /// Leader → Shards → Controllers. Consensus or retry directive + /// (`max_etcd_revision == 0` means consensus; non-zero means retry). + #[prost(message, optional, tag = "22")] + pub joined: ::core::option::Option, + /// Controller → Shard, and Shard zero (only) → Leader. + /// Defines the task being processed by the now-joined topology. + #[prost(message, optional, tag = "23")] + pub task: ::core::option::Option, + /// Each Shard → Leader. State recovered from RocksDB on startup. + /// Non-zero shards MUST send `Recover{}` (default) — see Recover. + #[prost(message, optional, tag = "24")] + pub recover: ::core::option::Option, + /// Leader → Shard zero. Re-applied until `Applied` returns no patches. + #[prost(message, optional, tag = "25")] + pub apply: ::core::option::Option, + /// Shard zero → Leader. C:Apply complete. If `Applied` emits state + /// patches the leader runs Persist/Persisted and calls Apply again, + /// until it emits no further patches. + #[prost(message, optional, tag = "26")] + pub applied: ::core::option::Option, + /// Leader → Shards. Open connector and prepare for transactions. + #[prost(message, optional, tag = "27")] + pub open: ::core::option::Option, + #[prost(message, optional, tag = "28")] + pub opened: ::core::option::Option, + #[prost(message, optional, tag = "40")] + pub load: ::core::option::Option, + #[prost(message, optional, tag = "41")] + pub loaded: ::core::option::Option, + #[prost(message, optional, tag = "42")] + pub flush: ::core::option::Option, + #[prost(message, optional, tag = "43")] + pub flushed: ::core::option::Option, + #[prost(message, optional, tag = "44")] + pub store: ::core::option::Option, + #[prost(message, optional, tag = "45")] + pub stored: ::core::option::Option, + #[prost(message, optional, tag = "46")] + pub start_commit: ::core::option::Option, + #[prost(message, optional, tag = "47")] + pub started_commit: ::core::option::Option, + #[prost(message, optional, tag = "48")] + pub acknowledge: ::core::option::Option, + #[prost(message, optional, tag = "49")] + pub acknowledged: ::core::option::Option, + /// Leader → Shard zero. Durably persist state. + #[prost(message, optional, tag = "50")] + pub persist: ::core::option::Option, + /// Shard zero → Leader. State is durable. + #[prost(message, optional, tag = "51")] + pub persisted: ::core::option::Option, + /// Controller → Shard → Leader. Request immediate close of the + /// currently-open transaction. + #[prost(message, optional, tag = "52")] + pub close_now: ::core::option::Option, + /// Controller → Shard → Leader. Graceful shutdown request. + #[prost(message, optional, tag = "60")] + pub stop: ::core::option::Option, + /// Leader → Shards → Controllers. Shutdown confirmed; EOF follows on + /// each leg. + #[prost(message, optional, tag = "61")] + pub stopped: ::core::option::Option, +} +/// Nested message and enum types in `Materialize`. +pub mod materialize { + /// Each Shard → Leader, and each Shard → Controller. Connector is + /// running and session startup is complete. + /// + /// The shard sends *two* Opened messages per session — one to each + /// peer — populated differently: + /// + /// * To the leader: `container` is empty; `connector_checkpoint` is + /// the C:Opened checkpoint (if any). Non-zero shards MUST send + /// `Opened{}` (default) here, mirroring the Recover migration + /// guard: a non-empty checkpoint from a non-zero shard is treated + /// as evidence of stale per-shard state, and startup fails. + /// * To the controller: `container` describes the running connector + /// container; `connector_checkpoint` is empty. + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct Opened { + /// Description of the running connector container. + #[prost(message, optional, tag = "1")] + pub container: ::core::option::Option, + /// Optional connector consumer.Checkpoint returned by C:Opened. + /// + /// Remote-authoritative connectors commit to their endpoint during + /// StartCommit, and the leader compares the close-clock embedded in + /// this checkpoint against the recovered `hinted_close_clock` to + /// detect a commit that landed in the endpoint but not in the + /// recovery log. See `Recover` for the full hinted/committed model. + /// + /// Migration: if no close-clock is embedded the checkpoint is + /// assumed to be a V1 legacy checkpoint and replaces the recovered + /// committed Frontier. The V2 runtime may write additional checkpoint + /// keys for rollback compatibility with V1. + #[prost(message, optional, tag = "2")] + pub connector_checkpoint: ::core::option::Option<::proto_gazette::consumer::Checkpoint>, + } + /// Leader → Shards. Incremental Frontier to process into transaction. + /// Shards add source docs to their combiner and issue C:Load RPCs for + /// documents that may already exist in the endpoint. + /// Multiple Load messages may be sent per transaction. + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct Load { + #[prost(message, optional, tag = "1")] + pub frontier: ::core::option::Option, + } + /// Shard → Leader. All frontier documents have been loaded into the + /// combiner and all C:Load requests sent into the connector. + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct Loaded { + #[prost(message, repeated, tag = "1")] + pub bindings: ::prost::alloc::vec::Vec, + /// On-disk size of this shard's combiner, for transaction close policy. + #[prost(uint64, tag = "2")] + pub combiner_usage_bytes: u64, + } + /// Nested message and enum types in `Loaded`. + pub mod loaded { + /// Binding which participated in this Loaded frontier. + #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] + pub struct Binding { + /// Binding index. + #[prost(uint32, tag = "1")] + pub index: u32, + /// Minimum source document Clock observed + #[prost(fixed64, tag = "2")] + pub min_source_clock: u64, + /// Maximum source document Clock observed + #[prost(fixed64, tag = "3")] + pub max_source_clock: u64, + /// Number of source documents processed. + #[prost(uint64, tag = "4")] + pub sourced_docs_total: u64, + /// Number of source document bytes processed. + #[prost(uint64, tag = "5")] + pub sourced_bytes_total: u64, + /// When non-empty, a new maximum key was observed. + #[prost(bytes = "bytes", tag = "6")] + pub max_key_delta: ::prost::bytes::Bytes, + } + } + /// Leader → Shards. Signals end of Load phase. + #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] + pub struct Flush { + /// Prior transaction's aggregated C:Acknowledged state patches. + /// State Update Wire Format. + #[prost(bytes = "bytes", tag = "1")] + pub connector_patches_json: ::prost::bytes::Bytes, + } + /// Shard → Leader. Flush phase complete. + /// Reports connector state patches and max-key deltas from C:Flushed. + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct Flushed { + #[prost(message, repeated, tag = "1")] + pub bindings: ::prost::alloc::vec::Vec, + /// Connector state patches from this shard's C:Flushed. + /// State Update Wire Format. + #[prost(bytes = "bytes", tag = "2")] + pub connector_patches_json: ::prost::bytes::Bytes, + } + /// Nested message and enum types in `Flushed`. + pub mod flushed { + /// Binding which participated in this Flushed phase. + #[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] + pub struct Binding { + /// Binding index. + #[prost(uint32, tag = "1")] + pub index: u32, + /// Number of endpoint documents loaded by this binding + /// between the last L:Loaded and C:Flushed. + #[prost(uint64, tag = "2")] + pub loaded_docs_total: u64, + /// Number of endpoint document bytes loaded by this binding + /// between the last L:Loaded and C:Flushed. + #[prost(uint64, tag = "3")] + pub loaded_bytes_total: u64, + } + } + /// Leader → Shards. Idempotency Persist now complete; drain combiners + /// into C:Store requests. + #[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] + pub struct Store {} + /// Shard → Leader. Store phase complete; shard is ready to commit. + /// Reports final per-binding transaction statistics. + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct Stored { + #[prost(message, repeated, tag = "1")] + pub bindings: ::prost::alloc::vec::Vec, + } + /// Nested message and enum types in `Stored`. + pub mod stored { + /// Binding which participated in this Stored phase. + #[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] + pub struct Binding { + /// Binding index. + #[prost(uint32, tag = "1")] + pub index: u32, + /// Number of endpoint documents stored by this binding. + #[prost(uint64, tag = "2")] + pub stored_docs_total: u64, + /// Number of endpoint document bytes stored by this binding. + #[prost(uint64, tag = "3")] + pub stored_bytes_total: u64, + } + } + /// Leader → Shards. Begin commit phase. + /// Distributes aggregated L:Flushed patches from all shards so each connector + /// instance can observe its peers' state for cooperative strategies (such as + /// parallel file staging). + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct StartCommit { + /// Aggregated Flushed connector state patches from all shards. + /// State Update Wire Format. + #[prost(bytes = "bytes", tag = "1")] + pub connector_patches_json: ::prost::bytes::Bytes, + /// Transaction Checkpoint for remote-authoritative connectors. + #[prost(message, optional, tag = "2")] + pub connector_checkpoint: ::core::option::Option<::proto_gazette::consumer::Checkpoint>, + } + /// Shard → Leader. Commit initiated. + /// Remote-authoritative connectors will literally commit prior to return. + #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] + pub struct StartedCommit { + /// Connector state patches from this shard's C:StartedCommit. + /// State Update Wire Format. + #[prost(bytes = "bytes", tag = "1")] + pub connector_patches_json: ::prost::bytes::Bytes, + } + /// Leader → Shards. Transaction complete. + #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] + pub struct Acknowledge { + /// Aggregated StartedCommit connector state patches from all shards. + /// State Update Wire Format. + #[prost(bytes = "bytes", tag = "1")] + pub connector_patches_json: ::prost::bytes::Bytes, + } + /// Shard → Leader. Reports C:Acknowledged state from prior transaction. + /// Sent by each shard after its connector responds to C:Acknowledge. + #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] + pub struct Acknowledged { + /// Connector state patches from this shard's C:Acknowledged. + /// State Update Wire Format. + #[prost(bytes = "bytes", tag = "1")] + pub connector_patches_json: ::prost::bytes::Bytes, + } +} /// Plane describes the type of data plane in which the runtime is operating. #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] diff --git a/crates/proto-flow/src/runtime.serde.rs b/crates/proto-flow/src/runtime.serde.rs new file mode 100644 index 00000000000..89c06951649 --- /dev/null +++ b/crates/proto-flow/src/runtime.serde.rs @@ -0,0 +1,7950 @@ +impl serde::Serialize for Applied { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.action_description.is_empty() { + len += 1; + } + if !self.connector_patches_json.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Applied", len)?; + if !self.action_description.is_empty() { + struct_ser.serialize_field("actionDescription", &self.action_description)?; + } + if !self.connector_patches_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorPatches", &crate::as_raw_json(&self.connector_patches_json)?)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Applied { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "action_description", + "actionDescription", + "connector_patches_json", + "connectorPatches", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + ActionDescription, + ConnectorPatchesJson, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "actionDescription" | "action_description" => Ok(GeneratedField::ActionDescription), + "connectorPatches" | "connector_patches_json" => Ok(GeneratedField::ConnectorPatchesJson), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Applied; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Applied") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut action_description__ = None; + let mut connector_patches_json__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::ActionDescription => { + if action_description__.is_some() { + return Err(serde::de::Error::duplicate_field("actionDescription")); + } + action_description__ = Some(map_.next_value()?); + } + GeneratedField::ConnectorPatchesJson => { + if connector_patches_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorPatches")); + } + connector_patches_json__ = + Some(map_.next_value::()?.0) + ; + } + } + } + Ok(Applied { + action_description: action_description__.unwrap_or_default(), + connector_patches_json: connector_patches_json__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Applied", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Apply { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.spec.is_empty() { + len += 1; + } + if !self.version.is_empty() { + len += 1; + } + if !self.last_spec.is_empty() { + len += 1; + } + if !self.last_version.is_empty() { + len += 1; + } + if !self.connector_state_json.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Apply", len)?; + if !self.spec.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("spec", pbjson::private::base64::encode(&self.spec).as_str())?; + } + if !self.version.is_empty() { + struct_ser.serialize_field("version", &self.version)?; + } + if !self.last_spec.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("lastSpec", pbjson::private::base64::encode(&self.last_spec).as_str())?; + } + if !self.last_version.is_empty() { + struct_ser.serialize_field("lastVersion", &self.last_version)?; + } + if !self.connector_state_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorStateJson", &crate::as_raw_json(&self.connector_state_json)?)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Apply { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "spec", + "version", + "last_spec", + "lastSpec", + "last_version", + "lastVersion", + "connector_state_json", + "connectorStateJson", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Spec, + Version, + LastSpec, + LastVersion, + ConnectorStateJson, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "spec" => Ok(GeneratedField::Spec), + "version" => Ok(GeneratedField::Version), + "lastSpec" | "last_spec" => Ok(GeneratedField::LastSpec), + "lastVersion" | "last_version" => Ok(GeneratedField::LastVersion), + "connectorStateJson" | "connector_state_json" => Ok(GeneratedField::ConnectorStateJson), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Apply; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Apply") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut spec__ = None; + let mut version__ = None; + let mut last_spec__ = None; + let mut last_version__ = None; + let mut connector_state_json__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Spec => { + if spec__.is_some() { + return Err(serde::de::Error::duplicate_field("spec")); + } + spec__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + GeneratedField::Version => { + if version__.is_some() { + return Err(serde::de::Error::duplicate_field("version")); + } + version__ = Some(map_.next_value()?); + } + GeneratedField::LastSpec => { + if last_spec__.is_some() { + return Err(serde::de::Error::duplicate_field("lastSpec")); + } + last_spec__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + GeneratedField::LastVersion => { + if last_version__.is_some() { + return Err(serde::de::Error::duplicate_field("lastVersion")); + } + last_version__ = Some(map_.next_value()?); + } + GeneratedField::ConnectorStateJson => { + if connector_state_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorStateJson")); + } + connector_state_json__ = + Some(map_.next_value::()?.0) + ; + } + } + } + Ok(Apply { + spec: spec__.unwrap_or_default(), + version: version__.unwrap_or_default(), + last_spec: last_spec__.unwrap_or_default(), + last_version: last_version__.unwrap_or_default(), + connector_state_json: connector_state_json__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Apply", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for CaptureRequestExt { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.log_level != 0 { + len += 1; + } + if self.rocksdb_descriptor.is_some() { + len += 1; + } + if self.start_commit.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.CaptureRequestExt", len)?; + if self.log_level != 0 { + let v = super::ops::log::Level::try_from(self.log_level) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.log_level)))?; + struct_ser.serialize_field("logLevel", &v)?; + } + if let Some(v) = self.rocksdb_descriptor.as_ref() { + struct_ser.serialize_field("rocksdbDescriptor", v)?; + } + if let Some(v) = self.start_commit.as_ref() { + struct_ser.serialize_field("startCommit", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CaptureRequestExt { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "log_level", + "logLevel", + "rocksdb_descriptor", + "rocksdbDescriptor", + "start_commit", + "startCommit", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + LogLevel, + RocksdbDescriptor, + StartCommit, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "logLevel" | "log_level" => Ok(GeneratedField::LogLevel), + "rocksdbDescriptor" | "rocksdb_descriptor" => Ok(GeneratedField::RocksdbDescriptor), + "startCommit" | "start_commit" => Ok(GeneratedField::StartCommit), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CaptureRequestExt; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CaptureRequestExt") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut log_level__ = None; + let mut rocksdb_descriptor__ = None; + let mut start_commit__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::LogLevel => { + if log_level__.is_some() { + return Err(serde::de::Error::duplicate_field("logLevel")); + } + log_level__ = Some(map_.next_value::()? as i32); + } + GeneratedField::RocksdbDescriptor => { + if rocksdb_descriptor__.is_some() { + return Err(serde::de::Error::duplicate_field("rocksdbDescriptor")); + } + rocksdb_descriptor__ = map_.next_value()?; + } + GeneratedField::StartCommit => { + if start_commit__.is_some() { + return Err(serde::de::Error::duplicate_field("startCommit")); + } + start_commit__ = map_.next_value()?; + } + } + } + Ok(CaptureRequestExt { + log_level: log_level__.unwrap_or_default(), + rocksdb_descriptor: rocksdb_descriptor__, + start_commit: start_commit__, + }) + } + } + deserializer.deserialize_struct("runtime.CaptureRequestExt", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for capture_request_ext::StartCommit { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.runtime_checkpoint.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.CaptureRequestExt.StartCommit", len)?; + if let Some(v) = self.runtime_checkpoint.as_ref() { + struct_ser.serialize_field("runtimeCheckpoint", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for capture_request_ext::StartCommit { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "runtime_checkpoint", + "runtimeCheckpoint", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + RuntimeCheckpoint, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "runtimeCheckpoint" | "runtime_checkpoint" => Ok(GeneratedField::RuntimeCheckpoint), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = capture_request_ext::StartCommit; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CaptureRequestExt.StartCommit") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut runtime_checkpoint__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::RuntimeCheckpoint => { + if runtime_checkpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("runtimeCheckpoint")); + } + runtime_checkpoint__ = map_.next_value()?; + } + } + } + Ok(capture_request_ext::StartCommit { + runtime_checkpoint: runtime_checkpoint__, + }) + } + } + deserializer.deserialize_struct("runtime.CaptureRequestExt.StartCommit", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for CaptureResponseExt { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.container.is_some() { + len += 1; + } + if self.opened.is_some() { + len += 1; + } + if self.captured.is_some() { + len += 1; + } + if self.checkpoint.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.CaptureResponseExt", len)?; + if let Some(v) = self.container.as_ref() { + struct_ser.serialize_field("container", v)?; + } + if let Some(v) = self.opened.as_ref() { + struct_ser.serialize_field("opened", v)?; + } + if let Some(v) = self.captured.as_ref() { + struct_ser.serialize_field("captured", v)?; + } + if let Some(v) = self.checkpoint.as_ref() { + struct_ser.serialize_field("checkpoint", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CaptureResponseExt { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "container", + "opened", + "captured", + "checkpoint", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Container, + Opened, + Captured, + Checkpoint, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "container" => Ok(GeneratedField::Container), + "opened" => Ok(GeneratedField::Opened), + "captured" => Ok(GeneratedField::Captured), + "checkpoint" => Ok(GeneratedField::Checkpoint), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CaptureResponseExt; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CaptureResponseExt") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut container__ = None; + let mut opened__ = None; + let mut captured__ = None; + let mut checkpoint__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Container => { + if container__.is_some() { + return Err(serde::de::Error::duplicate_field("container")); + } + container__ = map_.next_value()?; + } + GeneratedField::Opened => { + if opened__.is_some() { + return Err(serde::de::Error::duplicate_field("opened")); + } + opened__ = map_.next_value()?; + } + GeneratedField::Captured => { + if captured__.is_some() { + return Err(serde::de::Error::duplicate_field("captured")); + } + captured__ = map_.next_value()?; + } + GeneratedField::Checkpoint => { + if checkpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("checkpoint")); + } + checkpoint__ = map_.next_value()?; + } + } + } + Ok(CaptureResponseExt { + container: container__, + opened: opened__, + captured: captured__, + checkpoint: checkpoint__, + }) + } + } + deserializer.deserialize_struct("runtime.CaptureResponseExt", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for capture_response_ext::Captured { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.key_packed.is_empty() { + len += 1; + } + if !self.partitions_packed.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.CaptureResponseExt.Captured", len)?; + if !self.key_packed.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("keyPacked", pbjson::private::base64::encode(&self.key_packed).as_str())?; + } + if !self.partitions_packed.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("partitionsPacked", pbjson::private::base64::encode(&self.partitions_packed).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for capture_response_ext::Captured { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "key_packed", + "keyPacked", + "partitions_packed", + "partitionsPacked", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + KeyPacked, + PartitionsPacked, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "keyPacked" | "key_packed" => Ok(GeneratedField::KeyPacked), + "partitionsPacked" | "partitions_packed" => Ok(GeneratedField::PartitionsPacked), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = capture_response_ext::Captured; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CaptureResponseExt.Captured") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut key_packed__ = None; + let mut partitions_packed__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::KeyPacked => { + if key_packed__.is_some() { + return Err(serde::de::Error::duplicate_field("keyPacked")); + } + key_packed__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + GeneratedField::PartitionsPacked => { + if partitions_packed__.is_some() { + return Err(serde::de::Error::duplicate_field("partitionsPacked")); + } + partitions_packed__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + } + } + Ok(capture_response_ext::Captured { + key_packed: key_packed__.unwrap_or_default(), + partitions_packed: partitions_packed__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.CaptureResponseExt.Captured", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for capture_response_ext::Checkpoint { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.stats.is_some() { + len += 1; + } + if self.poll_result != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.CaptureResponseExt.Checkpoint", len)?; + if let Some(v) = self.stats.as_ref() { + struct_ser.serialize_field("stats", v)?; + } + if self.poll_result != 0 { + let v = capture_response_ext::PollResult::try_from(self.poll_result) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.poll_result)))?; + struct_ser.serialize_field("pollResult", &v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for capture_response_ext::Checkpoint { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "stats", + "poll_result", + "pollResult", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Stats, + PollResult, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "stats" => Ok(GeneratedField::Stats), + "pollResult" | "poll_result" => Ok(GeneratedField::PollResult), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = capture_response_ext::Checkpoint; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CaptureResponseExt.Checkpoint") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut stats__ = None; + let mut poll_result__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Stats => { + if stats__.is_some() { + return Err(serde::de::Error::duplicate_field("stats")); + } + stats__ = map_.next_value()?; + } + GeneratedField::PollResult => { + if poll_result__.is_some() { + return Err(serde::de::Error::duplicate_field("pollResult")); + } + poll_result__ = Some(map_.next_value::()? as i32); + } + } + } + Ok(capture_response_ext::Checkpoint { + stats: stats__, + poll_result: poll_result__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.CaptureResponseExt.Checkpoint", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for capture_response_ext::Opened { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.runtime_checkpoint.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.CaptureResponseExt.Opened", len)?; + if let Some(v) = self.runtime_checkpoint.as_ref() { + struct_ser.serialize_field("runtimeCheckpoint", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for capture_response_ext::Opened { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "runtime_checkpoint", + "runtimeCheckpoint", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + RuntimeCheckpoint, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "runtimeCheckpoint" | "runtime_checkpoint" => Ok(GeneratedField::RuntimeCheckpoint), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = capture_response_ext::Opened; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CaptureResponseExt.Opened") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut runtime_checkpoint__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::RuntimeCheckpoint => { + if runtime_checkpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("runtimeCheckpoint")); + } + runtime_checkpoint__ = map_.next_value()?; + } + } + } + Ok(capture_response_ext::Opened { + runtime_checkpoint: runtime_checkpoint__, + }) + } + } + deserializer.deserialize_struct("runtime.CaptureResponseExt.Opened", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for capture_response_ext::PollResult { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + let variant = match self { + Self::Invalid => "INVALID", + Self::Ready => "READY", + Self::NotReady => "NOT_READY", + Self::CoolOff => "COOL_OFF", + Self::Restart => "RESTART", + }; + serializer.serialize_str(variant) + } +} +impl<'de> serde::Deserialize<'de> for capture_response_ext::PollResult { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "INVALID", + "READY", + "NOT_READY", + "COOL_OFF", + "RESTART", + ]; + + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = capture_response_ext::PollResult; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + fn visit_i64(self, v: i64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) + }) + } + + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "INVALID" => Ok(capture_response_ext::PollResult::Invalid), + "READY" => Ok(capture_response_ext::PollResult::Ready), + "NOT_READY" => Ok(capture_response_ext::PollResult::NotReady), + "COOL_OFF" => Ok(capture_response_ext::PollResult::CoolOff), + "RESTART" => Ok(capture_response_ext::PollResult::Restart), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} +impl serde::Serialize for CloseNow { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let len = 0; + let struct_ser = serializer.serialize_struct("runtime.CloseNow", len)?; + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CloseNow { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + Err(serde::de::Error::unknown_field(value, FIELDS)) + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CloseNow; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CloseNow") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + while map_.next_key::()?.is_some() { + let _ = map_.next_value::()?; + } + Ok(CloseNow { + }) + } + } + deserializer.deserialize_struct("runtime.CloseNow", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for CombineRequest { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.open.is_some() { + len += 1; + } + if self.add.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.CombineRequest", len)?; + if let Some(v) = self.open.as_ref() { + struct_ser.serialize_field("open", v)?; + } + if let Some(v) = self.add.as_ref() { + struct_ser.serialize_field("add", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CombineRequest { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "open", + "add", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Open, + Add, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "open" => Ok(GeneratedField::Open), + "add" => Ok(GeneratedField::Add), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CombineRequest; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CombineRequest") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut open__ = None; + let mut add__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Open => { + if open__.is_some() { + return Err(serde::de::Error::duplicate_field("open")); + } + open__ = map_.next_value()?; + } + GeneratedField::Add => { + if add__.is_some() { + return Err(serde::de::Error::duplicate_field("add")); + } + add__ = map_.next_value()?; + } + } + } + Ok(CombineRequest { + open: open__, + add: add__, + }) + } + } + deserializer.deserialize_struct("runtime.CombineRequest", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for combine_request::Add { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.binding != 0 { + len += 1; + } + if !self.doc_json.is_empty() { + len += 1; + } + if self.front { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.CombineRequest.Add", len)?; + if self.binding != 0 { + struct_ser.serialize_field("binding", &self.binding)?; + } + if !self.doc_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("docJson", &crate::as_raw_json(&self.doc_json)?)?; + } + if self.front { + struct_ser.serialize_field("front", &self.front)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for combine_request::Add { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "binding", + "doc_json", + "docJson", + "front", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Binding, + DocJson, + Front, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "binding" => Ok(GeneratedField::Binding), + "docJson" | "doc_json" => Ok(GeneratedField::DocJson), + "front" => Ok(GeneratedField::Front), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = combine_request::Add; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CombineRequest.Add") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut binding__ = None; + let mut doc_json__ = None; + let mut front__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Binding => { + if binding__.is_some() { + return Err(serde::de::Error::duplicate_field("binding")); + } + binding__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::DocJson => { + if doc_json__.is_some() { + return Err(serde::de::Error::duplicate_field("docJson")); + } + doc_json__ = + Some(map_.next_value::()?.0) + ; + } + GeneratedField::Front => { + if front__.is_some() { + return Err(serde::de::Error::duplicate_field("front")); + } + front__ = Some(map_.next_value()?); + } + } + } + Ok(combine_request::Add { + binding: binding__.unwrap_or_default(), + doc_json: doc_json__.unwrap_or_default(), + front: front__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.CombineRequest.Add", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for combine_request::Open { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.bindings.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.CombineRequest.Open", len)?; + if !self.bindings.is_empty() { + struct_ser.serialize_field("bindings", &self.bindings)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for combine_request::Open { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "bindings", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Bindings, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "bindings" => Ok(GeneratedField::Bindings), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = combine_request::Open; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CombineRequest.Open") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut bindings__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Bindings => { + if bindings__.is_some() { + return Err(serde::de::Error::duplicate_field("bindings")); + } + bindings__ = Some(map_.next_value()?); + } + } + } + Ok(combine_request::Open { + bindings: bindings__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.CombineRequest.Open", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for combine_request::open::Binding { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.full { + len += 1; + } + if !self.key.is_empty() { + len += 1; + } + if !self.projections.is_empty() { + len += 1; + } + if !self.schema_json.is_empty() { + len += 1; + } + if self.ser_policy.is_some() { + len += 1; + } + if !self.uuid_ptr.is_empty() { + len += 1; + } + if !self.values.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.CombineRequest.Open.Binding", len)?; + if self.full { + struct_ser.serialize_field("full", &self.full)?; + } + if !self.key.is_empty() { + struct_ser.serialize_field("key", &self.key)?; + } + if !self.projections.is_empty() { + struct_ser.serialize_field("projections", &self.projections)?; + } + if !self.schema_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("schemaJson", &crate::as_raw_json(&self.schema_json)?)?; + } + if let Some(v) = self.ser_policy.as_ref() { + struct_ser.serialize_field("serPolicy", v)?; + } + if !self.uuid_ptr.is_empty() { + struct_ser.serialize_field("uuidPtr", &self.uuid_ptr)?; + } + if !self.values.is_empty() { + struct_ser.serialize_field("values", &self.values)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for combine_request::open::Binding { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "full", + "key", + "projections", + "schema_json", + "schemaJson", + "ser_policy", + "serPolicy", + "uuid_ptr", + "uuidPtr", + "values", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Full, + Key, + Projections, + SchemaJson, + SerPolicy, + UuidPtr, + Values, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "full" => Ok(GeneratedField::Full), + "key" => Ok(GeneratedField::Key), + "projections" => Ok(GeneratedField::Projections), + "schemaJson" | "schema_json" => Ok(GeneratedField::SchemaJson), + "serPolicy" | "ser_policy" => Ok(GeneratedField::SerPolicy), + "uuidPtr" | "uuid_ptr" => Ok(GeneratedField::UuidPtr), + "values" => Ok(GeneratedField::Values), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = combine_request::open::Binding; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CombineRequest.Open.Binding") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut full__ = None; + let mut key__ = None; + let mut projections__ = None; + let mut schema_json__ = None; + let mut ser_policy__ = None; + let mut uuid_ptr__ = None; + let mut values__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Full => { + if full__.is_some() { + return Err(serde::de::Error::duplicate_field("full")); + } + full__ = Some(map_.next_value()?); + } + GeneratedField::Key => { + if key__.is_some() { + return Err(serde::de::Error::duplicate_field("key")); + } + key__ = Some(map_.next_value()?); + } + GeneratedField::Projections => { + if projections__.is_some() { + return Err(serde::de::Error::duplicate_field("projections")); + } + projections__ = Some(map_.next_value()?); + } + GeneratedField::SchemaJson => { + if schema_json__.is_some() { + return Err(serde::de::Error::duplicate_field("schemaJson")); + } + schema_json__ = + Some(map_.next_value::()?.0) + ; + } + GeneratedField::SerPolicy => { + if ser_policy__.is_some() { + return Err(serde::de::Error::duplicate_field("serPolicy")); + } + ser_policy__ = map_.next_value()?; + } + GeneratedField::UuidPtr => { + if uuid_ptr__.is_some() { + return Err(serde::de::Error::duplicate_field("uuidPtr")); + } + uuid_ptr__ = Some(map_.next_value()?); + } + GeneratedField::Values => { + if values__.is_some() { + return Err(serde::de::Error::duplicate_field("values")); + } + values__ = Some(map_.next_value()?); + } + } + } + Ok(combine_request::open::Binding { + full: full__.unwrap_or_default(), + key: key__.unwrap_or_default(), + projections: projections__.unwrap_or_default(), + schema_json: schema_json__.unwrap_or_default(), + ser_policy: ser_policy__, + uuid_ptr: uuid_ptr__.unwrap_or_default(), + values: values__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.CombineRequest.Open.Binding", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for CombineResponse { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.binding != 0 { + len += 1; + } + if self.deleted { + len += 1; + } + if !self.doc_json.is_empty() { + len += 1; + } + if self.front { + len += 1; + } + if !self.key_packed.is_empty() { + len += 1; + } + if !self.values_packed.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.CombineResponse", len)?; + if self.binding != 0 { + struct_ser.serialize_field("binding", &self.binding)?; + } + if self.deleted { + struct_ser.serialize_field("deleted", &self.deleted)?; + } + if !self.doc_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("docJson", &crate::as_raw_json(&self.doc_json)?)?; + } + if self.front { + struct_ser.serialize_field("front", &self.front)?; + } + if !self.key_packed.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("keyPacked", pbjson::private::base64::encode(&self.key_packed).as_str())?; + } + if !self.values_packed.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("valuesPacked", pbjson::private::base64::encode(&self.values_packed).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CombineResponse { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "binding", + "deleted", + "doc_json", + "docJson", + "front", + "key_packed", + "keyPacked", + "values_packed", + "valuesPacked", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Binding, + Deleted, + DocJson, + Front, + KeyPacked, + ValuesPacked, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "binding" => Ok(GeneratedField::Binding), + "deleted" => Ok(GeneratedField::Deleted), + "docJson" | "doc_json" => Ok(GeneratedField::DocJson), + "front" => Ok(GeneratedField::Front), + "keyPacked" | "key_packed" => Ok(GeneratedField::KeyPacked), + "valuesPacked" | "values_packed" => Ok(GeneratedField::ValuesPacked), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CombineResponse; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.CombineResponse") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut binding__ = None; + let mut deleted__ = None; + let mut doc_json__ = None; + let mut front__ = None; + let mut key_packed__ = None; + let mut values_packed__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Binding => { + if binding__.is_some() { + return Err(serde::de::Error::duplicate_field("binding")); + } + binding__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::Deleted => { + if deleted__.is_some() { + return Err(serde::de::Error::duplicate_field("deleted")); + } + deleted__ = Some(map_.next_value()?); + } + GeneratedField::DocJson => { + if doc_json__.is_some() { + return Err(serde::de::Error::duplicate_field("docJson")); + } + doc_json__ = + Some(map_.next_value::()?.0) + ; + } + GeneratedField::Front => { + if front__.is_some() { + return Err(serde::de::Error::duplicate_field("front")); + } + front__ = Some(map_.next_value()?); + } + GeneratedField::KeyPacked => { + if key_packed__.is_some() { + return Err(serde::de::Error::duplicate_field("keyPacked")); + } + key_packed__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + GeneratedField::ValuesPacked => { + if values_packed__.is_some() { + return Err(serde::de::Error::duplicate_field("valuesPacked")); + } + values_packed__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + } + } + Ok(CombineResponse { + binding: binding__.unwrap_or_default(), + deleted: deleted__.unwrap_or_default(), + doc_json: doc_json__.unwrap_or_default(), + front: front__.unwrap_or_default(), + key_packed: key_packed__.unwrap_or_default(), + values_packed: values_packed__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.CombineResponse", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ConnectorProxyRequest { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let len = 0; + let struct_ser = serializer.serialize_struct("runtime.ConnectorProxyRequest", len)?; + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ConnectorProxyRequest { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + Err(serde::de::Error::unknown_field(value, FIELDS)) + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ConnectorProxyRequest; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.ConnectorProxyRequest") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + while map_.next_key::()?.is_some() { + let _ = map_.next_value::()?; + } + Ok(ConnectorProxyRequest { + }) + } + } + deserializer.deserialize_struct("runtime.ConnectorProxyRequest", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ConnectorProxyResponse { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.address.is_empty() { + len += 1; + } + if !self.proxy_id.is_empty() { + len += 1; + } + if self.log.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.ConnectorProxyResponse", len)?; + if !self.address.is_empty() { + struct_ser.serialize_field("address", &self.address)?; + } + if !self.proxy_id.is_empty() { + struct_ser.serialize_field("proxyId", &self.proxy_id)?; + } + if let Some(v) = self.log.as_ref() { + struct_ser.serialize_field("log", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ConnectorProxyResponse { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "address", + "proxy_id", + "proxyId", + "log", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Address, + ProxyId, + Log, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "address" => Ok(GeneratedField::Address), + "proxyId" | "proxy_id" => Ok(GeneratedField::ProxyId), + "log" => Ok(GeneratedField::Log), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ConnectorProxyResponse; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.ConnectorProxyResponse") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut address__ = None; + let mut proxy_id__ = None; + let mut log__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Address => { + if address__.is_some() { + return Err(serde::de::Error::duplicate_field("address")); + } + address__ = Some(map_.next_value()?); + } + GeneratedField::ProxyId => { + if proxy_id__.is_some() { + return Err(serde::de::Error::duplicate_field("proxyId")); + } + proxy_id__ = Some(map_.next_value()?); + } + GeneratedField::Log => { + if log__.is_some() { + return Err(serde::de::Error::duplicate_field("log")); + } + log__ = map_.next_value()?; + } + } + } + Ok(ConnectorProxyResponse { + address: address__.unwrap_or_default(), + proxy_id: proxy_id__.unwrap_or_default(), + log: log__, + }) + } + } + deserializer.deserialize_struct("runtime.ConnectorProxyResponse", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Container { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.ip_addr.is_empty() { + len += 1; + } + if !self.network_ports.is_empty() { + len += 1; + } + if !self.mapped_host_ports.is_empty() { + len += 1; + } + if self.usage_rate != 0. { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Container", len)?; + if !self.ip_addr.is_empty() { + struct_ser.serialize_field("ipAddr", &self.ip_addr)?; + } + if !self.network_ports.is_empty() { + struct_ser.serialize_field("networkPorts", &self.network_ports)?; + } + if !self.mapped_host_ports.is_empty() { + struct_ser.serialize_field("mappedHostPorts", &self.mapped_host_ports)?; + } + if self.usage_rate != 0. { + struct_ser.serialize_field("usageRate", &self.usage_rate)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Container { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "ip_addr", + "ipAddr", + "network_ports", + "networkPorts", + "mapped_host_ports", + "mappedHostPorts", + "usage_rate", + "usageRate", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + IpAddr, + NetworkPorts, + MappedHostPorts, + UsageRate, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "ipAddr" | "ip_addr" => Ok(GeneratedField::IpAddr), + "networkPorts" | "network_ports" => Ok(GeneratedField::NetworkPorts), + "mappedHostPorts" | "mapped_host_ports" => Ok(GeneratedField::MappedHostPorts), + "usageRate" | "usage_rate" => Ok(GeneratedField::UsageRate), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Container; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Container") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut ip_addr__ = None; + let mut network_ports__ = None; + let mut mapped_host_ports__ = None; + let mut usage_rate__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::IpAddr => { + if ip_addr__.is_some() { + return Err(serde::de::Error::duplicate_field("ipAddr")); + } + ip_addr__ = Some(map_.next_value()?); + } + GeneratedField::NetworkPorts => { + if network_ports__.is_some() { + return Err(serde::de::Error::duplicate_field("networkPorts")); + } + network_ports__ = Some(map_.next_value()?); + } + GeneratedField::MappedHostPorts => { + if mapped_host_ports__.is_some() { + return Err(serde::de::Error::duplicate_field("mappedHostPorts")); + } + mapped_host_ports__ = Some( + map_.next_value::, _>>()? + .into_iter().map(|(k,v)| (k.0, v)).collect() + ); + } + GeneratedField::UsageRate => { + if usage_rate__.is_some() { + return Err(serde::de::Error::duplicate_field("usageRate")); + } + usage_rate__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(Container { + ip_addr: ip_addr__.unwrap_or_default(), + network_ports: network_ports__.unwrap_or_default(), + mapped_host_ports: mapped_host_ports__.unwrap_or_default(), + usage_rate: usage_rate__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Container", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for DeriveRequestExt { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.log_level != 0 { + len += 1; + } + if self.rocksdb_descriptor.is_some() { + len += 1; + } + if self.open.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.DeriveRequestExt", len)?; + if self.log_level != 0 { + let v = super::ops::log::Level::try_from(self.log_level) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.log_level)))?; + struct_ser.serialize_field("logLevel", &v)?; + } + if let Some(v) = self.rocksdb_descriptor.as_ref() { + struct_ser.serialize_field("rocksdbDescriptor", v)?; + } + if let Some(v) = self.open.as_ref() { + struct_ser.serialize_field("open", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for DeriveRequestExt { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "log_level", + "logLevel", + "rocksdb_descriptor", + "rocksdbDescriptor", + "open", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + LogLevel, + RocksdbDescriptor, + Open, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "logLevel" | "log_level" => Ok(GeneratedField::LogLevel), + "rocksdbDescriptor" | "rocksdb_descriptor" => Ok(GeneratedField::RocksdbDescriptor), + "open" => Ok(GeneratedField::Open), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = DeriveRequestExt; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.DeriveRequestExt") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut log_level__ = None; + let mut rocksdb_descriptor__ = None; + let mut open__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::LogLevel => { + if log_level__.is_some() { + return Err(serde::de::Error::duplicate_field("logLevel")); + } + log_level__ = Some(map_.next_value::()? as i32); + } + GeneratedField::RocksdbDescriptor => { + if rocksdb_descriptor__.is_some() { + return Err(serde::de::Error::duplicate_field("rocksdbDescriptor")); + } + rocksdb_descriptor__ = map_.next_value()?; + } + GeneratedField::Open => { + if open__.is_some() { + return Err(serde::de::Error::duplicate_field("open")); + } + open__ = map_.next_value()?; + } + } + } + Ok(DeriveRequestExt { + log_level: log_level__.unwrap_or_default(), + rocksdb_descriptor: rocksdb_descriptor__, + open: open__, + }) + } + } + deserializer.deserialize_struct("runtime.DeriveRequestExt", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for derive_request_ext::Open { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.sqlite_vfs_uri.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.DeriveRequestExt.Open", len)?; + if !self.sqlite_vfs_uri.is_empty() { + struct_ser.serialize_field("sqliteVfsUri", &self.sqlite_vfs_uri)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for derive_request_ext::Open { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "sqlite_vfs_uri", + "sqliteVfsUri", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + SqliteVfsUri, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "sqliteVfsUri" | "sqlite_vfs_uri" => Ok(GeneratedField::SqliteVfsUri), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = derive_request_ext::Open; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.DeriveRequestExt.Open") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut sqlite_vfs_uri__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::SqliteVfsUri => { + if sqlite_vfs_uri__.is_some() { + return Err(serde::de::Error::duplicate_field("sqliteVfsUri")); + } + sqlite_vfs_uri__ = Some(map_.next_value()?); + } + } + } + Ok(derive_request_ext::Open { + sqlite_vfs_uri: sqlite_vfs_uri__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.DeriveRequestExt.Open", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for DeriveResponseExt { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.container.is_some() { + len += 1; + } + if self.opened.is_some() { + len += 1; + } + if self.published.is_some() { + len += 1; + } + if self.flushed.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.DeriveResponseExt", len)?; + if let Some(v) = self.container.as_ref() { + struct_ser.serialize_field("container", v)?; + } + if let Some(v) = self.opened.as_ref() { + struct_ser.serialize_field("opened", v)?; + } + if let Some(v) = self.published.as_ref() { + struct_ser.serialize_field("published", v)?; + } + if let Some(v) = self.flushed.as_ref() { + struct_ser.serialize_field("flushed", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for DeriveResponseExt { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "container", + "opened", + "published", + "flushed", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Container, + Opened, + Published, + Flushed, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "container" => Ok(GeneratedField::Container), + "opened" => Ok(GeneratedField::Opened), + "published" => Ok(GeneratedField::Published), + "flushed" => Ok(GeneratedField::Flushed), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = DeriveResponseExt; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.DeriveResponseExt") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut container__ = None; + let mut opened__ = None; + let mut published__ = None; + let mut flushed__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Container => { + if container__.is_some() { + return Err(serde::de::Error::duplicate_field("container")); + } + container__ = map_.next_value()?; + } + GeneratedField::Opened => { + if opened__.is_some() { + return Err(serde::de::Error::duplicate_field("opened")); + } + opened__ = map_.next_value()?; + } + GeneratedField::Published => { + if published__.is_some() { + return Err(serde::de::Error::duplicate_field("published")); + } + published__ = map_.next_value()?; + } + GeneratedField::Flushed => { + if flushed__.is_some() { + return Err(serde::de::Error::duplicate_field("flushed")); + } + flushed__ = map_.next_value()?; + } + } + } + Ok(DeriveResponseExt { + container: container__, + opened: opened__, + published: published__, + flushed: flushed__, + }) + } + } + deserializer.deserialize_struct("runtime.DeriveResponseExt", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for derive_response_ext::Flushed { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.stats.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.DeriveResponseExt.Flushed", len)?; + if let Some(v) = self.stats.as_ref() { + struct_ser.serialize_field("stats", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for derive_response_ext::Flushed { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "stats", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Stats, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "stats" => Ok(GeneratedField::Stats), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = derive_response_ext::Flushed; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.DeriveResponseExt.Flushed") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut stats__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Stats => { + if stats__.is_some() { + return Err(serde::de::Error::duplicate_field("stats")); + } + stats__ = map_.next_value()?; + } + } + } + Ok(derive_response_ext::Flushed { + stats: stats__, + }) + } + } + deserializer.deserialize_struct("runtime.DeriveResponseExt.Flushed", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for derive_response_ext::Opened { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.runtime_checkpoint.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.DeriveResponseExt.Opened", len)?; + if let Some(v) = self.runtime_checkpoint.as_ref() { + struct_ser.serialize_field("runtimeCheckpoint", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for derive_response_ext::Opened { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "runtime_checkpoint", + "runtimeCheckpoint", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + RuntimeCheckpoint, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "runtimeCheckpoint" | "runtime_checkpoint" => Ok(GeneratedField::RuntimeCheckpoint), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = derive_response_ext::Opened; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.DeriveResponseExt.Opened") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut runtime_checkpoint__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::RuntimeCheckpoint => { + if runtime_checkpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("runtimeCheckpoint")); + } + runtime_checkpoint__ = map_.next_value()?; + } + } + } + Ok(derive_response_ext::Opened { + runtime_checkpoint: runtime_checkpoint__, + }) + } + } + deserializer.deserialize_struct("runtime.DeriveResponseExt.Opened", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for derive_response_ext::Published { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.max_clock != 0 { + len += 1; + } + if !self.key_packed.is_empty() { + len += 1; + } + if !self.partitions_packed.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.DeriveResponseExt.Published", len)?; + if self.max_clock != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("maxClock", ToString::to_string(&self.max_clock).as_str())?; + } + if !self.key_packed.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("keyPacked", pbjson::private::base64::encode(&self.key_packed).as_str())?; + } + if !self.partitions_packed.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("partitionsPacked", pbjson::private::base64::encode(&self.partitions_packed).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for derive_response_ext::Published { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "max_clock", + "maxClock", + "key_packed", + "keyPacked", + "partitions_packed", + "partitionsPacked", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + MaxClock, + KeyPacked, + PartitionsPacked, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "maxClock" | "max_clock" => Ok(GeneratedField::MaxClock), + "keyPacked" | "key_packed" => Ok(GeneratedField::KeyPacked), + "partitionsPacked" | "partitions_packed" => Ok(GeneratedField::PartitionsPacked), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = derive_response_ext::Published; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.DeriveResponseExt.Published") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut max_clock__ = None; + let mut key_packed__ = None; + let mut partitions_packed__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::MaxClock => { + if max_clock__.is_some() { + return Err(serde::de::Error::duplicate_field("maxClock")); + } + max_clock__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::KeyPacked => { + if key_packed__.is_some() { + return Err(serde::de::Error::duplicate_field("keyPacked")); + } + key_packed__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + GeneratedField::PartitionsPacked => { + if partitions_packed__.is_some() { + return Err(serde::de::Error::duplicate_field("partitionsPacked")); + } + partitions_packed__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + } + } + Ok(derive_response_ext::Published { + max_clock: max_clock__.unwrap_or_default(), + key_packed: key_packed__.unwrap_or_default(), + partitions_packed: partitions_packed__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.DeriveResponseExt.Published", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Join { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.etcd_mod_revision != 0 { + len += 1; + } + if !self.shards.is_empty() { + len += 1; + } + if self.shard_index != 0 { + len += 1; + } + if !self.shuffle_directory.is_empty() { + len += 1; + } + if !self.shuffle_endpoint.is_empty() { + len += 1; + } + if !self.leader_endpoint.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Join", len)?; + if self.etcd_mod_revision != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("etcdModRevision", ToString::to_string(&self.etcd_mod_revision).as_str())?; + } + if !self.shards.is_empty() { + struct_ser.serialize_field("shards", &self.shards)?; + } + if self.shard_index != 0 { + struct_ser.serialize_field("shardIndex", &self.shard_index)?; + } + if !self.shuffle_directory.is_empty() { + struct_ser.serialize_field("shuffleDirectory", &self.shuffle_directory)?; + } + if !self.shuffle_endpoint.is_empty() { + struct_ser.serialize_field("shuffleEndpoint", &self.shuffle_endpoint)?; + } + if !self.leader_endpoint.is_empty() { + struct_ser.serialize_field("leaderEndpoint", &self.leader_endpoint)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Join { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "etcd_mod_revision", + "etcdModRevision", + "shards", + "shard_index", + "shardIndex", + "shuffle_directory", + "shuffleDirectory", + "shuffle_endpoint", + "shuffleEndpoint", + "leader_endpoint", + "leaderEndpoint", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + EtcdModRevision, + Shards, + ShardIndex, + ShuffleDirectory, + ShuffleEndpoint, + LeaderEndpoint, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "etcdModRevision" | "etcd_mod_revision" => Ok(GeneratedField::EtcdModRevision), + "shards" => Ok(GeneratedField::Shards), + "shardIndex" | "shard_index" => Ok(GeneratedField::ShardIndex), + "shuffleDirectory" | "shuffle_directory" => Ok(GeneratedField::ShuffleDirectory), + "shuffleEndpoint" | "shuffle_endpoint" => Ok(GeneratedField::ShuffleEndpoint), + "leaderEndpoint" | "leader_endpoint" => Ok(GeneratedField::LeaderEndpoint), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Join; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Join") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut etcd_mod_revision__ = None; + let mut shards__ = None; + let mut shard_index__ = None; + let mut shuffle_directory__ = None; + let mut shuffle_endpoint__ = None; + let mut leader_endpoint__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::EtcdModRevision => { + if etcd_mod_revision__.is_some() { + return Err(serde::de::Error::duplicate_field("etcdModRevision")); + } + etcd_mod_revision__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::Shards => { + if shards__.is_some() { + return Err(serde::de::Error::duplicate_field("shards")); + } + shards__ = Some(map_.next_value()?); + } + GeneratedField::ShardIndex => { + if shard_index__.is_some() { + return Err(serde::de::Error::duplicate_field("shardIndex")); + } + shard_index__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::ShuffleDirectory => { + if shuffle_directory__.is_some() { + return Err(serde::de::Error::duplicate_field("shuffleDirectory")); + } + shuffle_directory__ = Some(map_.next_value()?); + } + GeneratedField::ShuffleEndpoint => { + if shuffle_endpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("shuffleEndpoint")); + } + shuffle_endpoint__ = Some(map_.next_value()?); + } + GeneratedField::LeaderEndpoint => { + if leader_endpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("leaderEndpoint")); + } + leader_endpoint__ = Some(map_.next_value()?); + } + } + } + Ok(Join { + etcd_mod_revision: etcd_mod_revision__.unwrap_or_default(), + shards: shards__.unwrap_or_default(), + shard_index: shard_index__.unwrap_or_default(), + shuffle_directory: shuffle_directory__.unwrap_or_default(), + shuffle_endpoint: shuffle_endpoint__.unwrap_or_default(), + leader_endpoint: leader_endpoint__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Join", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for join::Shard { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.id.is_empty() { + len += 1; + } + if self.labeling.is_some() { + len += 1; + } + if self.reactor.is_some() { + len += 1; + } + if self.etcd_create_revision != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Join.Shard", len)?; + if !self.id.is_empty() { + struct_ser.serialize_field("id", &self.id)?; + } + if let Some(v) = self.labeling.as_ref() { + struct_ser.serialize_field("labeling", v)?; + } + if let Some(v) = self.reactor.as_ref() { + struct_ser.serialize_field("reactor", v)?; + } + if self.etcd_create_revision != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("etcdCreateRevision", ToString::to_string(&self.etcd_create_revision).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for join::Shard { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "id", + "labeling", + "reactor", + "etcd_create_revision", + "etcdCreateRevision", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Id, + Labeling, + Reactor, + EtcdCreateRevision, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "id" => Ok(GeneratedField::Id), + "labeling" => Ok(GeneratedField::Labeling), + "reactor" => Ok(GeneratedField::Reactor), + "etcdCreateRevision" | "etcd_create_revision" => Ok(GeneratedField::EtcdCreateRevision), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = join::Shard; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Join.Shard") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut id__ = None; + let mut labeling__ = None; + let mut reactor__ = None; + let mut etcd_create_revision__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Id => { + if id__.is_some() { + return Err(serde::de::Error::duplicate_field("id")); + } + id__ = Some(map_.next_value()?); + } + GeneratedField::Labeling => { + if labeling__.is_some() { + return Err(serde::de::Error::duplicate_field("labeling")); + } + labeling__ = map_.next_value()?; + } + GeneratedField::Reactor => { + if reactor__.is_some() { + return Err(serde::de::Error::duplicate_field("reactor")); + } + reactor__ = map_.next_value()?; + } + GeneratedField::EtcdCreateRevision => { + if etcd_create_revision__.is_some() { + return Err(serde::de::Error::duplicate_field("etcdCreateRevision")); + } + etcd_create_revision__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(join::Shard { + id: id__.unwrap_or_default(), + labeling: labeling__, + reactor: reactor__, + etcd_create_revision: etcd_create_revision__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Join.Shard", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Joined { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.max_etcd_revision != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Joined", len)?; + if self.max_etcd_revision != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("maxEtcdRevision", ToString::to_string(&self.max_etcd_revision).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Joined { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "max_etcd_revision", + "maxEtcdRevision", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + MaxEtcdRevision, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "maxEtcdRevision" | "max_etcd_revision" => Ok(GeneratedField::MaxEtcdRevision), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Joined; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Joined") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut max_etcd_revision__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::MaxEtcdRevision => { + if max_etcd_revision__.is_some() { + return Err(serde::de::Error::duplicate_field("maxEtcdRevision")); + } + max_etcd_revision__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(Joined { + max_etcd_revision: max_etcd_revision__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Joined", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Materialize { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.spec.is_some() { + len += 1; + } + if self.spec_response.is_some() { + len += 1; + } + if self.validate.is_some() { + len += 1; + } + if self.validated.is_some() { + len += 1; + } + if self.session_loop.is_some() { + len += 1; + } + if self.join.is_some() { + len += 1; + } + if self.joined.is_some() { + len += 1; + } + if self.task.is_some() { + len += 1; + } + if self.recover.is_some() { + len += 1; + } + if self.apply.is_some() { + len += 1; + } + if self.applied.is_some() { + len += 1; + } + if self.open.is_some() { + len += 1; + } + if self.opened.is_some() { + len += 1; + } + if self.load.is_some() { + len += 1; + } + if self.loaded.is_some() { + len += 1; + } + if self.flush.is_some() { + len += 1; + } + if self.flushed.is_some() { + len += 1; + } + if self.store.is_some() { + len += 1; + } + if self.stored.is_some() { + len += 1; + } + if self.start_commit.is_some() { + len += 1; + } + if self.started_commit.is_some() { + len += 1; + } + if self.acknowledge.is_some() { + len += 1; + } + if self.acknowledged.is_some() { + len += 1; + } + if self.persist.is_some() { + len += 1; + } + if self.persisted.is_some() { + len += 1; + } + if self.close_now.is_some() { + len += 1; + } + if self.stop.is_some() { + len += 1; + } + if self.stopped.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize", len)?; + if let Some(v) = self.spec.as_ref() { + struct_ser.serialize_field("spec", v)?; + } + if let Some(v) = self.spec_response.as_ref() { + struct_ser.serialize_field("specResponse", v)?; + } + if let Some(v) = self.validate.as_ref() { + struct_ser.serialize_field("validate", v)?; + } + if let Some(v) = self.validated.as_ref() { + struct_ser.serialize_field("validated", v)?; + } + if let Some(v) = self.session_loop.as_ref() { + struct_ser.serialize_field("sessionLoop", v)?; + } + if let Some(v) = self.join.as_ref() { + struct_ser.serialize_field("join", v)?; + } + if let Some(v) = self.joined.as_ref() { + struct_ser.serialize_field("joined", v)?; + } + if let Some(v) = self.task.as_ref() { + struct_ser.serialize_field("task", v)?; + } + if let Some(v) = self.recover.as_ref() { + struct_ser.serialize_field("recover", v)?; + } + if let Some(v) = self.apply.as_ref() { + struct_ser.serialize_field("apply", v)?; + } + if let Some(v) = self.applied.as_ref() { + struct_ser.serialize_field("applied", v)?; + } + if let Some(v) = self.open.as_ref() { + struct_ser.serialize_field("open", v)?; + } + if let Some(v) = self.opened.as_ref() { + struct_ser.serialize_field("opened", v)?; + } + if let Some(v) = self.load.as_ref() { + struct_ser.serialize_field("load", v)?; + } + if let Some(v) = self.loaded.as_ref() { + struct_ser.serialize_field("loaded", v)?; + } + if let Some(v) = self.flush.as_ref() { + struct_ser.serialize_field("flush", v)?; + } + if let Some(v) = self.flushed.as_ref() { + struct_ser.serialize_field("flushed", v)?; + } + if let Some(v) = self.store.as_ref() { + struct_ser.serialize_field("store", v)?; + } + if let Some(v) = self.stored.as_ref() { + struct_ser.serialize_field("stored", v)?; + } + if let Some(v) = self.start_commit.as_ref() { + struct_ser.serialize_field("startCommit", v)?; + } + if let Some(v) = self.started_commit.as_ref() { + struct_ser.serialize_field("startedCommit", v)?; + } + if let Some(v) = self.acknowledge.as_ref() { + struct_ser.serialize_field("acknowledge", v)?; + } + if let Some(v) = self.acknowledged.as_ref() { + struct_ser.serialize_field("acknowledged", v)?; + } + if let Some(v) = self.persist.as_ref() { + struct_ser.serialize_field("persist", v)?; + } + if let Some(v) = self.persisted.as_ref() { + struct_ser.serialize_field("persisted", v)?; + } + if let Some(v) = self.close_now.as_ref() { + struct_ser.serialize_field("closeNow", v)?; + } + if let Some(v) = self.stop.as_ref() { + struct_ser.serialize_field("stop", v)?; + } + if let Some(v) = self.stopped.as_ref() { + struct_ser.serialize_field("stopped", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Materialize { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "spec", + "spec_response", + "specResponse", + "validate", + "validated", + "session_loop", + "sessionLoop", + "join", + "joined", + "task", + "recover", + "apply", + "applied", + "open", + "opened", + "load", + "loaded", + "flush", + "flushed", + "store", + "stored", + "start_commit", + "startCommit", + "started_commit", + "startedCommit", + "acknowledge", + "acknowledged", + "persist", + "persisted", + "close_now", + "closeNow", + "stop", + "stopped", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Spec, + SpecResponse, + Validate, + Validated, + SessionLoop, + Join, + Joined, + Task, + Recover, + Apply, + Applied, + Open, + Opened, + Load, + Loaded, + Flush, + Flushed, + Store, + Stored, + StartCommit, + StartedCommit, + Acknowledge, + Acknowledged, + Persist, + Persisted, + CloseNow, + Stop, + Stopped, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "spec" => Ok(GeneratedField::Spec), + "specResponse" | "spec_response" => Ok(GeneratedField::SpecResponse), + "validate" => Ok(GeneratedField::Validate), + "validated" => Ok(GeneratedField::Validated), + "sessionLoop" | "session_loop" => Ok(GeneratedField::SessionLoop), + "join" => Ok(GeneratedField::Join), + "joined" => Ok(GeneratedField::Joined), + "task" => Ok(GeneratedField::Task), + "recover" => Ok(GeneratedField::Recover), + "apply" => Ok(GeneratedField::Apply), + "applied" => Ok(GeneratedField::Applied), + "open" => Ok(GeneratedField::Open), + "opened" => Ok(GeneratedField::Opened), + "load" => Ok(GeneratedField::Load), + "loaded" => Ok(GeneratedField::Loaded), + "flush" => Ok(GeneratedField::Flush), + "flushed" => Ok(GeneratedField::Flushed), + "store" => Ok(GeneratedField::Store), + "stored" => Ok(GeneratedField::Stored), + "startCommit" | "start_commit" => Ok(GeneratedField::StartCommit), + "startedCommit" | "started_commit" => Ok(GeneratedField::StartedCommit), + "acknowledge" => Ok(GeneratedField::Acknowledge), + "acknowledged" => Ok(GeneratedField::Acknowledged), + "persist" => Ok(GeneratedField::Persist), + "persisted" => Ok(GeneratedField::Persisted), + "closeNow" | "close_now" => Ok(GeneratedField::CloseNow), + "stop" => Ok(GeneratedField::Stop), + "stopped" => Ok(GeneratedField::Stopped), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Materialize; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut spec__ = None; + let mut spec_response__ = None; + let mut validate__ = None; + let mut validated__ = None; + let mut session_loop__ = None; + let mut join__ = None; + let mut joined__ = None; + let mut task__ = None; + let mut recover__ = None; + let mut apply__ = None; + let mut applied__ = None; + let mut open__ = None; + let mut opened__ = None; + let mut load__ = None; + let mut loaded__ = None; + let mut flush__ = None; + let mut flushed__ = None; + let mut store__ = None; + let mut stored__ = None; + let mut start_commit__ = None; + let mut started_commit__ = None; + let mut acknowledge__ = None; + let mut acknowledged__ = None; + let mut persist__ = None; + let mut persisted__ = None; + let mut close_now__ = None; + let mut stop__ = None; + let mut stopped__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Spec => { + if spec__.is_some() { + return Err(serde::de::Error::duplicate_field("spec")); + } + spec__ = map_.next_value()?; + } + GeneratedField::SpecResponse => { + if spec_response__.is_some() { + return Err(serde::de::Error::duplicate_field("specResponse")); + } + spec_response__ = map_.next_value()?; + } + GeneratedField::Validate => { + if validate__.is_some() { + return Err(serde::de::Error::duplicate_field("validate")); + } + validate__ = map_.next_value()?; + } + GeneratedField::Validated => { + if validated__.is_some() { + return Err(serde::de::Error::duplicate_field("validated")); + } + validated__ = map_.next_value()?; + } + GeneratedField::SessionLoop => { + if session_loop__.is_some() { + return Err(serde::de::Error::duplicate_field("sessionLoop")); + } + session_loop__ = map_.next_value()?; + } + GeneratedField::Join => { + if join__.is_some() { + return Err(serde::de::Error::duplicate_field("join")); + } + join__ = map_.next_value()?; + } + GeneratedField::Joined => { + if joined__.is_some() { + return Err(serde::de::Error::duplicate_field("joined")); + } + joined__ = map_.next_value()?; + } + GeneratedField::Task => { + if task__.is_some() { + return Err(serde::de::Error::duplicate_field("task")); + } + task__ = map_.next_value()?; + } + GeneratedField::Recover => { + if recover__.is_some() { + return Err(serde::de::Error::duplicate_field("recover")); + } + recover__ = map_.next_value()?; + } + GeneratedField::Apply => { + if apply__.is_some() { + return Err(serde::de::Error::duplicate_field("apply")); + } + apply__ = map_.next_value()?; + } + GeneratedField::Applied => { + if applied__.is_some() { + return Err(serde::de::Error::duplicate_field("applied")); + } + applied__ = map_.next_value()?; + } + GeneratedField::Open => { + if open__.is_some() { + return Err(serde::de::Error::duplicate_field("open")); + } + open__ = map_.next_value()?; + } + GeneratedField::Opened => { + if opened__.is_some() { + return Err(serde::de::Error::duplicate_field("opened")); + } + opened__ = map_.next_value()?; + } + GeneratedField::Load => { + if load__.is_some() { + return Err(serde::de::Error::duplicate_field("load")); + } + load__ = map_.next_value()?; + } + GeneratedField::Loaded => { + if loaded__.is_some() { + return Err(serde::de::Error::duplicate_field("loaded")); + } + loaded__ = map_.next_value()?; + } + GeneratedField::Flush => { + if flush__.is_some() { + return Err(serde::de::Error::duplicate_field("flush")); + } + flush__ = map_.next_value()?; + } + GeneratedField::Flushed => { + if flushed__.is_some() { + return Err(serde::de::Error::duplicate_field("flushed")); + } + flushed__ = map_.next_value()?; + } + GeneratedField::Store => { + if store__.is_some() { + return Err(serde::de::Error::duplicate_field("store")); + } + store__ = map_.next_value()?; + } + GeneratedField::Stored => { + if stored__.is_some() { + return Err(serde::de::Error::duplicate_field("stored")); + } + stored__ = map_.next_value()?; + } + GeneratedField::StartCommit => { + if start_commit__.is_some() { + return Err(serde::de::Error::duplicate_field("startCommit")); + } + start_commit__ = map_.next_value()?; + } + GeneratedField::StartedCommit => { + if started_commit__.is_some() { + return Err(serde::de::Error::duplicate_field("startedCommit")); + } + started_commit__ = map_.next_value()?; + } + GeneratedField::Acknowledge => { + if acknowledge__.is_some() { + return Err(serde::de::Error::duplicate_field("acknowledge")); + } + acknowledge__ = map_.next_value()?; + } + GeneratedField::Acknowledged => { + if acknowledged__.is_some() { + return Err(serde::de::Error::duplicate_field("acknowledged")); + } + acknowledged__ = map_.next_value()?; + } + GeneratedField::Persist => { + if persist__.is_some() { + return Err(serde::de::Error::duplicate_field("persist")); + } + persist__ = map_.next_value()?; + } + GeneratedField::Persisted => { + if persisted__.is_some() { + return Err(serde::de::Error::duplicate_field("persisted")); + } + persisted__ = map_.next_value()?; + } + GeneratedField::CloseNow => { + if close_now__.is_some() { + return Err(serde::de::Error::duplicate_field("closeNow")); + } + close_now__ = map_.next_value()?; + } + GeneratedField::Stop => { + if stop__.is_some() { + return Err(serde::de::Error::duplicate_field("stop")); + } + stop__ = map_.next_value()?; + } + GeneratedField::Stopped => { + if stopped__.is_some() { + return Err(serde::de::Error::duplicate_field("stopped")); + } + stopped__ = map_.next_value()?; + } + } + } + Ok(Materialize { + spec: spec__, + spec_response: spec_response__, + validate: validate__, + validated: validated__, + session_loop: session_loop__, + join: join__, + joined: joined__, + task: task__, + recover: recover__, + apply: apply__, + applied: applied__, + open: open__, + opened: opened__, + load: load__, + loaded: loaded__, + flush: flush__, + flushed: flushed__, + store: store__, + stored: stored__, + start_commit: start_commit__, + started_commit: started_commit__, + acknowledge: acknowledge__, + acknowledged: acknowledged__, + persist: persist__, + persisted: persisted__, + close_now: close_now__, + stop: stop__, + stopped: stopped__, + }) + } + } + deserializer.deserialize_struct("runtime.Materialize", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::Acknowledge { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.connector_patches_json.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.Acknowledge", len)?; + if !self.connector_patches_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorPatches", &crate::as_raw_json(&self.connector_patches_json)?)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::Acknowledge { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "connector_patches_json", + "connectorPatches", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + ConnectorPatchesJson, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "connectorPatches" | "connector_patches_json" => Ok(GeneratedField::ConnectorPatchesJson), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::Acknowledge; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Acknowledge") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut connector_patches_json__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::ConnectorPatchesJson => { + if connector_patches_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorPatches")); + } + connector_patches_json__ = + Some(map_.next_value::()?.0) + ; + } + } + } + Ok(materialize::Acknowledge { + connector_patches_json: connector_patches_json__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Acknowledge", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::Acknowledged { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.connector_patches_json.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.Acknowledged", len)?; + if !self.connector_patches_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorPatches", &crate::as_raw_json(&self.connector_patches_json)?)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::Acknowledged { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "connector_patches_json", + "connectorPatches", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + ConnectorPatchesJson, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "connectorPatches" | "connector_patches_json" => Ok(GeneratedField::ConnectorPatchesJson), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::Acknowledged; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Acknowledged") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut connector_patches_json__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::ConnectorPatchesJson => { + if connector_patches_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorPatches")); + } + connector_patches_json__ = + Some(map_.next_value::()?.0) + ; + } + } + } + Ok(materialize::Acknowledged { + connector_patches_json: connector_patches_json__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Acknowledged", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::Flush { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.connector_patches_json.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.Flush", len)?; + if !self.connector_patches_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorPatches", &crate::as_raw_json(&self.connector_patches_json)?)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::Flush { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "connector_patches_json", + "connectorPatches", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + ConnectorPatchesJson, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "connectorPatches" | "connector_patches_json" => Ok(GeneratedField::ConnectorPatchesJson), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::Flush; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Flush") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut connector_patches_json__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::ConnectorPatchesJson => { + if connector_patches_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorPatches")); + } + connector_patches_json__ = + Some(map_.next_value::()?.0) + ; + } + } + } + Ok(materialize::Flush { + connector_patches_json: connector_patches_json__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Flush", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::Flushed { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.bindings.is_empty() { + len += 1; + } + if !self.connector_patches_json.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.Flushed", len)?; + if !self.bindings.is_empty() { + struct_ser.serialize_field("bindings", &self.bindings)?; + } + if !self.connector_patches_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorPatches", &crate::as_raw_json(&self.connector_patches_json)?)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::Flushed { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "bindings", + "connector_patches_json", + "connectorPatches", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Bindings, + ConnectorPatchesJson, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "bindings" => Ok(GeneratedField::Bindings), + "connectorPatches" | "connector_patches_json" => Ok(GeneratedField::ConnectorPatchesJson), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::Flushed; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Flushed") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut bindings__ = None; + let mut connector_patches_json__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Bindings => { + if bindings__.is_some() { + return Err(serde::de::Error::duplicate_field("bindings")); + } + bindings__ = Some(map_.next_value()?); + } + GeneratedField::ConnectorPatchesJson => { + if connector_patches_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorPatches")); + } + connector_patches_json__ = + Some(map_.next_value::()?.0) + ; + } + } + } + Ok(materialize::Flushed { + bindings: bindings__.unwrap_or_default(), + connector_patches_json: connector_patches_json__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Flushed", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::flushed::Binding { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.index != 0 { + len += 1; + } + if self.loaded_docs_total != 0 { + len += 1; + } + if self.loaded_bytes_total != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.Flushed.Binding", len)?; + if self.index != 0 { + struct_ser.serialize_field("index", &self.index)?; + } + if self.loaded_docs_total != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("loadedDocsTotal", ToString::to_string(&self.loaded_docs_total).as_str())?; + } + if self.loaded_bytes_total != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("loadedBytesTotal", ToString::to_string(&self.loaded_bytes_total).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::flushed::Binding { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "index", + "loaded_docs_total", + "loadedDocsTotal", + "loaded_bytes_total", + "loadedBytesTotal", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Index, + LoadedDocsTotal, + LoadedBytesTotal, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "index" => Ok(GeneratedField::Index), + "loadedDocsTotal" | "loaded_docs_total" => Ok(GeneratedField::LoadedDocsTotal), + "loadedBytesTotal" | "loaded_bytes_total" => Ok(GeneratedField::LoadedBytesTotal), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::flushed::Binding; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Flushed.Binding") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut index__ = None; + let mut loaded_docs_total__ = None; + let mut loaded_bytes_total__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Index => { + if index__.is_some() { + return Err(serde::de::Error::duplicate_field("index")); + } + index__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::LoadedDocsTotal => { + if loaded_docs_total__.is_some() { + return Err(serde::de::Error::duplicate_field("loadedDocsTotal")); + } + loaded_docs_total__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::LoadedBytesTotal => { + if loaded_bytes_total__.is_some() { + return Err(serde::de::Error::duplicate_field("loadedBytesTotal")); + } + loaded_bytes_total__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(materialize::flushed::Binding { + index: index__.unwrap_or_default(), + loaded_docs_total: loaded_docs_total__.unwrap_or_default(), + loaded_bytes_total: loaded_bytes_total__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Flushed.Binding", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::Load { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.frontier.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.Load", len)?; + if let Some(v) = self.frontier.as_ref() { + struct_ser.serialize_field("frontier", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::Load { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "frontier", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Frontier, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "frontier" => Ok(GeneratedField::Frontier), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::Load; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Load") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut frontier__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Frontier => { + if frontier__.is_some() { + return Err(serde::de::Error::duplicate_field("frontier")); + } + frontier__ = map_.next_value()?; + } + } + } + Ok(materialize::Load { + frontier: frontier__, + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Load", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::Loaded { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.bindings.is_empty() { + len += 1; + } + if self.combiner_usage_bytes != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.Loaded", len)?; + if !self.bindings.is_empty() { + struct_ser.serialize_field("bindings", &self.bindings)?; + } + if self.combiner_usage_bytes != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("combinerUsageBytes", ToString::to_string(&self.combiner_usage_bytes).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::Loaded { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "bindings", + "combiner_usage_bytes", + "combinerUsageBytes", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Bindings, + CombinerUsageBytes, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "bindings" => Ok(GeneratedField::Bindings), + "combinerUsageBytes" | "combiner_usage_bytes" => Ok(GeneratedField::CombinerUsageBytes), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::Loaded; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Loaded") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut bindings__ = None; + let mut combiner_usage_bytes__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Bindings => { + if bindings__.is_some() { + return Err(serde::de::Error::duplicate_field("bindings")); + } + bindings__ = Some(map_.next_value()?); + } + GeneratedField::CombinerUsageBytes => { + if combiner_usage_bytes__.is_some() { + return Err(serde::de::Error::duplicate_field("combinerUsageBytes")); + } + combiner_usage_bytes__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(materialize::Loaded { + bindings: bindings__.unwrap_or_default(), + combiner_usage_bytes: combiner_usage_bytes__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Loaded", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::loaded::Binding { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.index != 0 { + len += 1; + } + if self.min_source_clock != 0 { + len += 1; + } + if self.max_source_clock != 0 { + len += 1; + } + if self.sourced_docs_total != 0 { + len += 1; + } + if self.sourced_bytes_total != 0 { + len += 1; + } + if !self.max_key_delta.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.Loaded.Binding", len)?; + if self.index != 0 { + struct_ser.serialize_field("index", &self.index)?; + } + if self.min_source_clock != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("minSourceClock", ToString::to_string(&self.min_source_clock).as_str())?; + } + if self.max_source_clock != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("maxSourceClock", ToString::to_string(&self.max_source_clock).as_str())?; + } + if self.sourced_docs_total != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("sourcedDocsTotal", ToString::to_string(&self.sourced_docs_total).as_str())?; + } + if self.sourced_bytes_total != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("sourcedBytesTotal", ToString::to_string(&self.sourced_bytes_total).as_str())?; + } + if !self.max_key_delta.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("maxKeyDelta", pbjson::private::base64::encode(&self.max_key_delta).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::loaded::Binding { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "index", + "min_source_clock", + "minSourceClock", + "max_source_clock", + "maxSourceClock", + "sourced_docs_total", + "sourcedDocsTotal", + "sourced_bytes_total", + "sourcedBytesTotal", + "max_key_delta", + "maxKeyDelta", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Index, + MinSourceClock, + MaxSourceClock, + SourcedDocsTotal, + SourcedBytesTotal, + MaxKeyDelta, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "index" => Ok(GeneratedField::Index), + "minSourceClock" | "min_source_clock" => Ok(GeneratedField::MinSourceClock), + "maxSourceClock" | "max_source_clock" => Ok(GeneratedField::MaxSourceClock), + "sourcedDocsTotal" | "sourced_docs_total" => Ok(GeneratedField::SourcedDocsTotal), + "sourcedBytesTotal" | "sourced_bytes_total" => Ok(GeneratedField::SourcedBytesTotal), + "maxKeyDelta" | "max_key_delta" => Ok(GeneratedField::MaxKeyDelta), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::loaded::Binding; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Loaded.Binding") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut index__ = None; + let mut min_source_clock__ = None; + let mut max_source_clock__ = None; + let mut sourced_docs_total__ = None; + let mut sourced_bytes_total__ = None; + let mut max_key_delta__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Index => { + if index__.is_some() { + return Err(serde::de::Error::duplicate_field("index")); + } + index__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::MinSourceClock => { + if min_source_clock__.is_some() { + return Err(serde::de::Error::duplicate_field("minSourceClock")); + } + min_source_clock__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::MaxSourceClock => { + if max_source_clock__.is_some() { + return Err(serde::de::Error::duplicate_field("maxSourceClock")); + } + max_source_clock__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::SourcedDocsTotal => { + if sourced_docs_total__.is_some() { + return Err(serde::de::Error::duplicate_field("sourcedDocsTotal")); + } + sourced_docs_total__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::SourcedBytesTotal => { + if sourced_bytes_total__.is_some() { + return Err(serde::de::Error::duplicate_field("sourcedBytesTotal")); + } + sourced_bytes_total__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::MaxKeyDelta => { + if max_key_delta__.is_some() { + return Err(serde::de::Error::duplicate_field("maxKeyDelta")); + } + max_key_delta__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + } + } + Ok(materialize::loaded::Binding { + index: index__.unwrap_or_default(), + min_source_clock: min_source_clock__.unwrap_or_default(), + max_source_clock: max_source_clock__.unwrap_or_default(), + sourced_docs_total: sourced_docs_total__.unwrap_or_default(), + sourced_bytes_total: sourced_bytes_total__.unwrap_or_default(), + max_key_delta: max_key_delta__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Loaded.Binding", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::Opened { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.container.is_some() { + len += 1; + } + if self.connector_checkpoint.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.Opened", len)?; + if let Some(v) = self.container.as_ref() { + struct_ser.serialize_field("container", v)?; + } + if let Some(v) = self.connector_checkpoint.as_ref() { + struct_ser.serialize_field("connectorCheckpoint", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::Opened { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "container", + "connector_checkpoint", + "connectorCheckpoint", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Container, + ConnectorCheckpoint, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "container" => Ok(GeneratedField::Container), + "connectorCheckpoint" | "connector_checkpoint" => Ok(GeneratedField::ConnectorCheckpoint), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::Opened; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Opened") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut container__ = None; + let mut connector_checkpoint__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Container => { + if container__.is_some() { + return Err(serde::de::Error::duplicate_field("container")); + } + container__ = map_.next_value()?; + } + GeneratedField::ConnectorCheckpoint => { + if connector_checkpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorCheckpoint")); + } + connector_checkpoint__ = map_.next_value()?; + } + } + } + Ok(materialize::Opened { + container: container__, + connector_checkpoint: connector_checkpoint__, + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Opened", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::StartCommit { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.connector_patches_json.is_empty() { + len += 1; + } + if self.connector_checkpoint.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.StartCommit", len)?; + if !self.connector_patches_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorPatches", &crate::as_raw_json(&self.connector_patches_json)?)?; + } + if let Some(v) = self.connector_checkpoint.as_ref() { + struct_ser.serialize_field("connectorCheckpoint", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::StartCommit { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "connector_patches_json", + "connectorPatches", + "connector_checkpoint", + "connectorCheckpoint", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + ConnectorPatchesJson, + ConnectorCheckpoint, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "connectorPatches" | "connector_patches_json" => Ok(GeneratedField::ConnectorPatchesJson), + "connectorCheckpoint" | "connector_checkpoint" => Ok(GeneratedField::ConnectorCheckpoint), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::StartCommit; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.StartCommit") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut connector_patches_json__ = None; + let mut connector_checkpoint__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::ConnectorPatchesJson => { + if connector_patches_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorPatches")); + } + connector_patches_json__ = + Some(map_.next_value::()?.0) + ; + } + GeneratedField::ConnectorCheckpoint => { + if connector_checkpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorCheckpoint")); + } + connector_checkpoint__ = map_.next_value()?; + } + } + } + Ok(materialize::StartCommit { + connector_patches_json: connector_patches_json__.unwrap_or_default(), + connector_checkpoint: connector_checkpoint__, + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.StartCommit", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::StartedCommit { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.connector_patches_json.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.StartedCommit", len)?; + if !self.connector_patches_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorPatches", &crate::as_raw_json(&self.connector_patches_json)?)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::StartedCommit { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "connector_patches_json", + "connectorPatches", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + ConnectorPatchesJson, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "connectorPatches" | "connector_patches_json" => Ok(GeneratedField::ConnectorPatchesJson), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::StartedCommit; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.StartedCommit") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut connector_patches_json__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::ConnectorPatchesJson => { + if connector_patches_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorPatches")); + } + connector_patches_json__ = + Some(map_.next_value::()?.0) + ; + } + } + } + Ok(materialize::StartedCommit { + connector_patches_json: connector_patches_json__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.StartedCommit", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::Store { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let len = 0; + let struct_ser = serializer.serialize_struct("runtime.Materialize.Store", len)?; + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::Store { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + Err(serde::de::Error::unknown_field(value, FIELDS)) + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::Store; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Store") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + while map_.next_key::()?.is_some() { + let _ = map_.next_value::()?; + } + Ok(materialize::Store { + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Store", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::Stored { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.bindings.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.Stored", len)?; + if !self.bindings.is_empty() { + struct_ser.serialize_field("bindings", &self.bindings)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::Stored { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "bindings", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Bindings, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "bindings" => Ok(GeneratedField::Bindings), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::Stored; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Stored") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut bindings__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Bindings => { + if bindings__.is_some() { + return Err(serde::de::Error::duplicate_field("bindings")); + } + bindings__ = Some(map_.next_value()?); + } + } + } + Ok(materialize::Stored { + bindings: bindings__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Stored", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize::stored::Binding { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.index != 0 { + len += 1; + } + if self.stored_docs_total != 0 { + len += 1; + } + if self.stored_bytes_total != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Materialize.Stored.Binding", len)?; + if self.index != 0 { + struct_ser.serialize_field("index", &self.index)?; + } + if self.stored_docs_total != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("storedDocsTotal", ToString::to_string(&self.stored_docs_total).as_str())?; + } + if self.stored_bytes_total != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("storedBytesTotal", ToString::to_string(&self.stored_bytes_total).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize::stored::Binding { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "index", + "stored_docs_total", + "storedDocsTotal", + "stored_bytes_total", + "storedBytesTotal", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Index, + StoredDocsTotal, + StoredBytesTotal, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "index" => Ok(GeneratedField::Index), + "storedDocsTotal" | "stored_docs_total" => Ok(GeneratedField::StoredDocsTotal), + "storedBytesTotal" | "stored_bytes_total" => Ok(GeneratedField::StoredBytesTotal), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize::stored::Binding; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Materialize.Stored.Binding") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut index__ = None; + let mut stored_docs_total__ = None; + let mut stored_bytes_total__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Index => { + if index__.is_some() { + return Err(serde::de::Error::duplicate_field("index")); + } + index__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::StoredDocsTotal => { + if stored_docs_total__.is_some() { + return Err(serde::de::Error::duplicate_field("storedDocsTotal")); + } + stored_docs_total__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::StoredBytesTotal => { + if stored_bytes_total__.is_some() { + return Err(serde::de::Error::duplicate_field("storedBytesTotal")); + } + stored_bytes_total__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(materialize::stored::Binding { + index: index__.unwrap_or_default(), + stored_docs_total: stored_docs_total__.unwrap_or_default(), + stored_bytes_total: stored_bytes_total__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Materialize.Stored.Binding", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for MaterializeRequestExt { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.log_level != 0 { + len += 1; + } + if self.rocksdb_descriptor.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.MaterializeRequestExt", len)?; + if self.log_level != 0 { + let v = super::ops::log::Level::try_from(self.log_level) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.log_level)))?; + struct_ser.serialize_field("logLevel", &v)?; + } + if let Some(v) = self.rocksdb_descriptor.as_ref() { + struct_ser.serialize_field("rocksdbDescriptor", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for MaterializeRequestExt { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "log_level", + "logLevel", + "rocksdb_descriptor", + "rocksdbDescriptor", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + LogLevel, + RocksdbDescriptor, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "logLevel" | "log_level" => Ok(GeneratedField::LogLevel), + "rocksdbDescriptor" | "rocksdb_descriptor" => Ok(GeneratedField::RocksdbDescriptor), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MaterializeRequestExt; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.MaterializeRequestExt") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut log_level__ = None; + let mut rocksdb_descriptor__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::LogLevel => { + if log_level__.is_some() { + return Err(serde::de::Error::duplicate_field("logLevel")); + } + log_level__ = Some(map_.next_value::()? as i32); + } + GeneratedField::RocksdbDescriptor => { + if rocksdb_descriptor__.is_some() { + return Err(serde::de::Error::duplicate_field("rocksdbDescriptor")); + } + rocksdb_descriptor__ = map_.next_value()?; + } + } + } + Ok(MaterializeRequestExt { + log_level: log_level__.unwrap_or_default(), + rocksdb_descriptor: rocksdb_descriptor__, + }) + } + } + deserializer.deserialize_struct("runtime.MaterializeRequestExt", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for MaterializeResponseExt { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.container.is_some() { + len += 1; + } + if self.flushed.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.MaterializeResponseExt", len)?; + if let Some(v) = self.container.as_ref() { + struct_ser.serialize_field("container", v)?; + } + if let Some(v) = self.flushed.as_ref() { + struct_ser.serialize_field("flushed", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for MaterializeResponseExt { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "container", + "flushed", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Container, + Flushed, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "container" => Ok(GeneratedField::Container), + "flushed" => Ok(GeneratedField::Flushed), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MaterializeResponseExt; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.MaterializeResponseExt") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut container__ = None; + let mut flushed__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Container => { + if container__.is_some() { + return Err(serde::de::Error::duplicate_field("container")); + } + container__ = map_.next_value()?; + } + GeneratedField::Flushed => { + if flushed__.is_some() { + return Err(serde::de::Error::duplicate_field("flushed")); + } + flushed__ = map_.next_value()?; + } + } + } + Ok(MaterializeResponseExt { + container: container__, + flushed: flushed__, + }) + } + } + deserializer.deserialize_struct("runtime.MaterializeResponseExt", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for materialize_response_ext::Flushed { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.stats.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.MaterializeResponseExt.Flushed", len)?; + if let Some(v) = self.stats.as_ref() { + struct_ser.serialize_field("stats", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for materialize_response_ext::Flushed { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "stats", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Stats, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "stats" => Ok(GeneratedField::Stats), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = materialize_response_ext::Flushed; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.MaterializeResponseExt.Flushed") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut stats__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Stats => { + if stats__.is_some() { + return Err(serde::de::Error::duplicate_field("stats")); + } + stats__ = map_.next_value()?; + } + } + } + Ok(materialize_response_ext::Flushed { + stats: stats__, + }) + } + } + deserializer.deserialize_struct("runtime.MaterializeResponseExt.Flushed", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Open { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.spec.is_empty() { + len += 1; + } + if !self.version.is_empty() { + len += 1; + } + if self.range.is_some() { + len += 1; + } + if !self.connector_state_json.is_empty() { + len += 1; + } + if !self.max_keys.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Open", len)?; + if !self.spec.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("spec", pbjson::private::base64::encode(&self.spec).as_str())?; + } + if !self.version.is_empty() { + struct_ser.serialize_field("version", &self.version)?; + } + if let Some(v) = self.range.as_ref() { + struct_ser.serialize_field("range", v)?; + } + if !self.connector_state_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("state", &crate::as_raw_json(&self.connector_state_json)?)?; + } + if !self.max_keys.is_empty() { + let v: std::collections::HashMap<_, _> = self.max_keys.iter() + .map(|(k, v)| (k, pbjson::private::base64::encode(v))).collect(); + struct_ser.serialize_field("maxKeys", &v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Open { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "spec", + "version", + "range", + "connector_state_json", + "state", + "max_keys", + "maxKeys", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Spec, + Version, + Range, + ConnectorStateJson, + MaxKeys, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "spec" => Ok(GeneratedField::Spec), + "version" => Ok(GeneratedField::Version), + "range" => Ok(GeneratedField::Range), + "state" | "connector_state_json" => Ok(GeneratedField::ConnectorStateJson), + "maxKeys" | "max_keys" => Ok(GeneratedField::MaxKeys), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Open; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Open") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut spec__ = None; + let mut version__ = None; + let mut range__ = None; + let mut connector_state_json__ = None; + let mut max_keys__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Spec => { + if spec__.is_some() { + return Err(serde::de::Error::duplicate_field("spec")); + } + spec__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + GeneratedField::Version => { + if version__.is_some() { + return Err(serde::de::Error::duplicate_field("version")); + } + version__ = Some(map_.next_value()?); + } + GeneratedField::Range => { + if range__.is_some() { + return Err(serde::de::Error::duplicate_field("range")); + } + range__ = map_.next_value()?; + } + GeneratedField::ConnectorStateJson => { + if connector_state_json__.is_some() { + return Err(serde::de::Error::duplicate_field("state")); + } + connector_state_json__ = + Some(map_.next_value::()?.0) + ; + } + GeneratedField::MaxKeys => { + if max_keys__.is_some() { + return Err(serde::de::Error::duplicate_field("maxKeys")); + } + max_keys__ = Some( + map_.next_value::, ::pbjson::private::BytesDeserialize<_>>>()? + .into_iter().map(|(k,v)| (k.0, v.0)).collect() + ); + } + } + } + Ok(Open { + spec: spec__.unwrap_or_default(), + version: version__.unwrap_or_default(), + range: range__, + connector_state_json: connector_state_json__.unwrap_or_default(), + max_keys: max_keys__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Open", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Persist { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.nonce != 0 { + len += 1; + } + if self.delete_ack_intents { + len += 1; + } + if !self.ack_intents.is_empty() { + len += 1; + } + if self.committed_close_clock != 0 { + len += 1; + } + if self.committed_frontier.is_some() { + len += 1; + } + if !self.connector_patches_json.is_empty() { + len += 1; + } + if self.hinted_close_clock != 0 { + len += 1; + } + if self.delete_hinted_frontier { + len += 1; + } + if self.hinted_frontier.is_some() { + len += 1; + } + if !self.last_applied.is_empty() { + len += 1; + } + if self.legacy_checkpoint.is_some() { + len += 1; + } + if !self.max_keys.is_empty() { + len += 1; + } + if self.delete_trigger_params { + len += 1; + } + if !self.trigger_params_json.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Persist", len)?; + if self.nonce != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("nonce", ToString::to_string(&self.nonce).as_str())?; + } + if self.delete_ack_intents { + struct_ser.serialize_field("deleteAckIntents", &self.delete_ack_intents)?; + } + if !self.ack_intents.is_empty() { + let v: std::collections::HashMap<_, _> = self.ack_intents.iter() + .map(|(k, v)| (k, pbjson::private::base64::encode(v))).collect(); + struct_ser.serialize_field("ackIntents", &v)?; + } + if self.committed_close_clock != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("committedCloseClock", ToString::to_string(&self.committed_close_clock).as_str())?; + } + if let Some(v) = self.committed_frontier.as_ref() { + struct_ser.serialize_field("committedFrontier", v)?; + } + if !self.connector_patches_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorPatches", &crate::as_raw_json(&self.connector_patches_json)?)?; + } + if self.hinted_close_clock != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("hintedCloseClock", ToString::to_string(&self.hinted_close_clock).as_str())?; + } + if self.delete_hinted_frontier { + struct_ser.serialize_field("deleteHintedFrontier", &self.delete_hinted_frontier)?; + } + if let Some(v) = self.hinted_frontier.as_ref() { + struct_ser.serialize_field("hintedFrontier", v)?; + } + if !self.last_applied.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("lastApplied", pbjson::private::base64::encode(&self.last_applied).as_str())?; + } + if let Some(v) = self.legacy_checkpoint.as_ref() { + struct_ser.serialize_field("legacyCheckpoint", v)?; + } + if !self.max_keys.is_empty() { + let v: std::collections::HashMap<_, _> = self.max_keys.iter() + .map(|(k, v)| (k, pbjson::private::base64::encode(v))).collect(); + struct_ser.serialize_field("maxKeys", &v)?; + } + if self.delete_trigger_params { + struct_ser.serialize_field("deleteTriggerParams", &self.delete_trigger_params)?; + } + if !self.trigger_params_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("triggerParams", &crate::as_raw_json(&self.trigger_params_json)?)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Persist { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "nonce", + "delete_ack_intents", + "deleteAckIntents", + "ack_intents", + "ackIntents", + "committed_close_clock", + "committedCloseClock", + "committed_frontier", + "committedFrontier", + "connector_patches_json", + "connectorPatches", + "hinted_close_clock", + "hintedCloseClock", + "delete_hinted_frontier", + "deleteHintedFrontier", + "hinted_frontier", + "hintedFrontier", + "last_applied", + "lastApplied", + "legacy_checkpoint", + "legacyCheckpoint", + "max_keys", + "maxKeys", + "delete_trigger_params", + "deleteTriggerParams", + "trigger_params_json", + "triggerParams", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Nonce, + DeleteAckIntents, + AckIntents, + CommittedCloseClock, + CommittedFrontier, + ConnectorPatchesJson, + HintedCloseClock, + DeleteHintedFrontier, + HintedFrontier, + LastApplied, + LegacyCheckpoint, + MaxKeys, + DeleteTriggerParams, + TriggerParamsJson, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "nonce" => Ok(GeneratedField::Nonce), + "deleteAckIntents" | "delete_ack_intents" => Ok(GeneratedField::DeleteAckIntents), + "ackIntents" | "ack_intents" => Ok(GeneratedField::AckIntents), + "committedCloseClock" | "committed_close_clock" => Ok(GeneratedField::CommittedCloseClock), + "committedFrontier" | "committed_frontier" => Ok(GeneratedField::CommittedFrontier), + "connectorPatches" | "connector_patches_json" => Ok(GeneratedField::ConnectorPatchesJson), + "hintedCloseClock" | "hinted_close_clock" => Ok(GeneratedField::HintedCloseClock), + "deleteHintedFrontier" | "delete_hinted_frontier" => Ok(GeneratedField::DeleteHintedFrontier), + "hintedFrontier" | "hinted_frontier" => Ok(GeneratedField::HintedFrontier), + "lastApplied" | "last_applied" => Ok(GeneratedField::LastApplied), + "legacyCheckpoint" | "legacy_checkpoint" => Ok(GeneratedField::LegacyCheckpoint), + "maxKeys" | "max_keys" => Ok(GeneratedField::MaxKeys), + "deleteTriggerParams" | "delete_trigger_params" => Ok(GeneratedField::DeleteTriggerParams), + "triggerParams" | "trigger_params_json" => Ok(GeneratedField::TriggerParamsJson), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Persist; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Persist") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut nonce__ = None; + let mut delete_ack_intents__ = None; + let mut ack_intents__ = None; + let mut committed_close_clock__ = None; + let mut committed_frontier__ = None; + let mut connector_patches_json__ = None; + let mut hinted_close_clock__ = None; + let mut delete_hinted_frontier__ = None; + let mut hinted_frontier__ = None; + let mut last_applied__ = None; + let mut legacy_checkpoint__ = None; + let mut max_keys__ = None; + let mut delete_trigger_params__ = None; + let mut trigger_params_json__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Nonce => { + if nonce__.is_some() { + return Err(serde::de::Error::duplicate_field("nonce")); + } + nonce__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::DeleteAckIntents => { + if delete_ack_intents__.is_some() { + return Err(serde::de::Error::duplicate_field("deleteAckIntents")); + } + delete_ack_intents__ = Some(map_.next_value()?); + } + GeneratedField::AckIntents => { + if ack_intents__.is_some() { + return Err(serde::de::Error::duplicate_field("ackIntents")); + } + ack_intents__ = Some( + map_.next_value::>>()? + .into_iter().map(|(k,v)| (k, v.0)).collect() + ); + } + GeneratedField::CommittedCloseClock => { + if committed_close_clock__.is_some() { + return Err(serde::de::Error::duplicate_field("committedCloseClock")); + } + committed_close_clock__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::CommittedFrontier => { + if committed_frontier__.is_some() { + return Err(serde::de::Error::duplicate_field("committedFrontier")); + } + committed_frontier__ = map_.next_value()?; + } + GeneratedField::ConnectorPatchesJson => { + if connector_patches_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorPatches")); + } + connector_patches_json__ = + Some(map_.next_value::()?.0) + ; + } + GeneratedField::HintedCloseClock => { + if hinted_close_clock__.is_some() { + return Err(serde::de::Error::duplicate_field("hintedCloseClock")); + } + hinted_close_clock__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::DeleteHintedFrontier => { + if delete_hinted_frontier__.is_some() { + return Err(serde::de::Error::duplicate_field("deleteHintedFrontier")); + } + delete_hinted_frontier__ = Some(map_.next_value()?); + } + GeneratedField::HintedFrontier => { + if hinted_frontier__.is_some() { + return Err(serde::de::Error::duplicate_field("hintedFrontier")); + } + hinted_frontier__ = map_.next_value()?; + } + GeneratedField::LastApplied => { + if last_applied__.is_some() { + return Err(serde::de::Error::duplicate_field("lastApplied")); + } + last_applied__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + GeneratedField::LegacyCheckpoint => { + if legacy_checkpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("legacyCheckpoint")); + } + legacy_checkpoint__ = map_.next_value()?; + } + GeneratedField::MaxKeys => { + if max_keys__.is_some() { + return Err(serde::de::Error::duplicate_field("maxKeys")); + } + max_keys__ = Some( + map_.next_value::, ::pbjson::private::BytesDeserialize<_>>>()? + .into_iter().map(|(k,v)| (k.0, v.0)).collect() + ); + } + GeneratedField::DeleteTriggerParams => { + if delete_trigger_params__.is_some() { + return Err(serde::de::Error::duplicate_field("deleteTriggerParams")); + } + delete_trigger_params__ = Some(map_.next_value()?); + } + GeneratedField::TriggerParamsJson => { + if trigger_params_json__.is_some() { + return Err(serde::de::Error::duplicate_field("triggerParams")); + } + trigger_params_json__ = + Some(map_.next_value::()?.0) + ; + } + } + } + Ok(Persist { + nonce: nonce__.unwrap_or_default(), + delete_ack_intents: delete_ack_intents__.unwrap_or_default(), + ack_intents: ack_intents__.unwrap_or_default(), + committed_close_clock: committed_close_clock__.unwrap_or_default(), + committed_frontier: committed_frontier__, + connector_patches_json: connector_patches_json__.unwrap_or_default(), + hinted_close_clock: hinted_close_clock__.unwrap_or_default(), + delete_hinted_frontier: delete_hinted_frontier__.unwrap_or_default(), + hinted_frontier: hinted_frontier__, + last_applied: last_applied__.unwrap_or_default(), + legacy_checkpoint: legacy_checkpoint__, + max_keys: max_keys__.unwrap_or_default(), + delete_trigger_params: delete_trigger_params__.unwrap_or_default(), + trigger_params_json: trigger_params_json__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Persist", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Persisted { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.nonce != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Persisted", len)?; + if self.nonce != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("nonce", ToString::to_string(&self.nonce).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Persisted { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "nonce", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Nonce, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "nonce" => Ok(GeneratedField::Nonce), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Persisted; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Persisted") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut nonce__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Nonce => { + if nonce__.is_some() { + return Err(serde::de::Error::duplicate_field("nonce")); + } + nonce__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(Persisted { + nonce: nonce__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Persisted", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Plane { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + let variant = match self { + Self::Public => "PUBLIC", + Self::Private => "PRIVATE", + Self::Local => "LOCAL", + }; + serializer.serialize_str(variant) + } +} +impl<'de> serde::Deserialize<'de> for Plane { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "PUBLIC", + "PRIVATE", + "LOCAL", + ]; + + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Plane; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + fn visit_i64(self, v: i64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) + }) + } + + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "PUBLIC" => Ok(Plane::Public), + "PRIVATE" => Ok(Plane::Private), + "LOCAL" => Ok(Plane::Local), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} +impl serde::Serialize for Recover { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.ack_intents.is_empty() { + len += 1; + } + if self.committed_close_clock != 0 { + len += 1; + } + if self.committed_frontier.is_some() { + len += 1; + } + if !self.connector_state_json.is_empty() { + len += 1; + } + if self.hinted_close_clock != 0 { + len += 1; + } + if self.hinted_frontier.is_some() { + len += 1; + } + if !self.last_applied.is_empty() { + len += 1; + } + if self.legacy_checkpoint.is_some() { + len += 1; + } + if !self.max_keys.is_empty() { + len += 1; + } + if !self.trigger_params_json.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Recover", len)?; + if !self.ack_intents.is_empty() { + let v: std::collections::HashMap<_, _> = self.ack_intents.iter() + .map(|(k, v)| (k, pbjson::private::base64::encode(v))).collect(); + struct_ser.serialize_field("ackIntents", &v)?; + } + if self.committed_close_clock != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("committedCloseClock", ToString::to_string(&self.committed_close_clock).as_str())?; + } + if let Some(v) = self.committed_frontier.as_ref() { + struct_ser.serialize_field("committedFrontier", v)?; + } + if !self.connector_state_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("connectorState", &crate::as_raw_json(&self.connector_state_json)?)?; + } + if self.hinted_close_clock != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("hintedCloseClock", ToString::to_string(&self.hinted_close_clock).as_str())?; + } + if let Some(v) = self.hinted_frontier.as_ref() { + struct_ser.serialize_field("hintedFrontier", v)?; + } + if !self.last_applied.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("lastApplied", pbjson::private::base64::encode(&self.last_applied).as_str())?; + } + if let Some(v) = self.legacy_checkpoint.as_ref() { + struct_ser.serialize_field("legacyCheckpoint", v)?; + } + if !self.max_keys.is_empty() { + let v: std::collections::HashMap<_, _> = self.max_keys.iter() + .map(|(k, v)| (k, pbjson::private::base64::encode(v))).collect(); + struct_ser.serialize_field("maxKeys", &v)?; + } + if !self.trigger_params_json.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("triggerParams", &crate::as_raw_json(&self.trigger_params_json)?)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Recover { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "ack_intents", + "ackIntents", + "committed_close_clock", + "committedCloseClock", + "committed_frontier", + "committedFrontier", + "connector_state_json", + "connectorState", + "hinted_close_clock", + "hintedCloseClock", + "hinted_frontier", + "hintedFrontier", + "last_applied", + "lastApplied", + "legacy_checkpoint", + "legacyCheckpoint", + "max_keys", + "maxKeys", + "trigger_params_json", + "triggerParams", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + AckIntents, + CommittedCloseClock, + CommittedFrontier, + ConnectorStateJson, + HintedCloseClock, + HintedFrontier, + LastApplied, + LegacyCheckpoint, + MaxKeys, + TriggerParamsJson, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "ackIntents" | "ack_intents" => Ok(GeneratedField::AckIntents), + "committedCloseClock" | "committed_close_clock" => Ok(GeneratedField::CommittedCloseClock), + "committedFrontier" | "committed_frontier" => Ok(GeneratedField::CommittedFrontier), + "connectorState" | "connector_state_json" => Ok(GeneratedField::ConnectorStateJson), + "hintedCloseClock" | "hinted_close_clock" => Ok(GeneratedField::HintedCloseClock), + "hintedFrontier" | "hinted_frontier" => Ok(GeneratedField::HintedFrontier), + "lastApplied" | "last_applied" => Ok(GeneratedField::LastApplied), + "legacyCheckpoint" | "legacy_checkpoint" => Ok(GeneratedField::LegacyCheckpoint), + "maxKeys" | "max_keys" => Ok(GeneratedField::MaxKeys), + "triggerParams" | "trigger_params_json" => Ok(GeneratedField::TriggerParamsJson), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Recover; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Recover") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut ack_intents__ = None; + let mut committed_close_clock__ = None; + let mut committed_frontier__ = None; + let mut connector_state_json__ = None; + let mut hinted_close_clock__ = None; + let mut hinted_frontier__ = None; + let mut last_applied__ = None; + let mut legacy_checkpoint__ = None; + let mut max_keys__ = None; + let mut trigger_params_json__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::AckIntents => { + if ack_intents__.is_some() { + return Err(serde::de::Error::duplicate_field("ackIntents")); + } + ack_intents__ = Some( + map_.next_value::>>()? + .into_iter().map(|(k,v)| (k, v.0)).collect() + ); + } + GeneratedField::CommittedCloseClock => { + if committed_close_clock__.is_some() { + return Err(serde::de::Error::duplicate_field("committedCloseClock")); + } + committed_close_clock__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::CommittedFrontier => { + if committed_frontier__.is_some() { + return Err(serde::de::Error::duplicate_field("committedFrontier")); + } + committed_frontier__ = map_.next_value()?; + } + GeneratedField::ConnectorStateJson => { + if connector_state_json__.is_some() { + return Err(serde::de::Error::duplicate_field("connectorState")); + } + connector_state_json__ = + Some(map_.next_value::()?.0) + ; + } + GeneratedField::HintedCloseClock => { + if hinted_close_clock__.is_some() { + return Err(serde::de::Error::duplicate_field("hintedCloseClock")); + } + hinted_close_clock__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::HintedFrontier => { + if hinted_frontier__.is_some() { + return Err(serde::de::Error::duplicate_field("hintedFrontier")); + } + hinted_frontier__ = map_.next_value()?; + } + GeneratedField::LastApplied => { + if last_applied__.is_some() { + return Err(serde::de::Error::duplicate_field("lastApplied")); + } + last_applied__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + GeneratedField::LegacyCheckpoint => { + if legacy_checkpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("legacyCheckpoint")); + } + legacy_checkpoint__ = map_.next_value()?; + } + GeneratedField::MaxKeys => { + if max_keys__.is_some() { + return Err(serde::de::Error::duplicate_field("maxKeys")); + } + max_keys__ = Some( + map_.next_value::, ::pbjson::private::BytesDeserialize<_>>>()? + .into_iter().map(|(k,v)| (k.0, v.0)).collect() + ); + } + GeneratedField::TriggerParamsJson => { + if trigger_params_json__.is_some() { + return Err(serde::de::Error::duplicate_field("triggerParams")); + } + trigger_params_json__ = + Some(map_.next_value::()?.0) + ; + } + } + } + Ok(Recover { + ack_intents: ack_intents__.unwrap_or_default(), + committed_close_clock: committed_close_clock__.unwrap_or_default(), + committed_frontier: committed_frontier__, + connector_state_json: connector_state_json__.unwrap_or_default(), + hinted_close_clock: hinted_close_clock__.unwrap_or_default(), + hinted_frontier: hinted_frontier__, + last_applied: last_applied__.unwrap_or_default(), + legacy_checkpoint: legacy_checkpoint__, + max_keys: max_keys__.unwrap_or_default(), + trigger_params_json: trigger_params_json__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Recover", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for RocksDbDescriptor { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.rocksdb_env_memptr != 0 { + len += 1; + } + if !self.rocksdb_path.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.RocksDBDescriptor", len)?; + if self.rocksdb_env_memptr != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("rocksdbEnvMemptr", ToString::to_string(&self.rocksdb_env_memptr).as_str())?; + } + if !self.rocksdb_path.is_empty() { + struct_ser.serialize_field("rocksdbPath", &self.rocksdb_path)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for RocksDbDescriptor { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "rocksdb_env_memptr", + "rocksdbEnvMemptr", + "rocksdb_path", + "rocksdbPath", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + RocksdbEnvMemptr, + RocksdbPath, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "rocksdbEnvMemptr" | "rocksdb_env_memptr" => Ok(GeneratedField::RocksdbEnvMemptr), + "rocksdbPath" | "rocksdb_path" => Ok(GeneratedField::RocksdbPath), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = RocksDbDescriptor; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.RocksDBDescriptor") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut rocksdb_env_memptr__ = None; + let mut rocksdb_path__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::RocksdbEnvMemptr => { + if rocksdb_env_memptr__.is_some() { + return Err(serde::de::Error::duplicate_field("rocksdbEnvMemptr")); + } + rocksdb_env_memptr__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::RocksdbPath => { + if rocksdb_path__.is_some() { + return Err(serde::de::Error::duplicate_field("rocksdbPath")); + } + rocksdb_path__ = Some(map_.next_value()?); + } + } + } + Ok(RocksDbDescriptor { + rocksdb_env_memptr: rocksdb_env_memptr__.unwrap_or_default(), + rocksdb_path: rocksdb_path__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.RocksDBDescriptor", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for SessionLoop { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.rocksdb_descriptor.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.SessionLoop", len)?; + if let Some(v) = self.rocksdb_descriptor.as_ref() { + struct_ser.serialize_field("rocksdbDescriptor", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for SessionLoop { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "rocksdb_descriptor", + "rocksdbDescriptor", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + RocksdbDescriptor, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "rocksdbDescriptor" | "rocksdb_descriptor" => Ok(GeneratedField::RocksdbDescriptor), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = SessionLoop; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.SessionLoop") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut rocksdb_descriptor__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::RocksdbDescriptor => { + if rocksdb_descriptor__.is_some() { + return Err(serde::de::Error::duplicate_field("rocksdbDescriptor")); + } + rocksdb_descriptor__ = map_.next_value()?; + } + } + } + Ok(SessionLoop { + rocksdb_descriptor: rocksdb_descriptor__, + }) + } + } + deserializer.deserialize_struct("runtime.SessionLoop", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ShuffleRequest { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.journal.is_empty() { + len += 1; + } + if self.replay { + len += 1; + } + if !self.build_id.is_empty() { + len += 1; + } + if self.offset != 0 { + len += 1; + } + if self.end_offset != 0 { + len += 1; + } + if self.range.is_some() { + len += 1; + } + if !self.coordinator.is_empty() { + len += 1; + } + if self.resolution.is_some() { + len += 1; + } + if self.shuffle_index != 0 { + len += 1; + } + if self.derivation.is_some() { + len += 1; + } + if self.materialization.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.ShuffleRequest", len)?; + if !self.journal.is_empty() { + struct_ser.serialize_field("journal", &self.journal)?; + } + if self.replay { + struct_ser.serialize_field("replay", &self.replay)?; + } + if !self.build_id.is_empty() { + struct_ser.serialize_field("buildId", &self.build_id)?; + } + if self.offset != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("offset", ToString::to_string(&self.offset).as_str())?; + } + if self.end_offset != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("endOffset", ToString::to_string(&self.end_offset).as_str())?; + } + if let Some(v) = self.range.as_ref() { + struct_ser.serialize_field("range", v)?; + } + if !self.coordinator.is_empty() { + struct_ser.serialize_field("coordinator", &self.coordinator)?; + } + if let Some(v) = self.resolution.as_ref() { + struct_ser.serialize_field("resolution", v)?; + } + if self.shuffle_index != 0 { + struct_ser.serialize_field("shuffleIndex", &self.shuffle_index)?; + } + if let Some(v) = self.derivation.as_ref() { + struct_ser.serialize_field("derivation", v)?; + } + if let Some(v) = self.materialization.as_ref() { + struct_ser.serialize_field("materialization", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ShuffleRequest { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "journal", + "replay", + "build_id", + "buildId", + "offset", + "end_offset", + "endOffset", + "range", + "coordinator", + "resolution", + "shuffle_index", + "shuffleIndex", + "derivation", + "materialization", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Journal, + Replay, + BuildId, + Offset, + EndOffset, + Range, + Coordinator, + Resolution, + ShuffleIndex, + Derivation, + Materialization, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "journal" => Ok(GeneratedField::Journal), + "replay" => Ok(GeneratedField::Replay), + "buildId" | "build_id" => Ok(GeneratedField::BuildId), + "offset" => Ok(GeneratedField::Offset), + "endOffset" | "end_offset" => Ok(GeneratedField::EndOffset), + "range" => Ok(GeneratedField::Range), + "coordinator" => Ok(GeneratedField::Coordinator), + "resolution" => Ok(GeneratedField::Resolution), + "shuffleIndex" | "shuffle_index" => Ok(GeneratedField::ShuffleIndex), + "derivation" => Ok(GeneratedField::Derivation), + "materialization" => Ok(GeneratedField::Materialization), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ShuffleRequest; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.ShuffleRequest") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut journal__ = None; + let mut replay__ = None; + let mut build_id__ = None; + let mut offset__ = None; + let mut end_offset__ = None; + let mut range__ = None; + let mut coordinator__ = None; + let mut resolution__ = None; + let mut shuffle_index__ = None; + let mut derivation__ = None; + let mut materialization__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Journal => { + if journal__.is_some() { + return Err(serde::de::Error::duplicate_field("journal")); + } + journal__ = Some(map_.next_value()?); + } + GeneratedField::Replay => { + if replay__.is_some() { + return Err(serde::de::Error::duplicate_field("replay")); + } + replay__ = Some(map_.next_value()?); + } + GeneratedField::BuildId => { + if build_id__.is_some() { + return Err(serde::de::Error::duplicate_field("buildId")); + } + build_id__ = Some(map_.next_value()?); + } + GeneratedField::Offset => { + if offset__.is_some() { + return Err(serde::de::Error::duplicate_field("offset")); + } + offset__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::EndOffset => { + if end_offset__.is_some() { + return Err(serde::de::Error::duplicate_field("endOffset")); + } + end_offset__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::Range => { + if range__.is_some() { + return Err(serde::de::Error::duplicate_field("range")); + } + range__ = map_.next_value()?; + } + GeneratedField::Coordinator => { + if coordinator__.is_some() { + return Err(serde::de::Error::duplicate_field("coordinator")); + } + coordinator__ = Some(map_.next_value()?); + } + GeneratedField::Resolution => { + if resolution__.is_some() { + return Err(serde::de::Error::duplicate_field("resolution")); + } + resolution__ = map_.next_value()?; + } + GeneratedField::ShuffleIndex => { + if shuffle_index__.is_some() { + return Err(serde::de::Error::duplicate_field("shuffleIndex")); + } + shuffle_index__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::Derivation => { + if derivation__.is_some() { + return Err(serde::de::Error::duplicate_field("derivation")); + } + derivation__ = map_.next_value()?; + } + GeneratedField::Materialization => { + if materialization__.is_some() { + return Err(serde::de::Error::duplicate_field("materialization")); + } + materialization__ = map_.next_value()?; + } + } + } + Ok(ShuffleRequest { + journal: journal__.unwrap_or_default(), + replay: replay__.unwrap_or_default(), + build_id: build_id__.unwrap_or_default(), + offset: offset__.unwrap_or_default(), + end_offset: end_offset__.unwrap_or_default(), + range: range__, + coordinator: coordinator__.unwrap_or_default(), + resolution: resolution__, + shuffle_index: shuffle_index__.unwrap_or_default(), + derivation: derivation__, + materialization: materialization__, + }) + } + } + deserializer.deserialize_struct("runtime.ShuffleRequest", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ShuffleResponse { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.status != 0 { + len += 1; + } + if self.header.is_some() { + len += 1; + } + if !self.terminal_error.is_empty() { + len += 1; + } + if self.read_through != 0 { + len += 1; + } + if self.write_head != 0 { + len += 1; + } + if !self.arena.is_empty() { + len += 1; + } + if !self.docs.is_empty() { + len += 1; + } + if !self.offsets.is_empty() { + len += 1; + } + if !self.uuid_parts.is_empty() { + len += 1; + } + if !self.packed_key.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.ShuffleResponse", len)?; + if self.status != 0 { + let v = super::consumer::Status::try_from(self.status) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.status)))?; + struct_ser.serialize_field("status", &v)?; + } + if let Some(v) = self.header.as_ref() { + struct_ser.serialize_field("header", v)?; + } + if !self.terminal_error.is_empty() { + struct_ser.serialize_field("terminalError", &self.terminal_error)?; + } + if self.read_through != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("readThrough", ToString::to_string(&self.read_through).as_str())?; + } + if self.write_head != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("writeHead", ToString::to_string(&self.write_head).as_str())?; + } + if !self.arena.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("arena", pbjson::private::base64::encode(&self.arena).as_str())?; + } + if !self.docs.is_empty() { + struct_ser.serialize_field("docs", &self.docs)?; + } + if !self.offsets.is_empty() { + struct_ser.serialize_field("offsets", &self.offsets.iter().map(ToString::to_string).collect::>())?; + } + if !self.uuid_parts.is_empty() { + struct_ser.serialize_field("uuidParts", &self.uuid_parts)?; + } + if !self.packed_key.is_empty() { + struct_ser.serialize_field("packedKey", &self.packed_key)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ShuffleResponse { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "status", + "header", + "terminal_error", + "terminalError", + "read_through", + "readThrough", + "write_head", + "writeHead", + "arena", + "docs", + "offsets", + "uuid_parts", + "uuidParts", + "packed_key", + "packedKey", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Status, + Header, + TerminalError, + ReadThrough, + WriteHead, + Arena, + Docs, + Offsets, + UuidParts, + PackedKey, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "status" => Ok(GeneratedField::Status), + "header" => Ok(GeneratedField::Header), + "terminalError" | "terminal_error" => Ok(GeneratedField::TerminalError), + "readThrough" | "read_through" => Ok(GeneratedField::ReadThrough), + "writeHead" | "write_head" => Ok(GeneratedField::WriteHead), + "arena" => Ok(GeneratedField::Arena), + "docs" => Ok(GeneratedField::Docs), + "offsets" => Ok(GeneratedField::Offsets), + "uuidParts" | "uuid_parts" => Ok(GeneratedField::UuidParts), + "packedKey" | "packed_key" => Ok(GeneratedField::PackedKey), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ShuffleResponse; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.ShuffleResponse") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut status__ = None; + let mut header__ = None; + let mut terminal_error__ = None; + let mut read_through__ = None; + let mut write_head__ = None; + let mut arena__ = None; + let mut docs__ = None; + let mut offsets__ = None; + let mut uuid_parts__ = None; + let mut packed_key__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Status => { + if status__.is_some() { + return Err(serde::de::Error::duplicate_field("status")); + } + status__ = Some(map_.next_value::()? as i32); + } + GeneratedField::Header => { + if header__.is_some() { + return Err(serde::de::Error::duplicate_field("header")); + } + header__ = map_.next_value()?; + } + GeneratedField::TerminalError => { + if terminal_error__.is_some() { + return Err(serde::de::Error::duplicate_field("terminalError")); + } + terminal_error__ = Some(map_.next_value()?); + } + GeneratedField::ReadThrough => { + if read_through__.is_some() { + return Err(serde::de::Error::duplicate_field("readThrough")); + } + read_through__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::WriteHead => { + if write_head__.is_some() { + return Err(serde::de::Error::duplicate_field("writeHead")); + } + write_head__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::Arena => { + if arena__.is_some() { + return Err(serde::de::Error::duplicate_field("arena")); + } + arena__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + GeneratedField::Docs => { + if docs__.is_some() { + return Err(serde::de::Error::duplicate_field("docs")); + } + docs__ = Some(map_.next_value()?); + } + GeneratedField::Offsets => { + if offsets__.is_some() { + return Err(serde::de::Error::duplicate_field("offsets")); + } + offsets__ = + Some(map_.next_value::>>()? + .into_iter().map(|x| x.0).collect()) + ; + } + GeneratedField::UuidParts => { + if uuid_parts__.is_some() { + return Err(serde::de::Error::duplicate_field("uuidParts")); + } + uuid_parts__ = Some(map_.next_value()?); + } + GeneratedField::PackedKey => { + if packed_key__.is_some() { + return Err(serde::de::Error::duplicate_field("packedKey")); + } + packed_key__ = Some(map_.next_value()?); + } + } + } + Ok(ShuffleResponse { + status: status__.unwrap_or_default(), + header: header__, + terminal_error: terminal_error__.unwrap_or_default(), + read_through: read_through__.unwrap_or_default(), + write_head: write_head__.unwrap_or_default(), + arena: arena__.unwrap_or_default(), + docs: docs__.unwrap_or_default(), + offsets: offsets__.unwrap_or_default(), + uuid_parts: uuid_parts__.unwrap_or_default(), + packed_key: packed_key__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.ShuffleResponse", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Stop { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let len = 0; + let struct_ser = serializer.serialize_struct("runtime.Stop", len)?; + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Stop { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + Err(serde::de::Error::unknown_field(value, FIELDS)) + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Stop; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Stop") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + while map_.next_key::()?.is_some() { + let _ = map_.next_value::()?; + } + Ok(Stop { + }) + } + } + deserializer.deserialize_struct("runtime.Stop", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Stopped { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let len = 0; + let struct_ser = serializer.serialize_struct("runtime.Stopped", len)?; + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Stopped { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + Err(serde::de::Error::unknown_field(value, FIELDS)) + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Stopped; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Stopped") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + while map_.next_key::()?.is_some() { + let _ = map_.next_value::()?; + } + Ok(Stopped { + }) + } + } + deserializer.deserialize_struct("runtime.Stopped", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Task { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.spec.is_empty() { + len += 1; + } + if !self.ops_stats_journal.is_empty() { + len += 1; + } + if self.ops_stats_spec.is_some() { + len += 1; + } + if self.preview { + len += 1; + } + if self.max_transactions != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.Task", len)?; + if !self.spec.is_empty() { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("spec", pbjson::private::base64::encode(&self.spec).as_str())?; + } + if !self.ops_stats_journal.is_empty() { + struct_ser.serialize_field("opsStatsJournal", &self.ops_stats_journal)?; + } + if let Some(v) = self.ops_stats_spec.as_ref() { + struct_ser.serialize_field("opsStatsSpec", v)?; + } + if self.preview { + struct_ser.serialize_field("preview", &self.preview)?; + } + if self.max_transactions != 0 { + struct_ser.serialize_field("maxTransactions", &self.max_transactions)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Task { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "spec", + "ops_stats_journal", + "opsStatsJournal", + "ops_stats_spec", + "opsStatsSpec", + "preview", + "max_transactions", + "maxTransactions", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Spec, + OpsStatsJournal, + OpsStatsSpec, + Preview, + MaxTransactions, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "spec" => Ok(GeneratedField::Spec), + "opsStatsJournal" | "ops_stats_journal" => Ok(GeneratedField::OpsStatsJournal), + "opsStatsSpec" | "ops_stats_spec" => Ok(GeneratedField::OpsStatsSpec), + "preview" => Ok(GeneratedField::Preview), + "maxTransactions" | "max_transactions" => Ok(GeneratedField::MaxTransactions), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Task; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.Task") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut spec__ = None; + let mut ops_stats_journal__ = None; + let mut ops_stats_spec__ = None; + let mut preview__ = None; + let mut max_transactions__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Spec => { + if spec__.is_some() { + return Err(serde::de::Error::duplicate_field("spec")); + } + spec__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } + GeneratedField::OpsStatsJournal => { + if ops_stats_journal__.is_some() { + return Err(serde::de::Error::duplicate_field("opsStatsJournal")); + } + ops_stats_journal__ = Some(map_.next_value()?); + } + GeneratedField::OpsStatsSpec => { + if ops_stats_spec__.is_some() { + return Err(serde::de::Error::duplicate_field("opsStatsSpec")); + } + ops_stats_spec__ = map_.next_value()?; + } + GeneratedField::Preview => { + if preview__.is_some() { + return Err(serde::de::Error::duplicate_field("preview")); + } + preview__ = Some(map_.next_value()?); + } + GeneratedField::MaxTransactions => { + if max_transactions__.is_some() { + return Err(serde::de::Error::duplicate_field("maxTransactions")); + } + max_transactions__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(Task { + spec: spec__.unwrap_or_default(), + ops_stats_journal: ops_stats_journal__.unwrap_or_default(), + ops_stats_spec: ops_stats_spec__, + preview: preview__.unwrap_or_default(), + max_transactions: max_transactions__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.Task", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for TaskServiceConfig { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.log_file_fd != 0 { + len += 1; + } + if !self.task_name.is_empty() { + len += 1; + } + if !self.uds_path.is_empty() { + len += 1; + } + if !self.container_network.is_empty() { + len += 1; + } + if self.plane != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("runtime.TaskServiceConfig", len)?; + if self.log_file_fd != 0 { + struct_ser.serialize_field("logFileFd", &self.log_file_fd)?; + } + if !self.task_name.is_empty() { + struct_ser.serialize_field("taskName", &self.task_name)?; + } + if !self.uds_path.is_empty() { + struct_ser.serialize_field("udsPath", &self.uds_path)?; + } + if !self.container_network.is_empty() { + struct_ser.serialize_field("containerNetwork", &self.container_network)?; + } + if self.plane != 0 { + let v = Plane::try_from(self.plane) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.plane)))?; + struct_ser.serialize_field("plane", &v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for TaskServiceConfig { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "log_file_fd", + "logFileFd", + "task_name", + "taskName", + "uds_path", + "udsPath", + "container_network", + "containerNetwork", + "plane", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + LogFileFd, + TaskName, + UdsPath, + ContainerNetwork, + Plane, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "logFileFd" | "log_file_fd" => Ok(GeneratedField::LogFileFd), + "taskName" | "task_name" => Ok(GeneratedField::TaskName), + "udsPath" | "uds_path" => Ok(GeneratedField::UdsPath), + "containerNetwork" | "container_network" => Ok(GeneratedField::ContainerNetwork), + "plane" => Ok(GeneratedField::Plane), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = TaskServiceConfig; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct runtime.TaskServiceConfig") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut log_file_fd__ = None; + let mut task_name__ = None; + let mut uds_path__ = None; + let mut container_network__ = None; + let mut plane__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::LogFileFd => { + if log_file_fd__.is_some() { + return Err(serde::de::Error::duplicate_field("logFileFd")); + } + log_file_fd__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::TaskName => { + if task_name__.is_some() { + return Err(serde::de::Error::duplicate_field("taskName")); + } + task_name__ = Some(map_.next_value()?); + } + GeneratedField::UdsPath => { + if uds_path__.is_some() { + return Err(serde::de::Error::duplicate_field("udsPath")); + } + uds_path__ = Some(map_.next_value()?); + } + GeneratedField::ContainerNetwork => { + if container_network__.is_some() { + return Err(serde::de::Error::duplicate_field("containerNetwork")); + } + container_network__ = Some(map_.next_value()?); + } + GeneratedField::Plane => { + if plane__.is_some() { + return Err(serde::de::Error::duplicate_field("plane")); + } + plane__ = Some(map_.next_value::()? as i32); + } + } + } + Ok(TaskServiceConfig { + log_file_fd: log_file_fd__.unwrap_or_default(), + task_name: task_name__.unwrap_or_default(), + uds_path: uds_path__.unwrap_or_default(), + container_network: container_network__.unwrap_or_default(), + plane: plane__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("runtime.TaskServiceConfig", FIELDS, GeneratedVisitor) + } +} diff --git a/crates/proto-flow/src/shuffle.rs b/crates/proto-flow/src/shuffle.rs index 4a8cb157f8a..8ff3b4c7574 100644 --- a/crates/proto-flow/src/shuffle.rs +++ b/crates/proto-flow/src/shuffle.rs @@ -22,10 +22,6 @@ pub struct CollectionPartitions { /// Partition selector for filtering source collection journals. #[prost(message, optional, tag = "2")] pub partition_selector: ::core::option::Option<::proto_gazette::broker::LabelSelector>, - /// Disk backlog threshold in bytes before engaging back-pressure. - /// Used for ad-hoc reads where the caller controls the threshold. - #[prost(uint64, tag = "3")] - pub disk_backlog_threshold: u64, } /// Task which we're performing shuffles for. #[derive(Clone, PartialEq, ::prost::Message)] @@ -106,16 +102,13 @@ pub struct JournalFrontier { #[prost(message, repeated, tag = "6")] pub producers: ::prost::alloc::vec::Vec, } -/// FrontierChunk is a portion of a frontier sequence for streaming. +/// Frontier is a complete frontier (or frontier delta) of journal progress. /// Entries are sorted and unique on (journal name, binding). -/// A final empty chunk (empty `journals`) represents end-of-sequence. #[derive(Clone, PartialEq, ::prost::Message)] -pub struct FrontierChunk { +pub struct Frontier { #[prost(message, repeated, tag = "1")] pub journals: ::prost::alloc::vec::Vec, /// Per-shard flushed LSN, indexed by shard_index. - /// Populated only on the terminal (empty-journals) chunk of a - /// Progressed or NextCheckpoint sequence. Empty otherwise. #[prost(uint64, repeated, tag = "2")] pub flushed_lsn: ::prost::alloc::vec::Vec, } @@ -125,7 +118,7 @@ pub struct SessionRequest { #[prost(message, optional, tag = "1")] pub open: ::core::option::Option, /// The resume checkpoint: the non-delta frontier from which the - /// session is to resume. It's streamed by the Coordinator client after reading + /// session is to resume. It's sent by the Coordinator client after reading /// SessionResponse.Opened. /// /// This is a comprehensive checkpoint, reflecting all journals and producers @@ -147,7 +140,7 @@ pub struct SessionRequest { /// unfinished NextCheckpoint of its prior session, which enables the transaction /// to be idempotent. #[prost(message, optional, tag = "2")] - pub resume_checkpoint_chunk: ::core::option::Option, + pub resume_checkpoint: ::core::option::Option, #[prost(message, optional, tag = "3")] pub next_checkpoint: ::core::option::Option, } @@ -171,7 +164,7 @@ pub mod session_request { /// /// A NextCheckpoint response may be a fully-resolved checkpoint OR a "peek" /// of an in-flight frontier whose causal hints have not yet fully resolved. - /// See SessionResponse.next_checkpoint_chunk for the contract. + /// See SessionResponse.next_checkpoint for the contract. #[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] pub struct NextCheckpoint {} } @@ -204,7 +197,7 @@ pub struct SessionResponse { /// upon its durable completion of all downstream processing related to a /// fully-resolved NextCheckpoint, it should merge it into its base checkpoint. #[prost(message, optional, tag = "2")] - pub next_checkpoint_chunk: ::core::option::Option, + pub next_checkpoint: ::core::option::Option, } /// Nested message and enum types in `SessionResponse`. pub mod session_response { @@ -292,7 +285,7 @@ pub struct SliceResponse { /// It's sent in response to a Request.Progress, and only once at least one /// flush cycle has completed since the prior Progressed. #[prost(message, optional, tag = "3")] - pub progressed: ::core::option::Option, + pub progressed: ::core::option::Option, } /// Nested message and enum types in `SliceResponse`. pub mod slice_response { @@ -349,9 +342,6 @@ pub mod log_request { /// Index of the target Log shard within the session's shard list. #[prost(uint32, tag = "4")] pub log_shard_index: u32, - /// Disk backlog threshold in bytes before engaging back-pressure. - #[prost(uint64, tag = "5")] - pub disk_backlog_threshold: u64, } /// Append sends a document to be written to the log. /// The Log actor merges across Slice streams, ordering by (priority, clock). diff --git a/crates/proto-flow/src/shuffle.serde.rs b/crates/proto-flow/src/shuffle.serde.rs index fc2124cc174..900287889be 100644 --- a/crates/proto-flow/src/shuffle.serde.rs +++ b/crates/proto-flow/src/shuffle.serde.rs @@ -12,9 +12,6 @@ impl serde::Serialize for CollectionPartitions { if self.partition_selector.is_some() { len += 1; } - if self.disk_backlog_threshold != 0 { - len += 1; - } let mut struct_ser = serializer.serialize_struct("shuffle.CollectionPartitions", len)?; if let Some(v) = self.collection.as_ref() { struct_ser.serialize_field("collection", v)?; @@ -22,11 +19,6 @@ impl serde::Serialize for CollectionPartitions { if let Some(v) = self.partition_selector.as_ref() { struct_ser.serialize_field("partitionSelector", v)?; } - if self.disk_backlog_threshold != 0 { - #[allow(clippy::needless_borrow)] - #[allow(clippy::needless_borrows_for_generic_args)] - struct_ser.serialize_field("diskBacklogThreshold", ToString::to_string(&self.disk_backlog_threshold).as_str())?; - } struct_ser.end() } } @@ -40,15 +32,12 @@ impl<'de> serde::Deserialize<'de> for CollectionPartitions { "collection", "partition_selector", "partitionSelector", - "disk_backlog_threshold", - "diskBacklogThreshold", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { Collection, PartitionSelector, - DiskBacklogThreshold, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -72,7 +61,6 @@ impl<'de> serde::Deserialize<'de> for CollectionPartitions { match value { "collection" => Ok(GeneratedField::Collection), "partitionSelector" | "partition_selector" => Ok(GeneratedField::PartitionSelector), - "diskBacklogThreshold" | "disk_backlog_threshold" => Ok(GeneratedField::DiskBacklogThreshold), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -94,7 +82,6 @@ impl<'de> serde::Deserialize<'de> for CollectionPartitions { { let mut collection__ = None; let mut partition_selector__ = None; - let mut disk_backlog_threshold__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Collection => { @@ -109,27 +96,18 @@ impl<'de> serde::Deserialize<'de> for CollectionPartitions { } partition_selector__ = map_.next_value()?; } - GeneratedField::DiskBacklogThreshold => { - if disk_backlog_threshold__.is_some() { - return Err(serde::de::Error::duplicate_field("diskBacklogThreshold")); - } - disk_backlog_threshold__ = - Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; - } } } Ok(CollectionPartitions { collection: collection__, partition_selector: partition_selector__, - disk_backlog_threshold: disk_backlog_threshold__.unwrap_or_default(), }) } } deserializer.deserialize_struct("shuffle.CollectionPartitions", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for FrontierChunk { +impl serde::Serialize for Frontier { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -143,7 +121,7 @@ impl serde::Serialize for FrontierChunk { if !self.flushed_lsn.is_empty() { len += 1; } - let mut struct_ser = serializer.serialize_struct("shuffle.FrontierChunk", len)?; + let mut struct_ser = serializer.serialize_struct("shuffle.Frontier", len)?; if !self.journals.is_empty() { struct_ser.serialize_field("journals", &self.journals)?; } @@ -153,7 +131,7 @@ impl serde::Serialize for FrontierChunk { struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for FrontierChunk { +impl<'de> serde::Deserialize<'de> for Frontier { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where @@ -201,13 +179,13 @@ impl<'de> serde::Deserialize<'de> for FrontierChunk { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = FrontierChunk; + type Value = Frontier; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct shuffle.FrontierChunk") + formatter.write_str("struct shuffle.Frontier") } - fn visit_map(self, mut map_: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -232,13 +210,13 @@ impl<'de> serde::Deserialize<'de> for FrontierChunk { } } } - Ok(FrontierChunk { + Ok(Frontier { journals: journals__.unwrap_or_default(), flushed_lsn: flushed_lsn__.unwrap_or_default(), }) } } - deserializer.deserialize_struct("shuffle.FrontierChunk", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("shuffle.Frontier", FIELDS, GeneratedVisitor) } } impl serde::Serialize for JournalFrontier { @@ -970,9 +948,6 @@ impl serde::Serialize for log_request::Open { if self.log_shard_index != 0 { len += 1; } - if self.disk_backlog_threshold != 0 { - len += 1; - } let mut struct_ser = serializer.serialize_struct("shuffle.LogRequest.Open", len)?; if self.session_id != 0 { struct_ser.serialize_field("sessionId", &self.session_id)?; @@ -986,11 +961,6 @@ impl serde::Serialize for log_request::Open { if self.log_shard_index != 0 { struct_ser.serialize_field("logShardIndex", &self.log_shard_index)?; } - if self.disk_backlog_threshold != 0 { - #[allow(clippy::needless_borrow)] - #[allow(clippy::needless_borrows_for_generic_args)] - struct_ser.serialize_field("diskBacklogThreshold", ToString::to_string(&self.disk_backlog_threshold).as_str())?; - } struct_ser.end() } } @@ -1008,8 +978,6 @@ impl<'de> serde::Deserialize<'de> for log_request::Open { "sliceShardIndex", "log_shard_index", "logShardIndex", - "disk_backlog_threshold", - "diskBacklogThreshold", ]; #[allow(clippy::enum_variant_names)] @@ -1018,7 +986,6 @@ impl<'de> serde::Deserialize<'de> for log_request::Open { Shards, SliceShardIndex, LogShardIndex, - DiskBacklogThreshold, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -1044,7 +1011,6 @@ impl<'de> serde::Deserialize<'de> for log_request::Open { "shards" => Ok(GeneratedField::Shards), "sliceShardIndex" | "slice_shard_index" => Ok(GeneratedField::SliceShardIndex), "logShardIndex" | "log_shard_index" => Ok(GeneratedField::LogShardIndex), - "diskBacklogThreshold" | "disk_backlog_threshold" => Ok(GeneratedField::DiskBacklogThreshold), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -1068,7 +1034,6 @@ impl<'de> serde::Deserialize<'de> for log_request::Open { let mut shards__ = None; let mut slice_shard_index__ = None; let mut log_shard_index__ = None; - let mut disk_backlog_threshold__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::SessionId => { @@ -1101,14 +1066,6 @@ impl<'de> serde::Deserialize<'de> for log_request::Open { Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } - GeneratedField::DiskBacklogThreshold => { - if disk_backlog_threshold__.is_some() { - return Err(serde::de::Error::duplicate_field("diskBacklogThreshold")); - } - disk_backlog_threshold__ = - Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; - } } } Ok(log_request::Open { @@ -1116,7 +1073,6 @@ impl<'de> serde::Deserialize<'de> for log_request::Open { shards: shards__.unwrap_or_default(), slice_shard_index: slice_shard_index__.unwrap_or_default(), log_shard_index: log_shard_index__.unwrap_or_default(), - disk_backlog_threshold: disk_backlog_threshold__.unwrap_or_default(), }) } } @@ -1590,7 +1546,7 @@ impl serde::Serialize for SessionRequest { if self.open.is_some() { len += 1; } - if self.resume_checkpoint_chunk.is_some() { + if self.resume_checkpoint.is_some() { len += 1; } if self.next_checkpoint.is_some() { @@ -1600,8 +1556,8 @@ impl serde::Serialize for SessionRequest { if let Some(v) = self.open.as_ref() { struct_ser.serialize_field("open", v)?; } - if let Some(v) = self.resume_checkpoint_chunk.as_ref() { - struct_ser.serialize_field("resumeCheckpointChunk", v)?; + if let Some(v) = self.resume_checkpoint.as_ref() { + struct_ser.serialize_field("resumeCheckpoint", v)?; } if let Some(v) = self.next_checkpoint.as_ref() { struct_ser.serialize_field("nextCheckpoint", v)?; @@ -1617,8 +1573,8 @@ impl<'de> serde::Deserialize<'de> for SessionRequest { { const FIELDS: &[&str] = &[ "open", - "resume_checkpoint_chunk", - "resumeCheckpointChunk", + "resume_checkpoint", + "resumeCheckpoint", "next_checkpoint", "nextCheckpoint", ]; @@ -1626,7 +1582,7 @@ impl<'de> serde::Deserialize<'de> for SessionRequest { #[allow(clippy::enum_variant_names)] enum GeneratedField { Open, - ResumeCheckpointChunk, + ResumeCheckpoint, NextCheckpoint, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -1650,7 +1606,7 @@ impl<'de> serde::Deserialize<'de> for SessionRequest { { match value { "open" => Ok(GeneratedField::Open), - "resumeCheckpointChunk" | "resume_checkpoint_chunk" => Ok(GeneratedField::ResumeCheckpointChunk), + "resumeCheckpoint" | "resume_checkpoint" => Ok(GeneratedField::ResumeCheckpoint), "nextCheckpoint" | "next_checkpoint" => Ok(GeneratedField::NextCheckpoint), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } @@ -1672,7 +1628,7 @@ impl<'de> serde::Deserialize<'de> for SessionRequest { V: serde::de::MapAccess<'de>, { let mut open__ = None; - let mut resume_checkpoint_chunk__ = None; + let mut resume_checkpoint__ = None; let mut next_checkpoint__ = None; while let Some(k) = map_.next_key()? { match k { @@ -1682,11 +1638,11 @@ impl<'de> serde::Deserialize<'de> for SessionRequest { } open__ = map_.next_value()?; } - GeneratedField::ResumeCheckpointChunk => { - if resume_checkpoint_chunk__.is_some() { - return Err(serde::de::Error::duplicate_field("resumeCheckpointChunk")); + GeneratedField::ResumeCheckpoint => { + if resume_checkpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("resumeCheckpoint")); } - resume_checkpoint_chunk__ = map_.next_value()?; + resume_checkpoint__ = map_.next_value()?; } GeneratedField::NextCheckpoint => { if next_checkpoint__.is_some() { @@ -1698,7 +1654,7 @@ impl<'de> serde::Deserialize<'de> for SessionRequest { } Ok(SessionRequest { open: open__, - resume_checkpoint_chunk: resume_checkpoint_chunk__, + resume_checkpoint: resume_checkpoint__, next_checkpoint: next_checkpoint__, }) } @@ -1896,15 +1852,15 @@ impl serde::Serialize for SessionResponse { if self.opened.is_some() { len += 1; } - if self.next_checkpoint_chunk.is_some() { + if self.next_checkpoint.is_some() { len += 1; } let mut struct_ser = serializer.serialize_struct("shuffle.SessionResponse", len)?; if let Some(v) = self.opened.as_ref() { struct_ser.serialize_field("opened", v)?; } - if let Some(v) = self.next_checkpoint_chunk.as_ref() { - struct_ser.serialize_field("nextCheckpointChunk", v)?; + if let Some(v) = self.next_checkpoint.as_ref() { + struct_ser.serialize_field("nextCheckpoint", v)?; } struct_ser.end() } @@ -1917,14 +1873,14 @@ impl<'de> serde::Deserialize<'de> for SessionResponse { { const FIELDS: &[&str] = &[ "opened", - "next_checkpoint_chunk", - "nextCheckpointChunk", + "next_checkpoint", + "nextCheckpoint", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { Opened, - NextCheckpointChunk, + NextCheckpoint, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -1947,7 +1903,7 @@ impl<'de> serde::Deserialize<'de> for SessionResponse { { match value { "opened" => Ok(GeneratedField::Opened), - "nextCheckpointChunk" | "next_checkpoint_chunk" => Ok(GeneratedField::NextCheckpointChunk), + "nextCheckpoint" | "next_checkpoint" => Ok(GeneratedField::NextCheckpoint), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -1968,7 +1924,7 @@ impl<'de> serde::Deserialize<'de> for SessionResponse { V: serde::de::MapAccess<'de>, { let mut opened__ = None; - let mut next_checkpoint_chunk__ = None; + let mut next_checkpoint__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Opened => { @@ -1977,17 +1933,17 @@ impl<'de> serde::Deserialize<'de> for SessionResponse { } opened__ = map_.next_value()?; } - GeneratedField::NextCheckpointChunk => { - if next_checkpoint_chunk__.is_some() { - return Err(serde::de::Error::duplicate_field("nextCheckpointChunk")); + GeneratedField::NextCheckpoint => { + if next_checkpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("nextCheckpoint")); } - next_checkpoint_chunk__ = map_.next_value()?; + next_checkpoint__ = map_.next_value()?; } } } Ok(SessionResponse { opened: opened__, - next_checkpoint_chunk: next_checkpoint_chunk__, + next_checkpoint: next_checkpoint__, }) } } diff --git a/crates/proto-flow/tests/regression.rs b/crates/proto-flow/tests/regression.rs index 66fecf104e3..76ad4fe8a00 100644 --- a/crates/proto-flow/tests/regression.rs +++ b/crates/proto-flow/tests/regression.rs @@ -613,13 +613,17 @@ fn ex_materialize_request() -> materialize::Request { range: Some(ex_range()), state_json: json!({"connector": {"state": 42}}).to_string().into(), }), - acknowledge: Some(materialize::request::Acknowledge {}), + acknowledge: Some(materialize::request::Acknowledge { + connector_state_patches_json: json!([{"acked": true}]).to_string().into(), + }), load: Some(materialize::request::Load { binding: 12, key_packed: vec![86, 75, 30, 9].into(), key_json: json!([42, "hi"]).to_string().into(), }), - flush: Some(materialize::request::Flush {}), + flush: Some(materialize::request::Flush { + connector_state_patches_json: json!([{"flushed": 1}]).to_string().into(), + }), store: Some(materialize::request::Store { binding: 3, key_packed: vec![90, 21, 0].into(), @@ -632,6 +636,7 @@ fn ex_materialize_request() -> materialize::Request { }), start_commit: Some(materialize::request::StartCommit { runtime_checkpoint: Some(ex_consumer_checkpoint()), + connector_state_patches_json: json!([{"started": "commit"}]).to_string().into(), }), internal: ex_internal(), } diff --git a/crates/proto-flow/tests/snapshots/regression__materialize_request_json.snap b/crates/proto-flow/tests/snapshots/regression__materialize_request_json.snap index 69d9b0eb475..0b0b280a671 100644 --- a/crates/proto-flow/tests/snapshots/regression__materialize_request_json.snap +++ b/crates/proto-flow/tests/snapshots/regression__materialize_request_json.snap @@ -1,6 +1,5 @@ --- source: crates/proto-flow/tests/regression.rs -assertion_line: 1052 expression: json_test(msg) --- { @@ -514,7 +513,9 @@ expression: json_test(msg) "key": [42,"hi"], "keyPacked": "VkseCQ==" }, - "flush": {}, + "flush": { + "connectorStatePatches": [{"flushed":1}] + }, "store": { "binding": 3, "key": [true,null], @@ -551,8 +552,11 @@ expression: json_test(msg) "ackIntents": { "an/ack/journal": "AwQCBQ==" } - } + }, + "connectorStatePatches": [{"started":"commit"}] + }, + "acknowledge": { + "connectorStatePatches": [{"acked":true}] }, - "acknowledge": {}, "$internal": "EgJIaRgB" } diff --git a/crates/proto-flow/tests/snapshots/regression__materialize_request_proto.snap b/crates/proto-flow/tests/snapshots/regression__materialize_request_proto.snap index 45039b45bae..29047aad4aa 100644 --- a/crates/proto-flow/tests/snapshots/regression__materialize_request_proto.snap +++ b/crates/proto-flow/tests/snapshots/regression__materialize_request_proto.snap @@ -1,6 +1,5 @@ --- source: crates/proto-flow/tests/regression.rs -assertion_line: 1058 expression: proto_test(msg) --- |0a150808 12117b22 73706563 223a2263| ......{"spec":"c 00000000 @@ -191,16 +190,20 @@ expression: proto_test(msg) |7b22636f 6e6e6563 746f7222 3a7b2273| {"connector":{"s 00000b90 |74617465 223a3432 7d7d2a13 080c1209| tate":42}}*..... 00000ba0 |5b34322c 22686922 5d1a0456 4b1e0932| [42,"hi"]..VK..2 00000bb0 -|003a4508 03120b5b 74727565 2c6e756c| .:E....[true,nul 00000bc0 -|6c5d1a03 5a150022 125b332e 31343135| l]..Z..".[3.1415 00000bd0 -|392c2266 69656c64 21225d2a 023c5b32| 9,"field!"]*.<[2 00000be0 -|137b2266 756c6c22 3a22646f 63756d65| .{"full":"docume 00000bf0 -|6e74227d 38014001 42660a64 0a4a0a15| nt"}8.@.Bf.d.J.. 00000c00 -|612f7265 61642f6a 6f75726e 616c3b73| a/read/journal;s 00000c10 -|75666669 78123108 b9601215 0a050309| uffix.1..`...... 00000c20 -|08050712 0c09e321 00000000 000010d7| .......!........ 00000c30 -|0812150a 05070c66 2b1d120c 09350100| .......f+....5.. 00000c40 -|00000000 0010ae11 12160a0e 616e2f61| ............an/a 00000c50 -|636b2f6a 6f75726e 616c1204 03040205| ck/journal...... 00000c60 -|4a00a206 06120248 691801| J......Hi.. 00000c70 - 00000c7b +|110a0f5b 7b22666c 75736865 64223a31| ...[{"flushed":1 00000bc0 +|7d5d3a45 0803120b 5b747275 652c6e75| }]:E....[true,nu 00000bd0 +|6c6c5d1a 035a1500 22125b33 2e313431| ll]..Z..".[3.141 00000be0 +|35392c22 6669656c 6421225d 2a023c5b| 59,"field!"]*.<[ 00000bf0 +|32137b22 66756c6c 223a2264 6f63756d| 2.{"full":"docum 00000c00 +|656e7422 7d380140 01427e0a 640a4a0a| ent"}8.@.B~.d.J. 00000c10 +|15612f72 6561642f 6a6f7572 6e616c3b| .a/read/journal; 00000c20 +|73756666 69781231 08b96012 150a0503| suffix.1..`..... 00000c30 +|09080507 120c09e3 21000000 00000010| ........!....... 00000c40 +|d7081215 0a05070c 662b1d12 0c093501| ........f+....5. 00000c50 +|00000000 000010ae 1112160a 0e616e2f| .............an/ 00000c60 +|61636b2f 6a6f7572 6e616c12 04030402| ack/journal..... 00000c70 +|0512165b 7b227374 61727465 64223a22| ...[{"started":" 00000c80 +|636f6d6d 6974227d 5d4a120a 105b7b22| commit"}]J...[{" 00000c90 +|61636b65 64223a74 7275657d 5da20606| acked":true}]... 00000ca0 +|12024869 1801| ..Hi.. 00000cb0 + 00000cb6 diff --git a/crates/proto-gazette/build.rs b/crates/proto-gazette/build.rs index 657aa13f304..33229d94b1f 100644 --- a/crates/proto-gazette/build.rs +++ b/crates/proto-gazette/build.rs @@ -5,7 +5,12 @@ fn main() { prost_build::Config::new() .out_dir(&b.src_dir) - .bytes(&["AppendRequest.content", "ReadResponse.content"]) + .btree_map(&["."]) + .bytes(&[ + ".consumer.Checkpoint", + ".protocol.AppendRequest.content", + ".protocol.ReadResponse.content", + ]) .file_descriptor_set_path(&b.descriptor_path) .compile_well_known_types() .extern_path(".google.protobuf", "::pbjson_types") @@ -16,6 +21,7 @@ fn main() { .out_dir(&b.src_dir) .register_descriptors(&std::fs::read(b.descriptor_path).expect("read descriptors")) .unwrap() + .btree_map(["."]) .build(&[".protocol", ".consumer", ".recoverylog"]) .expect("building pbjson"); } diff --git a/crates/proto-gazette/src/consumer.rs b/crates/proto-gazette/src/consumer.rs index db94df2b08f..9ef44e20849 100644 --- a/crates/proto-gazette/src/consumer.rs +++ b/crates/proto-gazette/src/consumer.rs @@ -241,14 +241,17 @@ pub mod replica_status { #[derive(Clone, PartialEq, ::prost::Message)] pub struct Checkpoint { /// Sources is metadata of journals consumed by the shard. - #[prost(map = "string, message", tag = "1")] - pub sources: ::std::collections::HashMap<::prost::alloc::string::String, checkpoint::Source>, + #[prost(btree_map = "string, message", tag = "1")] + pub sources: + ::prost::alloc::collections::BTreeMap<::prost::alloc::string::String, checkpoint::Source>, /// AckIntents is acknowledgement intents to be written to journals to which /// uncommitted messages were published during the transaction which produced /// this Checkpoint. - #[prost(map = "string, bytes", tag = "2")] - pub ack_intents: - ::std::collections::HashMap<::prost::alloc::string::String, ::prost::alloc::vec::Vec>, + #[prost(btree_map = "string, bytes", tag = "2")] + pub ack_intents: ::prost::alloc::collections::BTreeMap< + ::prost::alloc::string::String, + ::prost::bytes::Bytes, + >, } /// Nested message and enum types in `Checkpoint`. pub mod checkpoint { @@ -267,8 +270,8 @@ pub mod checkpoint { /// RFC 4122 v1 node identifiers (see message.ProducerID). #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] pub struct ProducerEntry { - #[prost(bytes = "vec", tag = "1")] - pub id: ::prost::alloc::vec::Vec, + #[prost(bytes = "bytes", tag = "1")] + pub id: ::prost::bytes::Bytes, #[prost(message, optional, tag = "2")] pub state: ::core::option::Option, } @@ -391,8 +394,8 @@ pub struct StatRequest { /// Journals and offsets which must be reflected in a completed consumer /// transaction before Stat returns, blocking if required. Offsets of journals /// not read by this shard are ignored. - #[prost(map = "string, int64", tag = "3")] - pub read_through: ::std::collections::HashMap<::prost::alloc::string::String, i64>, + #[prost(btree_map = "string, int64", tag = "3")] + pub read_through: ::prost::alloc::collections::BTreeMap<::prost::alloc::string::String, i64>, /// Optional extension of the StatRequest. #[prost(bytes = "vec", tag = "100")] pub extension: ::prost::alloc::vec::Vec, @@ -407,8 +410,8 @@ pub struct StatResponse { pub header: ::core::option::Option, /// Journals and offsets read through by the most recent completed consumer /// transaction. - #[prost(map = "string, int64", tag = "3")] - pub read_through: ::std::collections::HashMap<::prost::alloc::string::String, i64>, + #[prost(btree_map = "string, int64", tag = "3")] + pub read_through: ::prost::alloc::collections::BTreeMap<::prost::alloc::string::String, i64>, /// Journals and offsets this shard has published through, including /// acknowledgements, as-of the most recent completed consumer transaction. /// @@ -420,8 +423,8 @@ pub struct StatResponse { /// to provide read-your-writes consistency, even if written events pass /// through multiple intermediate consumers and arbitrary transformations /// before arriving at the materialized view which is ultimately queried. - #[prost(map = "string, int64", tag = "4")] - pub publish_at: ::std::collections::HashMap<::prost::alloc::string::String, i64>, + #[prost(btree_map = "string, int64", tag = "4")] + pub publish_at: ::prost::alloc::collections::BTreeMap<::prost::alloc::string::String, i64>, /// Optional extension of the StatResponse. #[prost(bytes = "vec", tag = "100")] pub extension: ::prost::alloc::vec::Vec, diff --git a/crates/proto-gazette/src/consumer.serde.rs b/crates/proto-gazette/src/consumer.serde.rs index 3810f4dcc66..b3f3b11b7ae 100644 --- a/crates/proto-gazette/src/consumer.serde.rs +++ b/crates/proto-gazette/src/consumer.serde.rs @@ -482,7 +482,7 @@ impl<'de> serde::Deserialize<'de> for Checkpoint { return Err(serde::de::Error::duplicate_field("sources")); } sources__ = Some( - map_.next_value::>()? + map_.next_value::>()? ); } GeneratedField::AckIntents => { @@ -490,7 +490,7 @@ impl<'de> serde::Deserialize<'de> for Checkpoint { return Err(serde::de::Error::duplicate_field("ackIntents")); } ack_intents__ = Some( - map_.next_value::>>()? + map_.next_value::>>()? .into_iter().map(|(k,v)| (k, v.0)).collect() ); } @@ -2517,7 +2517,7 @@ impl<'de> serde::Deserialize<'de> for StatRequest { return Err(serde::de::Error::duplicate_field("readThrough")); } read_through__ = Some( - map_.next_value::>>()? + map_.next_value::>>()? .into_iter().map(|(k,v)| (k, v.0)).collect() ); } @@ -2684,7 +2684,7 @@ impl<'de> serde::Deserialize<'de> for StatResponse { return Err(serde::de::Error::duplicate_field("readThrough")); } read_through__ = Some( - map_.next_value::>>()? + map_.next_value::>>()? .into_iter().map(|(k,v)| (k, v.0)).collect() ); } @@ -2693,7 +2693,7 @@ impl<'de> serde::Deserialize<'de> for StatResponse { return Err(serde::de::Error::duplicate_field("publishAt")); } publish_at__ = Some( - map_.next_value::>>()? + map_.next_value::>>()? .into_iter().map(|(k,v)| (k, v.0)).collect() ); } diff --git a/crates/proto-gazette/src/uuid.rs b/crates/proto-gazette/src/uuid.rs index 6e9373f3dd2..07c29bb477a 100644 --- a/crates/proto-gazette/src/uuid.rs +++ b/crates/proto-gazette/src/uuid.rs @@ -142,6 +142,15 @@ impl Clock { Some(pbjson_types::Timestamp { seconds, nanos }) } + /// Saturating difference of `a - b` expressed as `Duration`. + pub fn delta(a: Self, b: Self) -> std::time::Duration { + let (a_s, a_n) = a.to_unix(); + let (b_s, b_n) = b.to_unix(); + let a = std::time::Duration::new(a_s, a_n); + let b = std::time::Duration::new(b_s, b_n); + a.saturating_sub(b) + } + pub const UNIX_EPOCH: Self = Clock::from_unix(0, 0); } diff --git a/crates/proto-grpc/src/runtime.rs b/crates/proto-grpc/src/runtime.rs index fa7ee8c0f8a..e283f7691ef 100644 --- a/crates/proto-grpc/src/runtime.rs +++ b/crates/proto-grpc/src/runtime.rs @@ -863,3 +863,717 @@ pub mod connector_proxy_server { const NAME: &'static str = SERVICE_NAME; } } +/// Generated client implementations. +#[cfg(feature = "runtime_client")] +pub mod leader_client { + #![allow( + unused_variables, + dead_code, + missing_docs, + clippy::wildcard_imports, + clippy::let_unit_value + )] + use tonic::codegen::http::Uri; + use tonic::codegen::*; + /// # ===================================================================== + /// Runtime V2 protocol + /// + /// Three roles cooperate over two bidirectional gRPC services: + /// + /// Controller drives a shard's lifecycle. The Go runtime in production, + /// `flowctl preview` in dev, or a unit-test harness. The + /// Controller talks to the Shard service. + /// Shard runtime-next's per-shard service. Each Shard stream + /// terminates *both* the Controller-bound protocol and an + /// outbound Leader stream, translating between them and + /// the connector RPC. + /// Leader sidecar service that joins shards into a session and + /// drives multi-shard transactions. Hosted on the reactor + /// machine to which the task's shard zero is assigned. + /// + /// Both services carry the same `Materialize` / `Derive` message types, + /// and exactly one field is set per message. + /// + /// Shard zero is special: it hosts the recovery log, forwards `Task` to + /// the leader, and is the *only* shard that receives `Apply` and + /// `Persist`. Non-zero shards have no recovery log; they spin up against + /// a typically-empty RocksDB and MUST send `Recover` and `Opened` + /// messages equal to `default()` at session startup. The leader treats + /// any deviation as a hard protocol error — this is the migration guard + /// against stale per-shard state from before consolidation. + /// + /// All shards participate in every transaction (idle shards send empty + /// deltas). Shard topology is fail-stop: any shard drop aborts the + /// session and the next session re-joins from PRIMARY. + /// + /// Connector vs leader message naming: The runtime messages overlap by + /// name with messages of the connector protocols. We use the short-hand + /// `C:Foo` for the connector message and `L:Foo` for the runtime / + /// leader message. So `C:Acknowledged` is the connector's response on + /// its acknowledge phase; `L:Acknowledged` is what a shard relays back + /// to the leader after observing `C:Acknowledged`. + /// + /// Hinted vs committed Frontier (used by Recover, Persist, Open): + /// `committed` state is durable in the recovery log. `hinted` state is + /// what the leader *intended* to commit but may or may not yet be + /// durable. Remote-authoritative connectors may commit a transaction in + /// the *endpoint* ahead of the recovery log: at session startup we + /// compare the close-clock embedded in `C:Opened`'s checkpoint against + /// our recovered `hinted_close_clock` to detect this case and promote + /// the hinted Frontier to committed. + /// + /// Leader joins a task's shards into a session and drives multi-shard + /// transactions. Shard instances dial the leader, which joins them and + /// leads them through Recover → Apply → Open → ⟨transactions⟩ → Stopped. + /// + /// The protocol is "inverted" compared to typical client-server: sometimes + /// the shard initiates (Join, Recover, Flushed) and sometimes the leader + /// initiates (Open, Load, Flush, Persist). + #[derive(Debug, Clone)] + pub struct LeaderClient { + inner: tonic::client::Grpc, + } + impl LeaderClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl LeaderClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + std::marker::Send + 'static, + ::Error: Into + std::marker::Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> LeaderClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + >>::Error: + Into + std::marker::Send + std::marker::Sync, + { + LeaderClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + /// rpc Derive(stream .runtime.Derive) returns (stream .runtime.Derive); + pub async fn materialize( + &mut self, + request: impl tonic::IntoStreamingRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner.ready().await.map_err(|e| { + tonic::Status::unknown(format!("Service was not ready: {}", e.into())) + })?; + let codec = tonic_prost::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/runtime.Leader/Materialize"); + let mut req = request.into_streaming_request(); + req.extensions_mut() + .insert(GrpcMethod::new("runtime.Leader", "Materialize")); + self.inner.streaming(req, path, codec).await + } + } +} +/// Generated server implementations. +#[cfg(feature = "runtime_server")] +pub mod leader_server { + #![allow( + unused_variables, + dead_code, + missing_docs, + clippy::wildcard_imports, + clippy::let_unit_value + )] + use tonic::codegen::*; + /// Generated trait containing gRPC methods that should be implemented for use with LeaderServer. + #[async_trait] + pub trait Leader: std::marker::Send + std::marker::Sync + 'static { + /// Server streaming response type for the Materialize method. + type MaterializeStream: tonic::codegen::tokio_stream::Stream< + Item = std::result::Result<::proto_flow::runtime::Materialize, tonic::Status>, + > + std::marker::Send + + 'static; + /// rpc Derive(stream .runtime.Derive) returns (stream .runtime.Derive); + async fn materialize( + &self, + request: tonic::Request>, + ) -> std::result::Result, tonic::Status>; + } + /// # ===================================================================== + /// Runtime V2 protocol + /// + /// Three roles cooperate over two bidirectional gRPC services: + /// + /// Controller drives a shard's lifecycle. The Go runtime in production, + /// `flowctl preview` in dev, or a unit-test harness. The + /// Controller talks to the Shard service. + /// Shard runtime-next's per-shard service. Each Shard stream + /// terminates *both* the Controller-bound protocol and an + /// outbound Leader stream, translating between them and + /// the connector RPC. + /// Leader sidecar service that joins shards into a session and + /// drives multi-shard transactions. Hosted on the reactor + /// machine to which the task's shard zero is assigned. + /// + /// Both services carry the same `Materialize` / `Derive` message types, + /// and exactly one field is set per message. + /// + /// Shard zero is special: it hosts the recovery log, forwards `Task` to + /// the leader, and is the *only* shard that receives `Apply` and + /// `Persist`. Non-zero shards have no recovery log; they spin up against + /// a typically-empty RocksDB and MUST send `Recover` and `Opened` + /// messages equal to `default()` at session startup. The leader treats + /// any deviation as a hard protocol error — this is the migration guard + /// against stale per-shard state from before consolidation. + /// + /// All shards participate in every transaction (idle shards send empty + /// deltas). Shard topology is fail-stop: any shard drop aborts the + /// session and the next session re-joins from PRIMARY. + /// + /// Connector vs leader message naming: The runtime messages overlap by + /// name with messages of the connector protocols. We use the short-hand + /// `C:Foo` for the connector message and `L:Foo` for the runtime / + /// leader message. So `C:Acknowledged` is the connector's response on + /// its acknowledge phase; `L:Acknowledged` is what a shard relays back + /// to the leader after observing `C:Acknowledged`. + /// + /// Hinted vs committed Frontier (used by Recover, Persist, Open): + /// `committed` state is durable in the recovery log. `hinted` state is + /// what the leader *intended* to commit but may or may not yet be + /// durable. Remote-authoritative connectors may commit a transaction in + /// the *endpoint* ahead of the recovery log: at session startup we + /// compare the close-clock embedded in `C:Opened`'s checkpoint against + /// our recovered `hinted_close_clock` to detect this case and promote + /// the hinted Frontier to committed. + /// + /// Leader joins a task's shards into a session and drives multi-shard + /// transactions. Shard instances dial the leader, which joins them and + /// leads them through Recover → Apply → Open → ⟨transactions⟩ → Stopped. + /// + /// The protocol is "inverted" compared to typical client-server: sometimes + /// the shard initiates (Join, Recover, Flushed) and sometimes the leader + /// initiates (Open, Load, Flush, Persist). + #[derive(Debug)] + pub struct LeaderServer { + inner: Arc, + accept_compression_encodings: EnabledCompressionEncodings, + send_compression_encodings: EnabledCompressionEncodings, + max_decoding_message_size: Option, + max_encoding_message_size: Option, + } + impl LeaderServer { + pub fn new(inner: T) -> Self { + Self::from_arc(Arc::new(inner)) + } + pub fn from_arc(inner: Arc) -> Self { + Self { + inner, + accept_compression_encodings: Default::default(), + send_compression_encodings: Default::default(), + max_decoding_message_size: None, + max_encoding_message_size: None, + } + } + pub fn with_interceptor(inner: T, interceptor: F) -> InterceptedService + where + F: tonic::service::Interceptor, + { + InterceptedService::new(Self::new(inner), interceptor) + } + /// Enable decompressing requests with the given encoding. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.accept_compression_encodings.enable(encoding); + self + } + /// Compress responses with the given encoding, if the client supports it. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.send_compression_encodings.enable(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.max_decoding_message_size = Some(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.max_encoding_message_size = Some(limit); + self + } + } + impl tonic::codegen::Service> for LeaderServer + where + T: Leader, + B: Body + std::marker::Send + 'static, + B::Error: Into + std::marker::Send + 'static, + { + type Response = http::Response; + type Error = std::convert::Infallible; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut Context<'_>, + ) -> Poll> { + Poll::Ready(Ok(())) + } + fn call(&mut self, req: http::Request) -> Self::Future { + match req.uri().path() { + "/runtime.Leader/Materialize" => { + #[allow(non_camel_case_types)] + struct MaterializeSvc(pub Arc); + impl + tonic::server::StreamingService<::proto_flow::runtime::Materialize> + for MaterializeSvc + { + type Response = ::proto_flow::runtime::Materialize; + type ResponseStream = T::MaterializeStream; + type Future = + BoxFuture, tonic::Status>; + fn call( + &mut self, + request: tonic::Request< + tonic::Streaming<::proto_flow::runtime::Materialize>, + >, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = + async move { ::materialize(&inner, request).await }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let method = MaterializeSvc(inner); + let codec = tonic_prost::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.streaming(method, req).await; + Ok(res) + }; + Box::pin(fut) + } + _ => Box::pin(async move { + let mut response = http::Response::new(tonic::body::Body::default()); + let headers = response.headers_mut(); + headers.insert( + tonic::Status::GRPC_STATUS, + (tonic::Code::Unimplemented as i32).into(), + ); + headers.insert( + http::header::CONTENT_TYPE, + tonic::metadata::GRPC_CONTENT_TYPE, + ); + Ok(response) + }), + } + } + } + impl Clone for LeaderServer { + fn clone(&self) -> Self { + let inner = self.inner.clone(); + Self { + inner, + accept_compression_encodings: self.accept_compression_encodings, + send_compression_encodings: self.send_compression_encodings, + max_decoding_message_size: self.max_decoding_message_size, + max_encoding_message_size: self.max_encoding_message_size, + } + } + } + /// Generated gRPC service name + pub const SERVICE_NAME: &str = "runtime.Leader"; + impl tonic::server::NamedService for LeaderServer { + const NAME: &'static str = SERVICE_NAME; + } +} +/// Generated client implementations. +#[cfg(feature = "runtime_client")] +pub mod shard_client { + #![allow( + unused_variables, + dead_code, + missing_docs, + clippy::wildcard_imports, + clippy::let_unit_value + )] + use tonic::codegen::http::Uri; + use tonic::codegen::*; + /// Shard is the controller-facing bidirectional service exposed by + /// runtime-next. It carries the same message types as Leader on the + /// wire, but is a distinct service: each Shard stream terminates one + /// shard's controller-bound protocol and (separately) dials the leader + /// sidecar's Leader service, translating between the two and the + /// connector RPC. + /// + /// Stream modes are determined by the first message: + /// + /// * Session mode: first message is `SessionLoop`. Then any number of + /// leader sessions cycle through Join → ... → Stopped, terminated by + /// controller EOF. + /// * Unary mode: first message is `spec` or `validate`. The leader is + /// not involved; the handler dials a transient connector and streams + /// the response back. A Validate may follow a Spec on the same stream. + #[derive(Debug, Clone)] + pub struct ShardClient { + inner: tonic::client::Grpc, + } + impl ShardClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl ShardClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + std::marker::Send + 'static, + ::Error: Into + std::marker::Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> ShardClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + >>::Error: + Into + std::marker::Send + std::marker::Sync, + { + ShardClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + /// rpc Derive(stream .runtime.Derive) returns (stream .runtime.Derive); + pub async fn materialize( + &mut self, + request: impl tonic::IntoStreamingRequest, + ) -> std::result::Result< + tonic::Response>, + tonic::Status, + > { + self.inner.ready().await.map_err(|e| { + tonic::Status::unknown(format!("Service was not ready: {}", e.into())) + })?; + let codec = tonic_prost::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static("/runtime.Shard/Materialize"); + let mut req = request.into_streaming_request(); + req.extensions_mut() + .insert(GrpcMethod::new("runtime.Shard", "Materialize")); + self.inner.streaming(req, path, codec).await + } + } +} +/// Generated server implementations. +#[cfg(feature = "runtime_server")] +pub mod shard_server { + #![allow( + unused_variables, + dead_code, + missing_docs, + clippy::wildcard_imports, + clippy::let_unit_value + )] + use tonic::codegen::*; + /// Generated trait containing gRPC methods that should be implemented for use with ShardServer. + #[async_trait] + pub trait Shard: std::marker::Send + std::marker::Sync + 'static { + /// Server streaming response type for the Materialize method. + type MaterializeStream: tonic::codegen::tokio_stream::Stream< + Item = std::result::Result<::proto_flow::runtime::Materialize, tonic::Status>, + > + std::marker::Send + + 'static; + /// rpc Derive(stream .runtime.Derive) returns (stream .runtime.Derive); + async fn materialize( + &self, + request: tonic::Request>, + ) -> std::result::Result, tonic::Status>; + } + /// Shard is the controller-facing bidirectional service exposed by + /// runtime-next. It carries the same message types as Leader on the + /// wire, but is a distinct service: each Shard stream terminates one + /// shard's controller-bound protocol and (separately) dials the leader + /// sidecar's Leader service, translating between the two and the + /// connector RPC. + /// + /// Stream modes are determined by the first message: + /// + /// * Session mode: first message is `SessionLoop`. Then any number of + /// leader sessions cycle through Join → ... → Stopped, terminated by + /// controller EOF. + /// * Unary mode: first message is `spec` or `validate`. The leader is + /// not involved; the handler dials a transient connector and streams + /// the response back. A Validate may follow a Spec on the same stream. + #[derive(Debug)] + pub struct ShardServer { + inner: Arc, + accept_compression_encodings: EnabledCompressionEncodings, + send_compression_encodings: EnabledCompressionEncodings, + max_decoding_message_size: Option, + max_encoding_message_size: Option, + } + impl ShardServer { + pub fn new(inner: T) -> Self { + Self::from_arc(Arc::new(inner)) + } + pub fn from_arc(inner: Arc) -> Self { + Self { + inner, + accept_compression_encodings: Default::default(), + send_compression_encodings: Default::default(), + max_decoding_message_size: None, + max_encoding_message_size: None, + } + } + pub fn with_interceptor(inner: T, interceptor: F) -> InterceptedService + where + F: tonic::service::Interceptor, + { + InterceptedService::new(Self::new(inner), interceptor) + } + /// Enable decompressing requests with the given encoding. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.accept_compression_encodings.enable(encoding); + self + } + /// Compress responses with the given encoding, if the client supports it. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.send_compression_encodings.enable(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.max_decoding_message_size = Some(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.max_encoding_message_size = Some(limit); + self + } + } + impl tonic::codegen::Service> for ShardServer + where + T: Shard, + B: Body + std::marker::Send + 'static, + B::Error: Into + std::marker::Send + 'static, + { + type Response = http::Response; + type Error = std::convert::Infallible; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut Context<'_>, + ) -> Poll> { + Poll::Ready(Ok(())) + } + fn call(&mut self, req: http::Request) -> Self::Future { + match req.uri().path() { + "/runtime.Shard/Materialize" => { + #[allow(non_camel_case_types)] + struct MaterializeSvc(pub Arc); + impl + tonic::server::StreamingService<::proto_flow::runtime::Materialize> + for MaterializeSvc + { + type Response = ::proto_flow::runtime::Materialize; + type ResponseStream = T::MaterializeStream; + type Future = + BoxFuture, tonic::Status>; + fn call( + &mut self, + request: tonic::Request< + tonic::Streaming<::proto_flow::runtime::Materialize>, + >, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = + async move { ::materialize(&inner, request).await }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let method = MaterializeSvc(inner); + let codec = tonic_prost::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.streaming(method, req).await; + Ok(res) + }; + Box::pin(fut) + } + _ => Box::pin(async move { + let mut response = http::Response::new(tonic::body::Body::default()); + let headers = response.headers_mut(); + headers.insert( + tonic::Status::GRPC_STATUS, + (tonic::Code::Unimplemented as i32).into(), + ); + headers.insert( + http::header::CONTENT_TYPE, + tonic::metadata::GRPC_CONTENT_TYPE, + ); + Ok(response) + }), + } + } + } + impl Clone for ShardServer { + fn clone(&self) -> Self { + let inner = self.inner.clone(); + Self { + inner, + accept_compression_encodings: self.accept_compression_encodings, + send_compression_encodings: self.send_compression_encodings, + max_decoding_message_size: self.max_decoding_message_size, + max_encoding_message_size: self.max_encoding_message_size, + } + } + } + /// Generated gRPC service name + pub const SERVICE_NAME: &str = "runtime.Shard"; + impl tonic::server::NamedService for ShardServer { + const NAME: &'static str = SERVICE_NAME; + } +} diff --git a/crates/runtime-next/Cargo.toml b/crates/runtime-next/Cargo.toml new file mode 100644 index 00000000000..0c541b58a92 --- /dev/null +++ b/crates/runtime-next/Cargo.toml @@ -0,0 +1,83 @@ +[package] +name = "runtime-next" +version.workspace = true +rust-version.workspace = true +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true + +[dependencies] +assemble = { path = "../assemble" } +async-process = { path = "../async-process" } +connector-init = { path = "../connector-init" } +coroutines = { path = "../coroutines" } +dekaf-connector = { path = "../dekaf-connector" } +doc = { path = "../doc", features = ["combine"] } +extractors = { path = "../extractors" } +flow-client-next = { path = "../flow-client-next" } +gazette = { path = "../gazette" } +iam-auth = { path = "../iam-auth" } +json = { path = "../json" } +labels = { path = "../labels" } +locate-bin = { path = "../locate-bin" } +models = { path = "../models" } +ops = { path = "../ops" } +proto-flow = { path = "../proto-flow" } +proto-gazette = { path = "../proto-gazette" } +proto-grpc = { path = "../proto-grpc", features = [ + "capture_client", + "capture_server", + "derive_client", + "derive_server", + "materialize_client", + "materialize_server", + "runtime_client", + "runtime_server", +] } +publisher = { path = "../publisher" } +shuffle = { path = "../shuffle" } +simd-doc = { path = "../simd-doc" } +tables = { path = "../tables" } +tokens = { path = "../tokens" } +tuple = { path = "../tuple" } +unseal = { path = "../unseal" } + +anyhow = { workspace = true } +aws-config = { workspace = true } +aws-credential-types = { workspace = true } +aws-sdk-sts = { workspace = true } +base64 = { workspace = true } +bytes = { workspace = true } +chrono = { workspace = true } +futures = { workspace = true } +handlebars = { workspace = true } +json-patch = { workspace = true } +jsonwebtoken = { workspace = true } +librocksdb-sys = { workspace = true } +pbjson-types = { workspace = true } +prost = { workspace = true } +rand = { workspace = true } +url = { workspace = true } +reqwest = { workspace = true } +rocksdb = { workspace = true } +serde = { workspace = true } +serde_json = { workspace = true } +tempfile = { workspace = true } +thiserror = { workspace = true } +time = { workspace = true } +tokio = { workspace = true } +tokio-stream = { workspace = true } +tonic = { workspace = true } +tracing = { workspace = true } +tracing-subscriber = { workspace = true } +uuid = { workspace = true, features = ["v4"] } +xxhash-rust = { workspace = true } +zeroize = { workspace = true } + +[dev-dependencies] +axum = { workspace = true } +insta = { workspace = true } +quickcheck = { workspace = true } +tokio = { workspace = true, features = ["test-util", "macros"] } diff --git a/crates/runtime-next/README.md b/crates/runtime-next/README.md new file mode 100644 index 00000000000..6587a75d6c2 --- /dev/null +++ b/crates/runtime-next/README.md @@ -0,0 +1,149 @@ +# runtime-next + +Rust task runtime that replaces the Go-driven transaction loop. For +derivations and materializations, a per-task **Shuffle Leader** coordinates +multi-shard transactions; the Go runtime becomes a thin shim for shard +lifecycle and ops logs. Captures use a simpler model with independent +per-shard transaction loops. + +This crate hosts both sides of the runtime-v2 protocol: + +- **`Shard`** — per-shard, controller-facing gRPC service. One instance per + assigned shard, embedded in the Go reactor process via CGO over a per-shard + Unix domain socket. +- **`Leader`** — sidecar gRPC service. One sidecar per reactor machine + (systemd-supervised, lifetime-bound to the reactor), serving every task + whose shard zero is assigned to a reactor on that machine. + +"Controller" throughout this crate is whatever drives a shard's lifecycle: +the Go runtime in production, an in-process driver such as `flowctl +preview`, or a unit-test harness. The crate is agnostic to which. + +## Architecture + +``` +Reactor machine + ├─ reactor process(es) (Go + Rust via CGO) + │ ├─ Go: Etcd watch, shard lifecycle, ops logs (OUTSIDE_TXN) + │ └─ Per-shard TaskService (this crate, via CGO): + │ ├─ Connector driving, combining, publishing CONTINUE_TXN docs + │ ├─ In-memory state (connector state, checkpoints, max-keys) + │ ├─ Derive/materialize: state persisted via Leader's Persist/Persisted + │ │ (RocksDB + Go Recorder on the shard hosting the recovery log) + │ └─ Capture: per-shard RocksDB with Go Recorder + │ + └─ shuffle sidecar process (Rust, one per machine) + ├─ Shuffle Leader service (this crate, per-task via Join) + ├─ Shuffle service (`crates/shuffle`, Session/Slice/Log RPCs) + └─ Listens on the fixed shuffle port, shared fleet-wide +``` + +The Gazette consumer framework's transaction lifecycle is **bypassed +entirely**: `StartReadingMessages` drains without producing messages, so +`BeginTxn`/`ConsumeMessage`/`FinalizeTxn`/`StartCommit` are never invoked. +The framework still manages assignment, Etcd state, and recovery log setup; +all document processing and commit sequencing happen here, via the Shuffle +Leader protocol (derive/materialize) or per-shard transaction loop (capture). + +For derive/materialize, only **shard zero** hosts a recovery log. Non-zero +shards have `ShardSpec.recovery_log_prefix = ""`, spin up instantly, and +acquire state through the Leader protocol — eliminating per-shard recovery +logs and simplifying crash recovery. The sidecar and per-shard TaskServices +communicate solely by gRPC; no shared memory. + +## Layout + +``` +src/ +├── lib.rs # crate root, shared helpers (Verify, LogHandler, Accumulator) +├── task_service.rs # CGO entry point: binds UDS, serves Shard service +├── publish.rs # Rust journal publishing (used by both leader and shard) +├── patches.rs # wire format for connector-state patch streams +│ +├── leader/ # sidecar Leader service +│ ├── service.rs # gRPC entry, per-task Join rendezvous +│ ├── join.rs # protocol primitives for joining shards into a session +│ └── materialize/ +│ ├── handler.rs # gRPC stream handler, dispatches to startup/actor +│ ├── startup.rs # Recover / Open / Apply / Recovered phase +│ ├── fsm.rs # pipelined HeadFSM / TailFSM state machines +│ ├── actor.rs # event loop driving open / commit / acknowledge / trigger +│ ├── frontier_mapping.rs # consumer.Checkpoint <-> shuffle::Frontier +│ ├── triggers.rs # webhook trigger delivery +│ └── task.rs # per-task state held by the leader actor +│ +└── shard/ # per-shard controller-facing service + ├── service.rs # gRPC entry, dispatches by task type + ├── recovery.rs # Persist <-> RocksDB WriteBatch encode/decode + scan + ├── rocksdb.rs # single Persist application path (capture will reuse) + └── materialize/ + ├── handler.rs # gRPC stream handler + ├── startup.rs # join leader, scan RocksDB, open connector + ├── scan.rs # in-memory state recovery from RocksDB + ├── connector.rs # connector RPC bridging + ├── actor.rs # per-shard transaction loop + └── drain.rs # graceful drain on Stop / CloseNow +``` + +## Key entry points + +- **`TaskService::new`** (`task_service.rs`) — CGO constructor invoked by Go + on shard assignment. Wires the data-plane environment (FQDN, control API, + signing key), constructs a `shard::Service`, and serves it over a per-shard + Unix domain socket. +- **`leader::Service::new`** (`leader/service.rs`) — sidecar process builds + one of these and registers it on the shuffle port alongside `shuffle::Service`. +- **`shard::Service`** (`shard/service.rs`) — implements the controller-facing + `Shard` trait. Each bidi stream terminates *both* the controller-bound + protocol and the leader-bound protocol, translating between them and the + connector RPC. + +The only messages that flow controller → runtime-next → leader unmodified are +`Stop` and `CloseNow`. + +## Protocol + +`go/protocols/runtime/runtime.proto` defines `Leader` and `Shard` RPCs. Both +carry the same `Derive` / `Materialize` message types; field semantics are +documented inline in the proto. + +## Invariants + +- **No dependency on `runtime`.** The legacy `runtime` crate may depend on + `runtime-next`, never the reverse. Files shared between the two live + physically in `runtime/` and are pulled in via `#[path]`. See the comment + at the top of `lib.rs`. +- **Shard-local processing is identical for all shards.** Shard zero is + special only at session startup (forwards `Task` to the leader; receives + Apply/Persist). The per-transaction loop has no `if shard_zero` branches — + the leader decides what each shard does and shards follow uniformly. +- **All shards participate in every transaction**, even idle ones — they + send empty deltas and respond immediately. Shard topology is fail-stop: + any shard drop aborts the session and tears down all surviving shards. + The Gazette allocator reassigns; the next session re-joins from PRIMARY. +- **Migration guards on non-zero shards.** Non-zero shards still open a + (typically empty, tempdir-backed) RocksDB and run the same `scan` path on + session start. Recovery is expected to error if a non-zero shard observes + unexpected committed state, or if its connector reports a non-empty + runtime checkpoint at `Opened` — both indicate stale per-shard state from + before consolidation. +- **`shard/rocksdb.rs` is the single Persist application path.** Capture + (unimplemented) will reuse it by synthesizing `Persist` messages locally + rather than receiving them from a leader. + +## Coexistence with `runtime` + +This crate ships **deployed inert** alongside the existing `runtime` crate; +both coexist on the same reactor. Per-task feature flags on shard labels +select which runtime serves a given task — all shards of a task use the +same runtime. The shuffle sidecar runs uniformly on every reactor machine +regardless of which tasks are assigned; old-runtime tasks simply don't talk +to it. Rollback for any task is a feature-flag flip. + +## Status + +- `leader::materialize` and `shard::materialize` are implemented. +- `leader::derive` and `leader::capture` are not yet implemented (see + commented modules in `leader/mod.rs`). +- Capture is unimplemented in `shard/`; the intended approach is local + Persist synthesis as noted above. diff --git a/crates/runtime-next/src/container.rs b/crates/runtime-next/src/container.rs new file mode 100644 index 00000000000..262848525f8 --- /dev/null +++ b/crates/runtime-next/src/container.rs @@ -0,0 +1,976 @@ +use crate::RuntimeProtocol; +use anyhow::Context; +use futures::channel::oneshot; +use proto_flow::{flow, runtime}; +use std::collections::BTreeMap; +use tokio::io::AsyncBufReadExt; + +// Port on which flow-connector-init listens for requests. +// This is its default, made explicit here. +// This number was chosen because it seemed unlikely that a connector would try to use it. +// The main thing is that we want to avoid any common port numbers to avoid conflicts with +// connectors. +const CONNECTOR_INIT_PORT: u16 = 49092; + +const RUNTIME_PROTO_LABEL: &str = "FLOW_RUNTIME_PROTOCOL"; +const USAGE_RATE_LABEL: &str = "dev.estuary.usage-rate"; +const PORT_PUBLIC_LABEL_PREFIX: &str = "dev.estuary.port-public."; +const PORT_PROTO_LABEL_PREFIX: &str = "dev.estuary.port-proto."; + +// TODO(johnny): Consider better packaging and versioning of `flow-connector-init`. +const CONNECTOR_INIT_IMAGE: &str = "ghcr.io/estuary/flow:v0.5.24-30-ga3eba41f95"; +const CONNECTOR_INIT_IMAGE_PATH: &str = "/usr/local/bin/flow-connector-init"; + +/// Determines the protocol of an image. If the image has a `FLOW_RUNTIME_PROTOCOL` label, +/// then it's value is used. Otherwise, this will apply a simple heuristic based on the image name, +/// for backward compatibility purposes. An error will be returned if it fails to inspect the image +/// or parse the label. The image must already have been pulled before calling this function. +pub async fn flow_runtime_protocol(image: &str) -> anyhow::Result { + if image.starts_with(models::DEKAF_IMAGE_NAME_PREFIX) { + return Ok(RuntimeProtocol::Materialize); + } + if !image.ends_with(":local") { + docker_pull(image).await.context("pulling image")?; + } + + let inspect_output = docker_cmd(&["inspect", image]) + .await + .context("inspecting image")?; + + let inspection = parse_image_inspection(&inspect_output)?; + tracing::info!( + %image, + inspection = ?ops::DebugJson(&inspection), + "inspected connector image" + ); + Ok(inspection.runtime_protocol) +} + +/// Start an image connector container, returning its description and a dialed tonic Channel. +/// The container is attached to the given `network`, and its logs are dispatched to `log_handler`. +/// `task_name` and `task_type` are used only to label the container. +pub async fn start( + image: &str, + log_handler: impl crate::LogHandler, + log_level: ops::LogLevel, + network: &str, + task_name: &str, + task_type: ops::TaskType, + plane: crate::Plane, +) -> anyhow::Result<(runtime::Container, tonic::transport::Channel, Guard)> { + validate_connector_image(image, plane)?; + + // Many operational contexts only allow for docker volume mounts + // from certain locations: + // * Docker for Mac restricts file shares to /User, /tmp, and a couple others. + // * Estuary's current K8s deployments use a separate docker daemon container + // within the pod, having a common /tmp tempdir volume. + // + // So, we use temporaries to ensure that files are readable within the container. + let tmp_connector_init = + tempfile::NamedTempFile::new().context("creating temp for flow-connector-init")?; + let mut tmp_docker_inspect = + tempfile::NamedTempFile::new().context("creating temp for docker inspect output")?; + + // Change mode of `docker_inspect` to be readable by all users. + // This is required because the effective container user may have a different UID. + #[cfg(unix)] + { + use std::os::unix::prelude::PermissionsExt; + let mut perms = tmp_docker_inspect.as_file_mut().metadata()?.permissions(); + perms.set_mode(0o644); + tmp_docker_inspect.as_file_mut().set_permissions(perms)?; + } + + // Concurrently 1) find or fetch a copy of `flow-connector-init`, copying it + // into a temp path, and 2) inspect the image, also copying into a temp path, + // and parsing its advertised network ports. + let ((), image_inspection) = futures::try_join!( + find_connector_init_and_copy(tmp_connector_init.path()), + inspect_image_and_copy(image, tmp_docker_inspect.path()), + )?; + + // Close our open files but retain a deletion guard. + let tmp_connector_init = tmp_connector_init.into_temp_path(); + let tmp_docker_inspect = tmp_docker_inspect.into_temp_path(); + + // This is default `docker run` behavior if --network is not provided. + let network = if network == "" { "bridge" } else { network }; + let log_level = log_level.or(ops::LogLevel::Warn); + + // Generate a unique name for this container instance. + let name = unique_container_name(); + + let mut docker_args = vec![ + "run".to_string(), + // Remove the docker container upon its exit. + "--rm".to_string(), + // Addressable name of this connector. + format!("--name={name}"), + // Network to which the container should attach. + format!("--network={}", network), + // The entrypoint into a connector is always flow-connector-init, + // which will delegate to the actual entrypoint of the connector. + "--entrypoint=/flow-connector-init".to_string(), + // Disable logging of connector containers. + "--log-driver=none".to_string(), + // Mount the flow-connector-init binary and `docker inspect` output. + format!( + "--mount=type=bind,source={},target=/flow-connector-init", + tmp_connector_init.to_string_lossy() + ), + format!( + "--mount=type=bind,source={},target=/image-inspect.json", + tmp_docker_inspect.to_string_lossy(), + ), + // Thread-through the logging configuration of the connector. + "--env=LOG_FORMAT=json".to_string(), + format!("--env=LOG_LEVEL={}", log_level.as_str_name()), + // Cgroup memory / CPU resource limits. + "--memory".to_string(), + connector_memory_limit(), + "--cpus".to_string(), + connector_cpu_limit(), + // For now, we support only Linux amd64 connectors. + "--platform=linux/amd64".to_string(), + // Attach labels that let us group connector resource usage under a few dimensions. + format!("--label=image={}", image), + format!("--label=task-name={}", task_name), + format!("--label=task-type={}", task_type.as_str_name()), + ]; + + // When running locally, we publish ports so that connectors are accessible + // on the host from Windows and MacOS (e.x. Docker Desktop). + if matches!(plane, crate::Plane::Local) { + // Bind a random port, and then check what port was given to us. + let l = tokio::net::TcpListener::bind("0.0.0.0:0") + .await + .context("failed to bind random port")?; + let port = l.local_addr()?.port(); + std::mem::drop(l); // Release so it can be re-bound. + + docker_args.append(&mut vec![ + // Support Docker Desktop in non-production contexts (for example, `flowctl`) + // where the container IP is not directly addressable. As an alternative, + // we ask Docker to provide mapped host ports that are then advertised + // in the attached runtime::Container description. + format!("--publish=0.0.0.0:{port}:{CONNECTOR_INIT_PORT}"), + "--publish-all".to_string(), + ]); + } + + if let Some(cgroup_parent) = std::env::var("CONNECTOR_CGROUP_PARENT").ok() { + docker_args.append(&mut vec!["--cgroup-parent".to_string(), cgroup_parent]); + } + + docker_args.append(&mut vec![ + // Image to run. + image.to_string(), + // The following are arguments of flow-connector-init, not docker. + "--image-inspect-json-path=/image-inspect.json".to_string(), + format!("--port={CONNECTOR_INIT_PORT}"), + ]); + + tracing::debug!(docker_args=?docker_args, "invoking docker"); + + let mut process: async_process::Child = async_process::Command::new(docker_cli()) + .args(docker_args) + .stdin(async_process::Stdio::null()) + .stdout(async_process::Stdio::null()) + .stderr(async_process::Stdio::piped()) + .spawn() + .context("failed to docker run the connector")? + .into(); + + // We've started the container and will need to inspect for its IP address. + // Docker has unfortunate race handling and will happily return an empty IPAddress for + // a created or even a running container while it's still performing background setup. + // The only reliable way to determine if the container is "ready" is to wait for + // our inner flow-connector-init process to produce its startup log. + let (ready_tx, ready_rx) = oneshot::channel::<()>(); + + // Service process stderr by decoding ops::Logs and sending to our handler. + let stderr = process.stderr.take().unwrap(); + let quoted_task_name: bytes::Bytes = format!("\"{task_name}\"").into(); + tokio::spawn(async move { + let mut stderr = tokio::io::BufReader::new(stderr); + let mut line = String::new(); + + // Wait for a non-empty read of stderr to complete or EOF/error. + // Note that `flow-connector-init` writes one whitespace byte on startup. + if let Ok(buf) = stderr.fill_buf().await { + if buf.first() == Some(&b' ') { + stderr.consume(1); // Discard. + } + } + std::mem::drop(ready_tx); // Signal that we're ready. + + loop { + line.clear(); + + match stderr.read_line(&mut line).await { + Err(error) => { + tracing::error!(%error, "failed to read from connector stderr"); + break; + } + Ok(0) => break, // Clean EOF. + Ok(_) => (), + } + + match serde_json::from_str(&line) { + Ok(log) => { + let sanitized = sanitize_event_type("ed_task_name, log); + log_handler.log(&sanitized) + } + Err(error) => { + tracing::error!(?error, %line, "failed to parse ops::Log from container"); + } + } + } + }); + + // Wait for container to become ready, or close its stderr (likely due to a crash), + // or for a minute to elapse (timeout). + tokio::select! { + _ = tokio::time::sleep(std::time::Duration::from_secs(60)) => { + anyhow::bail!("timeout waiting for the container to become ready"); + } + _ = ready_rx => (), + } + + // Ask docker for network configuration that it assigned to the container. + let (ip_addr, mapped_host_ports) = inspect_container_network(&name).await?; + + // Dial the gRPC endpoint hosted by `flow-connector-init` within the container context. + let init_address = if let Some(addr) = mapped_host_ports.get(&(CONNECTOR_INIT_PORT as u32)) { + format!("http://{addr}") + } else { + format!("http://{ip_addr}:{CONNECTOR_INIT_PORT}") + }; + let channel = tonic::transport::Endpoint::new(init_address.clone()) + .expect("formatting endpoint address") + .connect_timeout(std::time::Duration::from_secs(5)) + .http2_keep_alive_interval(std::time::Duration::from_secs(5)) + .connect() + .await + .with_context(|| { + format!("failed to connect to container connector-init at {init_address}") + })?; + + tracing::info!( + %image, + %init_address, + %ip_addr, + mapped_host_ports = ?ops::DebugJson(&mapped_host_ports), + %name, + image_inspection = ?ops::DebugJson(&image_inspection), + %task_name, + ?task_type, + "started connector container" + ); + let usage_rate = image_inspection.usage_rate; + let network_ports = image_inspection.network_ports; + + Ok(( + runtime::Container { + ip_addr: format!("{ip_addr}"), + network_ports, + usage_rate, + mapped_host_ports, + }, + channel, + Guard { + _tmp_connector_init: tmp_connector_init, + _tmp_docker_inspect: tmp_docker_inspect, + _process: process, + }, + )) +} + +/// Validates that a connector image is allowed to run in this data-plane. +fn validate_connector_image(image: &str, plane: crate::Plane) -> anyhow::Result<()> { + if matches!(plane, crate::Plane::Public) { + if !image.starts_with("ghcr.io/estuary/") { + anyhow::bail!( + "connector image '{image}' is not allowed in public data planes: only Estuary-managed images are permitted" + ); + } + if image.starts_with("ghcr.io/estuary/derive-python:") { + anyhow::bail!("Python derivations may only run in private data-planes"); + } + } + Ok(()) +} + +/// Performs a basic validation of logs that represent events, to restrict +/// connectors to emitting connectorStatus and configUpdate events for the +/// currently running task. +fn sanitize_event_type(quoted_task_name: &bytes::Bytes, mut log: ops::Log) -> ops::Log { + match log + .fields_json_map + .get("eventType") + .map(|v| v == "\"connectorStatus\"" || v == "\"configUpdate\"") + { + Some(true) => { + match log + .fields_json_map + .get("eventTarget") + .map(|t| t == quoted_task_name) + { + Some(true) => { /* eventTarget is valid */ } + Some(false) => { + let v = log.fields_json_map.remove("eventTarget").unwrap(); + log.fields_json_map + .insert("_sanitized_eventTarget".to_string(), v); + log.fields_json_map + .insert("eventTarget".to_string(), quoted_task_name.clone()); + } + None => { + log.fields_json_map + .insert("eventTarget".to_string(), quoted_task_name.clone()); + } + } + } + Some(false) => { + let v = log.fields_json_map.remove("eventType").unwrap(); + log.fields_json_map + .insert("_sanitized_eventType".to_string(), v); + } + None => { /* this is not an event */ } + } + log +} + +/// Guard contains a running image container instance, +/// which will be stopped and cleaned up when the Guard is dropped. +pub struct Guard { + _tmp_connector_init: tempfile::TempPath, + _tmp_docker_inspect: tempfile::TempPath, + _process: async_process::Child, +} + +fn unique_container_name() -> String { + let n = std::time::SystemTime::now() + .duration_since(std::time::UNIX_EPOCH) + .unwrap() + .as_nanos(); + + format!("fc_{:x}", n as u32) +} + +fn docker_cli() -> String { + std::env::var("DOCKER_CLI") + .ok() + .unwrap_or_else(|| "docker".to_string()) +} + +fn connector_memory_limit() -> String { + std::env::var("CONNECTOR_MEMORY_LIMIT") + .ok() + .unwrap_or_else(|| "1g".to_string()) +} + +fn connector_cpu_limit() -> String { + std::env::var("CONNECTOR_CPU_LIMIT") + .ok() + .unwrap_or_else(|| "2".to_string()) +} + +async fn docker_cmd(args: &[S]) -> anyhow::Result> +where + S: AsRef + std::fmt::Debug, +{ + let output = async_process::output(async_process::Command::new(docker_cli()).args(args)) + .await + .with_context(|| format!("failed to run docker command {args:?}"))?; + + if !output.status.success() { + anyhow::bail!( + "docker command {args:?} failed: {}", + String::from_utf8_lossy(&output.stderr), + ); + } + Ok(output.stdout) +} + +async fn docker_pull(image: &str) -> anyhow::Result<()> { + const MAX_RETRIES: u32 = 3; + const RETRY_DELAY: std::time::Duration = std::time::Duration::from_secs(2); + + for attempt in 1..=MAX_RETRIES { + let Err(err) = docker_cmd(&["pull", image, "--quiet"]).await else { + return Ok(()); + }; + + let err_str = format!("{err:#}"); + let is_transient = err_str.contains("TLS handshake timeout") + || err_str.contains("connection reset") + || err_str.contains("i/o timeout") + || err_str.contains("unexpected EOF"); + + if is_transient && attempt < MAX_RETRIES { + tracing::warn!( + %image, + attempt, + max_retries = MAX_RETRIES, + error = %err, + "transient error pulling image (will retry)" + ); + tokio::time::sleep(RETRY_DELAY).await; + } else { + return Err(err); + } + } + unreachable!() +} + +async fn inspect_container_network( + name: &str, +) -> anyhow::Result<(std::net::IpAddr, BTreeMap)> { + #[derive(serde::Deserialize)] + #[serde(rename_all = "PascalCase", deny_unknown_fields)] + struct HostPort { + host_ip: String, + host_port: String, + } + + #[derive(serde::Deserialize)] + struct Output { + status: String, + ip: std::net::IpAddr, + ports: BTreeMap>>, + } + + let output = docker_cmd(&[ + "inspect", + "--format", + r#"{ + "ip": "{{range.NetworkSettings.Networks}}{{.IPAddress}}{{end}}", + "ports": {{json .NetworkSettings.Ports}}, + "status": {{json .State.Status}} + }"#, + name, + ]) + .await + .context("failed to inspect a started docker container (did it crash?)")?; + + let output = String::from_utf8_lossy(&output); + let Output { status, ip, ports } = serde_json::from_str(&output) + .with_context(|| format!("malformed docker container inspection output: {output}"))?; + + if status != "running" { + anyhow::bail!("container failed to start; did it crash? (docker status is {status:?})"); + } + + let mut mapped_host_ports = BTreeMap::new(); + + for (container_port, mappings) in ports { + let Some(mappings) = mappings else { continue }; + + for HostPort { host_ip, host_port } in mappings { + if container_port.ends_with("/udp") { + continue; // Not supported. + } + + // `podman` inspect output will use an empty HostIp to represent + // dual-stack port bindings (either `::1` or `0.0.0.0`). + // `docker` will always emit a non-empty IP. + let host_ip = if host_ip.is_empty() { + "127.0.0.1".to_string() + } else { + host_ip + }; + + let host_ip: std::net::IpAddr = host_ip + .parse() + .with_context(|| format!("failed to parse HostIp: {host_ip:?}"))?; + + // Technically, ports are allowed to appear without the '/tcp' suffix. + let container_port = container_port + .strip_suffix("/tcp") + .unwrap_or(&container_port); + + let container_port = container_port.parse::().with_context(|| { + format!("invalid port in inspected NetworkSettings.Ports '{container_port}'") + })?; + let host_port = host_port.parse::().with_context(|| { + format!("invalid port in inspected NetworkSettings.Ports.*.HostPort '{host_port}'") + })?; + + _ = mapped_host_ports.insert( + container_port as u32, + if host_ip.is_ipv6() { + format!("[{host_ip}]:{host_port}") + } else { + format!("{host_ip}:{host_port}") + }, + ); + } + } + + Ok((ip, mapped_host_ports)) +} + +/// Information about a conector image, which is derived from `docker inspect` +#[derive(Debug, serde::Serialize)] +struct ImageInspection { + /// The type of connector + runtime_protocol: RuntimeProtocol, + /// Network ports that the connector wishes to expose + network_ports: Vec, + /// The number of usage credits per second that the connector incurs + usage_rate: f32, + /// A brief description of how the `usage_rate` was determined + usage_rate_source: &'static str, + /// The full id of the image, which allows determining when a given tag has been updated + /// by looking for changes to the id in the logs + id: String, + /// The creation timestamp of the image, for debugging purposes + image_created_at: String, +} + +fn parse_image_inspection(content: &[u8]) -> anyhow::Result { + #[derive(serde::Deserialize)] + #[serde(rename_all = "PascalCase")] + struct InspectConfig { + /// According to the [OCI spec](https://github.com/opencontainers/image-spec/blob/d60099175f88c47cd379c4738d158884749ed235/config.md?plain=1#L125) + /// `ExposedPorts` is a map where the keys are in the format `1234/tcp`, `456/udp`, or `789` (implicit default of tcp), and the values are + /// empty objects. The choice of `serde_json::Value` here is meant to convey that the actual values are irrelevant. + #[serde(default)] + exposed_ports: BTreeMap, + #[serde(default)] + labels: Option>, + } + + #[derive(serde::Deserialize)] + #[serde(rename_all = "PascalCase")] + struct InspectJson { + id: String, + created: String, + config: InspectConfig, + } + + // Deserialize into a destructured one-tuple. + let (InspectJson { + id, + created, + config: InspectConfig { + exposed_ports, + labels, + }, + },) = serde_json::from_slice(&content).with_context(|| { + format!( + "failed to parse `docker inspect` output: {}", + String::from_utf8_lossy(&content) + ) + })?; + + let labels = labels.unwrap_or_default(); + let mut network_ports = Vec::new(); + + for (exposed_port, _) in exposed_ports.iter() { + // We're unable to support UDP at this time. + if exposed_port.ends_with("/udp") { + continue; + } + // Technically, the ports are allowed to appear without the '/tcp' suffix, though + // I haven't actually observed that in practice. + let exposed_port = exposed_port.strip_suffix("/tcp").unwrap_or(exposed_port); + let number = exposed_port.parse::().with_context(|| { + format!("invalid key in inspected Config.ExposedPorts '{exposed_port}'") + })?; + + let protocol_label = format!("{PORT_PROTO_LABEL_PREFIX}{number}"); + let protocol = labels.get(&protocol_label).cloned(); + + let public_label = format!("{PORT_PUBLIC_LABEL_PREFIX}{number}"); + let public = labels + .get(&public_label) + .map(String::as_str) + .unwrap_or("false"); + let public = public.parse::().with_context(|| { + format!( + "invalid '{public_label}' label value: '{public}', must be either 'true' or 'false'" + ) + })?; + + network_ports.push(flow::NetworkPort { + number: number as u32, + protocol: protocol.unwrap_or_default(), + public, + }); + } + + let Some(rt_proto_label) = labels.get(RUNTIME_PROTO_LABEL) else { + anyhow::bail!("image is missing required '{RUNTIME_PROTO_LABEL}' label"); + }; + let runtime_protocol = + RuntimeProtocol::from_image_label(rt_proto_label.as_str()).map_err(|unknown| { + anyhow::anyhow!("image labels specify unknown protocol {RUNTIME_PROTO_LABEL}={unknown}") + })?; + + let (usage_rate, usage_rate_source) = if let Some(rate_value) = labels.get(USAGE_RATE_LABEL) { + let rate = rate_value + .parse::() + .with_context(|| format!("invalid '{USAGE_RATE_LABEL}' value {rate_value:?}"))?; + (rate, USAGE_RATE_LABEL) + } else { + if runtime_protocol == RuntimeProtocol::Derive { + (0.0f32, "default for derive protocol") + } else { + (1.0f32, "default for capture and materialize protocol") + } + }; + + Ok(ImageInspection { + runtime_protocol, + network_ports, + usage_rate, + usage_rate_source, + id, + image_created_at: created, + }) +} + +async fn find_connector_init_and_copy(tmp_path: &std::path::Path) -> anyhow::Result<()> { + // If we can locate an installed flow-connector-init, use that. + // This is common when developing or within a container workspace. + if let Ok(connector_init) = locate_bin::locate("flow-connector-init") { + tokio::fs::copy(connector_init, tmp_path).await?; + return Ok(()); + } + + // Create -- but don't start -- a container. + let name = format!("{}_fci", unique_container_name()); + docker_cmd(&[ + "create", + "--platform=linux/amd64", + &format!("--name={name}"), + CONNECTOR_INIT_IMAGE, + ]) + .await?; + + // Ask docker to copy the binary to our temp location. + docker_cmd(&[ + "cp", + &format!("{name}:{CONNECTOR_INIT_IMAGE_PATH}"), + &tmp_path.to_str().expect("temp is UTF-8"), + ]) + .await?; + + // Clean up the created container. + docker_cmd(&["rm", "--volumes", &name]).await?; + + Ok(()) +} + +async fn inspect_image_and_copy( + image: &str, + tmp_path: &std::path::Path, +) -> anyhow::Result { + if !image.ends_with(":local") { + docker_pull(image).await.context("pulling image")?; + } + + let inspect_content = docker_cmd(&["inspect", image]) + .await + .context("inspecting image")?; + + tokio::fs::write(tmp_path, &inspect_content) + .await + .context("writing docker inspect output")?; + + parse_image_inspection(&inspect_content) +} + +#[cfg(test)] +mod test { + use super::{parse_image_inspection, sanitize_event_type, start}; + use futures::stream::StreamExt; + use proto_flow::flow; + use serde_json::json; + + #[tokio::test] + async fn test_http_ingest_spec() { + if let Err(_) = locate_bin::locate("flow-connector-init") { + // Skip if `flow-connector-init` isn't available (yet). We're probably on CI. + // This test is useful as a sanity check for local development + // and we have plenty of other coverage during CI. + return; + } + + let (container, channel, _guard) = start( + "ghcr.io/estuary/source-http-ingest:dev", + ops::tracing_log_handler, + ops::LogLevel::Debug, + "", + "a-task-name", + proto_flow::ops::TaskType::Capture, + crate::Plane::Local, + ) + .await + .unwrap(); + + let mut rx = proto_grpc::capture::connector_client::ConnectorClient::new(channel) + .capture(futures::stream::once(async move { + serde_json::from_value(json!({ + "spec": {"connectorType": "IMAGE", "config": {}} + })) + .unwrap() + })) + .await + .unwrap() + .into_inner(); + + let resp = rx + .next() + .await + .expect("should get a spec response") + .unwrap(); + + assert!(resp.spec.is_some()); + + assert_eq!( + container.network_ports, + [flow::NetworkPort { + number: 8080, + protocol: String::new(), + public: true + }] + ); + + assert_eq!( + container + .mapped_host_ports + .keys() + .copied() + .collect::>(), + vec![8080, 49092] + ); + + assert_eq!(1.0, container.usage_rate); + } + + #[tokio::test] + async fn test_container_fails_to_start() { + if let Err(_) = locate_bin::locate("flow-connector-init") { + // Skip if `flow-connector-init` isn't available (yet). We're probably on CI. + // This test is useful as a sanity check for local development + // and we have plenty of other coverage during CI. + return; + } + + let Err(err) = start( + "alpine", // Not a connector. + ops::tracing_log_handler, + ops::LogLevel::Debug, + "", + "a-task-name", + proto_flow::ops::TaskType::Capture, + crate::Plane::Local, + ) + .await + else { + panic!("didn't crash") + }; + + println!("{err:#}") + } + + #[test] + fn test_parsing_inspection_output() { + let fixture = json!([ + { + "Id": "test-image-id", + "Created": "2024-02-02T14:39:11.958Z", + "Config":{ + "ExposedPorts": {"567/tcp":{}, "123/udp": {}, "789":{} }, + "Labels":{ + "FLOW_RUNTIME_PROTOCOL": "derive", + "dev.estuary.port-public.567":"true", + "dev.estuary.port-proto.789":"h2", + "dev.estuary.usage-rate": "1.3", + } + } + } + ]); + let inspection = parse_image_inspection(fixture.to_string().as_bytes()).unwrap(); + + assert_eq!( + &inspection.network_ports, + &[ + flow::NetworkPort { + number: 567, + protocol: String::new(), + public: true + }, + flow::NetworkPort { + number: 789, + protocol: "h2".to_string(), + public: false + }, + ] + ); + assert_eq!(1.3, inspection.usage_rate); + assert_eq!("test-image-id", &inspection.id); + assert_eq!("2024-02-02T14:39:11.958Z", &inspection.image_created_at); + } + + #[test] + fn parse_image_inspection_failure_cases() { + let fixture = json!([{ + "Id": "missing FLOW_RUNTIME_PROTOCOL", + "Created": "any time will do", + "Config": { + "Labels": {}, + } + }]); + insta::assert_debug_snapshot!(parse_image_inspection(fixture.to_string().as_bytes()).unwrap_err(), @r###""image is missing required 'FLOW_RUNTIME_PROTOCOL' label""###); + + let fixture = json!([ + { + "Id": "any", + "Created": "any time will do", + "Config":{ + "Labels": { + "FLOW_RUNTIME_PROTOCOL": "derive", + }, + "ExposedPorts": {"whoops":{}}, + } + } + ]); + insta::assert_debug_snapshot!(parse_image_inspection(fixture.to_string().as_bytes()).unwrap_err(), @r###" + Error { + context: "invalid key in inspected Config.ExposedPorts \'whoops\'", + source: ParseIntError { + kind: InvalidDigit, + }, + } + "###); + + let fixture = json!([ + { + "Id": "any", + "Created": "any time will do", + "Config":{ + "ExposedPorts": {"111/tcp":{}}, + "Labels":{ + "dev.estuary.port-public.111":"whoops", + "FLOW_RUNTIME_PROTOCOL": "derive", + } + } + } + ]); + insta::assert_debug_snapshot!(parse_image_inspection(fixture.to_string().as_bytes()).unwrap_err(), @r###" + Error { + context: "invalid \'dev.estuary.port-public.111\' label value: \'whoops\', must be either \'true\' or \'false\'", + source: ParseBoolError, + } + "###); + } + + #[test] + fn test_validate_connector_image_public_plane() { + let validate = |img| super::validate_connector_image(img, crate::Plane::Public); + + // Allowed: estuary connectors (except derive-python) + assert!(validate("ghcr.io/estuary/source-http-ingest:dev").is_ok()); + assert!(validate("ghcr.io/estuary/materialize-postgres:v1").is_ok()); + + // Blocked: derive-python + assert!(validate("ghcr.io/estuary/derive-python:latest").is_err()); + + // Blocked: non-estuary images + assert!(validate("alpine").is_err()); + assert!(validate("docker.io/estuary/source-postgres:v1").is_err()); + } + + #[test] + fn test_validate_connector_image_private_and_local() { + for plane in [crate::Plane::Private, crate::Plane::Local] { + let validate = |img| super::validate_connector_image(img, plane); + + // All images allowed in private/local planes + assert!(validate("ghcr.io/estuary/source-http-ingest:dev").is_ok()); + assert!(validate("ghcr.io/estuary/derive-python:latest").is_ok()); + assert!(validate("alpine").is_ok()); + assert!(validate("docker.io/custom/connector:v1").is_ok()); + } + } + + #[test] + fn test_log_event_validation() { + let abc = bytes::Bytes::from("\"a/b/c\""); + + let good = json!({ + "shard": { + "name": "a/b/c", + "keyBegin": "00000000", + "rClockBegin": "00000000", + "build": "1122334455667788" + }, + "level": "info", + "ts": "2025-01-02T03:04:05.06Z", + "message": "a test status", + "fields": { + "eventType": "connectorStatus", + "eventTarget": "a/b/c" + } + }); + let log: ops::Log = serde_json::from_value(good).unwrap(); + let out = sanitize_event_type(&abc, log.clone()); + assert_eq!(log, out); + + let naughty_type = json!({ + "shard": { + "name": "a/b/c", + "keyBegin": "00000000", + "rClockBegin": "00000000", + "build": "1122334455667788" + }, + "level": "info", + "ts": "2025-01-02T03:04:05.06Z", + "message": "a test status", + "fields": { + "eventType": "foo", + "eventTarget": "a/b/c" + } + }); + let log: ops::Log = serde_json::from_value(naughty_type).unwrap(); + let out = sanitize_event_type(&abc, log); + + assert!(!out.fields_json_map.contains_key("eventType")); + assert_eq!( + out.fields_json_map + .get("_sanitized_eventType") + .map(|v| v.as_ref()), + Some("\"foo\"".as_bytes()), + ); + + let naughty_target = json!({ + "shard": { + "name": "a/b/c", + "keyBegin": "00000000", + "rClockBegin": "00000000", + "build": "1122334455667788" + }, + "level": "info", + "ts": "2025-01-02T03:04:05.06Z", + "message": "a test status", + "fields": { + "eventType": "connectorStatus", + "eventTarget": "another/thing" + } + }); + let log: ops::Log = serde_json::from_value(naughty_target).unwrap(); + let out = sanitize_event_type(&abc, log); + assert_eq!( + out.fields_json_map.get("eventTarget").map(|v| v.as_ref()), + Some("\"a/b/c\"".as_bytes()), + ); + assert_eq!( + out.fields_json_map.get("eventType").map(|v| v.as_ref()), + Some("\"connectorStatus\"".as_bytes()), + ); + } +} diff --git a/crates/runtime-next/src/image_connector.rs b/crates/runtime-next/src/image_connector.rs new file mode 100644 index 00000000000..b58f8117bd7 --- /dev/null +++ b/crates/runtime-next/src/image_connector.rs @@ -0,0 +1,52 @@ +use super::container; +use futures::{Stream, StreamExt, future::BoxFuture}; +use tokio::sync::mpsc; + +/// StartRpcFuture is the response type of a function that starts a connector RPC. +pub type StartRpcFuture = + BoxFuture<'static, tonic::Result>>>; + +/// Serve an image-based connector by starting a container, dialing connector-init, +/// and then starting a gRPC request. +pub async fn serve( + image: String, // Container image to run. + log_handler: L, // Handler for connector logs. + log_level: ops::LogLevel, // Log-level of the connector, if known. + network: &str, // Container network to use. + request_rx: mpsc::Receiver, // Caller's input request stream. + start_rpc: StartRpc, // Begins RPC over a started container channel. + task_name: &str, // Name of this task, used to label container. + task_type: ops::TaskType, // Type of this task, for labeling container. + plane: crate::Plane, // Data-plane context in which the connector is running. +) -> anyhow::Result<( + impl Stream> + Send + use, + crate::proto::Container, +)> +where + Request: serde::Serialize + Send + 'static, + Response: Send + Sync + 'static, + StartRpc: Fn(tonic::transport::Channel, mpsc::Receiver) -> StartRpcFuture + + Send + + 'static, +{ + let (container, channel, guard) = container::start( + &image, + log_handler.clone(), + log_level, + &network, + &task_name, + task_type, + plane, + ) + .await?; + + // Start RPC over the container's gRPC `channel`. + let container_rx = (start_rpc)(channel, request_rx).await?.into_inner(); + + let container_rx = container_rx.map(move |result| { + let _guard = &guard; // Move into Stream. + result + }); + + Ok((container_rx, container)) +} diff --git a/crates/runtime-next/src/leader/join.rs b/crates/runtime-next/src/leader/join.rs new file mode 100644 index 00000000000..26158120644 --- /dev/null +++ b/crates/runtime-next/src/leader/join.rs @@ -0,0 +1,259 @@ +use crate::proto; +use anyhow::Context; +use futures::stream::BoxStream; +use tokio::sync::mpsc; + +/// A single filled slot: the shard's Join payload plus parked streams it owns. +pub struct JoinSlot { + // Join request of this shard. + pub join: proto::Join, + // Shard's client request stream. + pub request_rx: BoxStream<'static, tonic::Result>, + // Shard's client response channel. + pub response_tx: mpsc::UnboundedSender>, +} + +/// PendingJoin holds a partially complete set of shard-ordered JoinSlots. +#[derive(Default)] +pub struct PendingJoin { + slots: Vec>>, +} + +/// The outcome of feeding a single Join into `PendingJoin::register`. +pub enum JoinOutcome { + /// Slot filled. More shard Joins still need to arrive. + Pending { filled: usize, target: usize }, + /// Final slot filled with a consensus Join. + /// Caller takes ownership of slots and spawns the leader actor, + /// and must drop this join key from the service map. + Consensus(Vec>), + /// The Join disagrees with the established target, and should retry. + /// Caller takes ownership and broadcasts Joined retry, + /// and must drop this join key from the service map. + Disagreement(Vec>), +} + +impl PendingJoin { + /// Feed a Join into this PendingJoin. + pub fn register( + &mut self, + join: proto::Join, + request_rx: BoxStream<'static, tonic::Result>, + response_tx: mpsc::UnboundedSender>, + ) -> JoinOutcome { + let slot = JoinSlot { + join, + request_rx, + response_tx, + }; + let index = slot.join.shard_index as usize; + + // Is this is the first Join to arrive? + let Some(other) = self.slots.iter().flatten().next() else { + if slot.join.shards.len() > 1 { + self.slots.resize_with(slot.join.shards.len(), || None); + self.slots[index] = Some(slot); + + return JoinOutcome::Pending { + filled: 1, + target: self.slots.len(), + }; + } else { + // Single-shard topology is immediately complete. + return JoinOutcome::Consensus(vec![slot]); + } + }; + + // Is `index` already taken, or does `join` topology disagree? + if index >= self.slots.len() + || self.slots[index].is_some() + || slot.join.shards != other.join.shards + { + let slots: Vec> = self + .slots + .iter_mut() + .filter_map(Option::take) + .chain(std::iter::once(slot)) + .collect(); + + return JoinOutcome::Disagreement(slots); + } + + self.slots[index] = Some(slot); + let filled = self.slots.iter().flatten().count(); + + if filled == self.slots.len() { + let slots = self.slots.drain(..).map(Option::unwrap).collect(); + JoinOutcome::Consensus(slots) + } else { + JoinOutcome::Pending { + filled, + target: self.slots.len(), + } + } + } +} + +/// Validate a Join message is well-formed and internally consistent. +// This routine doesn't test for valid 2D tiling of shard ranges, +// because the `shuffle` session will do so. +pub fn validate(join: &proto::Join) -> anyhow::Result<&str> { + anyhow::ensure!(!join.shards.is_empty()); + anyhow::ensure!((join.shard_index as usize) < join.shards.len()); + anyhow::ensure!(!join.shuffle_directory.is_empty()); + + let mut task_name = ""; + + for (i, shard) in join.shards.iter().enumerate() { + let labeling = shard + .labeling + .as_ref() + .with_context(|| format!("shards[{i}].labeling is missing"))?; + + anyhow::ensure!(!labeling.task_name.is_empty()); + anyhow::ensure!(!shard.id.is_empty()); + anyhow::ensure!(shard.reactor.is_some()); + + if i == 0 { + task_name = &labeling.task_name; + } else { + anyhow::ensure!(labeling.task_name == task_name); + } + } + + Ok(task_name) +} + +#[cfg(test)] +mod tests { + use super::*; + use futures::StreamExt; + + fn shard(task_name: &str, id: &str) -> proto::join::Shard { + proto::join::Shard { + id: id.into(), + labeling: Some(::ops::ShardLabeling { + task_name: task_name.into(), + ..Default::default() + }), + reactor: Some(proto_gazette::broker::process_spec::Id::default()), + etcd_create_revision: 1, + } + } + + fn join(shards: Vec, shard_index: u32) -> proto::Join { + proto::Join { + shards, + shard_index, + shuffle_directory: "/tmp".into(), + ..Default::default() + } + } + + fn register( + pending: &mut PendingJoin, + j: proto::Join, + ) -> ( + JoinOutcome, + mpsc::UnboundedReceiver>, + ) { + let (tx, rx) = mpsc::unbounded_channel(); + let outcome = pending.register(j, futures::stream::empty().boxed(), tx); + (outcome, rx) + } + + #[test] + fn validate_cases() { + // (mutator, expect_ok) + let cases: &[(&str, fn(&mut proto::Join), bool)] = &[ + ("well-formed", |_| {}, true), + ("empty shards", |j| j.shards.clear(), false), + ("shard_index out of range", |j| j.shard_index = 9, false), + ( + "empty shuffle_directory", + |j| j.shuffle_directory.clear(), + false, + ), + ("missing labeling", |j| j.shards[0].labeling = None, false), + ( + "empty task_name", + |j| j.shards[0].labeling.as_mut().unwrap().task_name.clear(), + false, + ), + ("empty shard id", |j| j.shards[0].id.clear(), false), + ("missing reactor", |j| j.shards[0].reactor = None, false), + ( + "task_name mismatch", + |j| j.shards[1].labeling.as_mut().unwrap().task_name = "other".into(), + false, + ), + ]; + + for (name, mutate, expect_ok) in cases { + let mut j = join(vec![shard("task", "s0"), shard("task", "s1")], 0); + mutate(&mut j); + assert_eq!(validate(&j).is_ok(), *expect_ok, "case: {name}"); + } + // Spot-check the success path returns the task name. + let j = join(vec![shard("task", "s0")], 0); + assert_eq!(validate(&j).unwrap(), "task"); + } + + #[test] + fn single_shard_is_immediate_consensus() { + let mut p = PendingJoin::::default(); + let (out, _) = register(&mut p, join(vec![shard("t", "s0")], 0)); + assert!(matches!(out, JoinOutcome::Consensus(s) if s.len() == 1)); + assert!(p.slots.is_empty()); // Never resized. + } + + #[test] + fn multi_shard_assembles_in_order_despite_arrival_order() { + let mut p = PendingJoin::::default(); + let shards = vec![shard("t", "s0"), shard("t", "s1"), shard("t", "s2")]; + + // Arrive 2, 0, 1. + for (step, idx) in [2u32, 0, 1].into_iter().enumerate() { + let (out, _) = register(&mut p, join(shards.clone(), idx)); + match (step, out) { + (0 | 1, JoinOutcome::Pending { filled, target }) => { + assert_eq!((filled, target), (step + 1, 3)); + } + (2, JoinOutcome::Consensus(slots)) => { + let indices: Vec = slots.iter().map(|s| s.join.shard_index).collect(); + assert_eq!(indices, vec![0, 1, 2]); + } + (s, _) => panic!("unexpected outcome at step {s}"), + } + } + } + + #[test] + fn disagreement_cases() { + let three = vec![shard("t", "s0"), shard("t", "s1"), shard("t", "s2")]; + let two = vec![shard("t", "s0"), shard("t", "s1")]; + let mut two_alt = two.clone(); + two_alt[1].id = "other".into(); + + // (label, second join) + let cases: Vec<(&str, proto::Join)> = vec![ + ("duplicate shard_index", join(three.clone(), 0)), + ( + "index out of range vs established topology", + join(two.clone(), 1), + ), + ("same-size differing shards content", join(two_alt, 1)), + ]; + + for (name, second) in cases { + let mut p = PendingJoin::::default(); + let (_, _) = register(&mut p, join(three.clone(), 0)); + let (out, _) = register(&mut p, second); + let JoinOutcome::Disagreement(slots) = out else { + panic!("{name}: expected Disagreement"); + }; + assert_eq!(slots.len(), 2, "{name}: returns parked + new"); + assert!(p.slots.iter().all(Option::is_none), "{name}: drained"); + } + } +} diff --git a/crates/runtime-next/src/leader/materialize/actor.rs b/crates/runtime-next/src/leader/materialize/actor.rs new file mode 100644 index 00000000000..3ca2eb93ddf --- /dev/null +++ b/crates/runtime-next/src/leader/materialize/actor.rs @@ -0,0 +1,700 @@ +use super::{Task, fsm}; +use crate::proto; +use anyhow::Context; +use bytes::Bytes; +use futures::stream::{BoxStream, FuturesUnordered}; +use futures::{FutureExt, StreamExt, future::BoxFuture}; +use proto_gazette::{consumer, uuid}; +use std::collections::BTreeMap; +use tokio::sync::mpsc; + +/// Actor leads transactions of an established materialization task session. +pub struct Actor { + // Client used for trigger dispatch. + http_client: reqwest::Client, + // Future for an in-flight ACK intents write, if any. + intents_write_fut: Option>>, + // Optional full Frontier and Checkpoint, used for V1 rollback support. + legacy_checkpoint: Option<(shuffle::Frontier, consumer::Checkpoint)>, + // Publisher for stats and ACK intents, parked while no async operation is in-flight. + parked_publisher: Option, + // ACK intents to persist and append at later transaction stages. + pending_ack_intents: BTreeMap, + // One channel to each shard for synchronously sending it messages. + // We use UnboundedSender because Actor never "pumps" messages to shards: + // it follows a strict request/response pattern. + shard_tx: Vec>>, + // Future for an in-flight stats flush, if any, yielding ACK intents. + stats_write_fut: + Option)>>>, + // Task being executed by this actor. + task: Task, + // Future for an in-flight trigger dispatch, if any. + trigger_fut: Option>>, +} + +impl Actor { + pub fn new( + http_client: reqwest::Client, + legacy_checkpoint: Option, + publisher: crate::Publisher, + shard_tx: Vec>>, + task: Task, + ) -> Self { + Self { + http_client, + intents_write_fut: None, + legacy_checkpoint: legacy_checkpoint.map(|f| (f, consumer::Checkpoint::default())), + parked_publisher: Some(publisher), + pending_ack_intents: BTreeMap::new(), + shard_tx, + stats_write_fut: None, + task, + trigger_fut: None, + } + } + + #[tracing::instrument(level = "debug", err(Debug, level = "warn"), skip_all)] + pub async fn serve( + &mut self, + mut head: fsm::Head, + mut tail: fsm::Tail, + session: shuffle::SessionClient, + shard_rx: Vec>>, + ) -> anyhow::Result<()> { + tracing::info!(self.task.n_shards, "materialize actor started"); + assert_eq!(self.task.n_shards, shard_rx.len()); + assert_eq!(self.task.n_shards, self.shard_tx.len()); + + // Build a stream of receive futures for each shard. + let mut shard_rx: FuturesUnordered<_> = shard_rx + .into_iter() + .enumerate() + .map(next_shard_rx) + .collect(); + + // Build a stream of frontier checkpoints from the shuffle Session. + let frontier_rx = futures::stream::unfold(session, |mut session| async { + let result = session.next_checkpoint().await; + Some((result, session)) + }); + let mut frontier_rx = std::pin::pin!(frontier_rx); + + // Per-binding absolute measure, into which deltas are reduced. + let mut binding_bytes_behind = vec![0; self.task.binding_collection_names.len()]; + // When true, Head should close its current open transaction ASAP. + let mut close_requested = false; + // Iteration counter for the per-loop trace event. + let mut loop_count: u64 = 0; + // Monotonic Clock which is ticked on loop iterations, and updated on IO. + let mut now = now_clock(); + // When Some, a Frontier that's ready to extend a transaction. + let mut ready_frontier: Option = None; + // When Some, a message from a shard that's ready to consume. + let mut ready_shard_rx = None; + // When true, the topology should gracefully exit. + let mut stopping = false; + // Transactions completed in this task session, for preview harness limits. + let mut transactions_completed = 0u32; + + loop { + loop_count += 1; + let mut wake_after = crate::ACTOR_TICK_INTERVAL; + now.tick(); // Strictly increasing iteration values. + + tracing::trace!( + loop_count, + close_requested, + head = ?head, + intents_in_flight = self.intents_write_fut.is_some(), + pending_ack_intents = self.pending_ack_intents.len(), + ready_frontier = ready_frontier.is_some(), + ready_shard_rx = ready_shard_rx.is_some(), + stats_in_flight = self.stats_write_fut.is_some(), + stopping, + tail = ?tail, + trigger_in_flight = self.trigger_fut.is_some(), + "leader materialize Actor::serve iteration" + ); + + // Drive `tail` to idle. + let action: fsm::Action; + (action, tail) = tail.step( + self.intents_write_fut.is_none(), + now, + &mut ready_shard_rx, + stopping, + &self.task, + self.trigger_fut.is_some(), + ); + + match action { + fsm::Action::Idle => (), + fsm::Action::Sleep { .. } => unreachable!("Tail does not Sleep"), + fsm::Action::Rotate { .. } => unreachable!("Tail does not Rotate"), + action => { + self.dispatch(action)?; + continue; + } + } + + // Drive `head` to idle or stop. + let action: fsm::Action; + (action, head) = head.step( + &mut binding_bytes_behind, + &mut close_requested, + &mut self.legacy_checkpoint, + now, + &mut ready_frontier, + &mut ready_shard_rx, + self.stats_write_fut + .is_none() + .then_some(&mut self.pending_ack_intents), + stopping, + &mut tail, + &self.task, + ); + + match action { + fsm::Action::Idle => (), + fsm::Action::Sleep { wake_after: w } => wake_after = w, + fsm::Action::Rotate { pending } => { + assert!(matches!(tail, fsm::Tail::Done(_))); + on_transaction_completed( + self.task.max_transactions, + &mut transactions_completed, + &mut stopping, + ); + tail = fsm::Tail::Begin(fsm::TailBegin { pending }); + continue; + } + action => { + self.dispatch(action)?; + continue; + } + } + + if matches!(head, fsm::Head::Stop) { + break; + } + + // If `ready_shard_rx` was not consumed by either `head` or `tail`, + // then it was unexpected and is a protocol error. + if let Some((shard_index, msg)) = ready_shard_rx.take() { + anyhow::bail!( + "unexpected message {msg:?} from {} (index {shard_index})", + self.task.peers[shard_index], + ); + } + + tokio::select! { + biased; + + // Prioritize completions of leader IO first. + Some(result) = maybe_fut(&mut self.stats_write_fut) => { + let (publisher, intents) = result.map_err(crate::status_to_anyhow) + .context("writing ops stats document")?; + + self.parked_publisher = Some(publisher); + self.pending_ack_intents = intents; + self.stats_write_fut = None; + } + Some(result) = maybe_fut(&mut self.intents_write_fut) => { + let publisher = result.map_err(crate::status_to_anyhow) + .context("writing ACK intents")?; + + self.parked_publisher = Some(publisher); + self.intents_write_fut = None; + } + Some(result) = maybe_fut(&mut self.trigger_fut) => { + () = result?; + self.trigger_fut = None; + } + // Process shard messages next. + Some((shard_index, msg, rx)) = shard_rx.next() => { + if let Some(msg) = self.on_shard_rx( + &mut close_requested, + &mut stopping, + shard_index, + msg, + )? { + ready_shard_rx = Some((shard_index, msg)); + } + shard_rx.push(next_shard_rx((shard_index, rx))); + } + // Poll for a next frontier when otherwise idle. + Some(result) = frontier_rx.next(), if ready_frontier.is_none() => { + ready_frontier = Some(result?); + } + + // Lowest priority. + _ = tokio::time::sleep(wake_after) => {} + } + + now.update(now_clock()); // Resync after blocking IO. + } + + tracing::info!("materialize actor exiting"); + + for tx in &self.shard_tx { + let _ = tx.send(Ok(proto::Materialize { + stopped: Some(proto::Stopped {}), + ..Default::default() + })); + } + + Ok(()) + } + + /// Execute the outgoing-IO primitive for an Action. + #[tracing::instrument(level = "trace", fields(action = ?action), skip_all)] + fn dispatch(&mut self, action: fsm::Action) -> anyhow::Result<()> { + match action { + fsm::Action::Idle | fsm::Action::Sleep { .. } | fsm::Action::Rotate { .. } => { + unreachable!("never dispatched"); + } + + fsm::Action::Load { frontier } => { + tracing::debug!(journals = frontier.journals.len(), "broadcasting L:Load"); + self.broadcast(proto::Materialize { + load: Some(proto::materialize::Load { + frontier: Some(frontier.encode()), + }), + ..Default::default() + }); + } + + fsm::Action::Flush { connector_patches } => { + tracing::debug!( + patches_bytes = connector_patches.len(), + "broadcasting L:Flush" + ); + self.broadcast(proto::Materialize { + flush: Some(proto::materialize::Flush { + connector_patches_json: connector_patches, + }), + ..Default::default() + }); + } + + fsm::Action::Store => { + tracing::debug!("broadcasting L:Store"); + self.broadcast(proto::Materialize { + store: Some(proto::materialize::Store {}), + ..Default::default() + }); + } + + fsm::Action::StartCommit { + connector_checkpoint, + connector_patches, + } => { + tracing::debug!( + patches_bytes = connector_patches.len(), + "broadcasting L:StartCommit" + ); + + self.broadcast(proto::Materialize { + start_commit: Some(proto::materialize::StartCommit { + connector_checkpoint: Some(connector_checkpoint), + connector_patches_json: connector_patches, + }), + ..Default::default() + }); + } + + fsm::Action::Acknowledge { connector_patches } => { + tracing::debug!( + patches_bytes = connector_patches.len(), + "broadcasting L:Acknowledge" + ); + self.broadcast(proto::Materialize { + acknowledge: Some(proto::materialize::Acknowledge { + connector_patches_json: connector_patches, + }), + ..Default::default() + }); + } + + fsm::Action::Persist { persist } => { + tracing::debug!("dispatching L:Persist to shard zero"); + + let _ = self.shard_tx[0].send(Ok(proto::Materialize { + persist: Some(persist), + ..Default::default() + })); + } + + fsm::Action::WriteStats { stats } => { + let mut publisher = self + .parked_publisher + .take() + .expect("publisher owned at WriteOpsStats dispatch"); + + self.stats_write_fut = Some( + async move { + () = publisher.publish_stats(stats).await?; + + let intents = match publisher.commit_intents() { + Some(commit) => { + publisher::intents::build_transaction_intents(&[commit]) + } + None => BTreeMap::new(), + }; + + Ok((publisher, intents)) + } + .boxed(), + ); + } + + fsm::Action::WriteIntents { ack_intents } => { + let mut publisher = self + .parked_publisher + .take() + .expect("publisher owned at WriteIntents dispatch"); + + self.intents_write_fut = Some( + async move { + () = publisher.write_intents(ack_intents).await?; + Ok(publisher) + } + .boxed(), + ); + } + + fsm::Action::CallTrigger { trigger_params } => { + let Some(compiled) = self.task.triggers.clone() else { + tracing::info!( + trigger_params_bytes = trigger_params.len(), + "discarding recovered trigger parameters for materialization without triggers" + ); + return Ok(()); + }; + let variables: models::TriggerVariables = + serde_json::from_slice(&trigger_params) + .context("decoding trigger_variables JSON")?; + let client = self.http_client.clone(); + self.trigger_fut = Some( + async move { + super::triggers::fire_pending_triggers(&compiled, &variables, &client).await + } + .boxed(), + ); + } + } + + Ok(()) + } + + /// Receive a message from a shard. Returns the message for the + /// FSM to consume, or `None` if this was a control message (Stop) + /// the Actor handled itself. + fn on_shard_rx( + &self, + close_requested: &mut bool, + stopping: &mut bool, + shard_index: usize, + result: Option>, + ) -> anyhow::Result> { + let verify = crate::verify( + "Materialize", + "actor message", + &self.task.peers[shard_index], + ); + let msg = verify.not_eof(result)?; + + if matches!(msg.stop, Some(proto::Stop {})) { + *stopping = true; + return Ok(None); + } else if matches!(msg.close_now, Some(proto::CloseNow {})) { + *close_requested = true; + return Ok(None); + } + + let kind = if msg.loaded.is_some() { + "L:Loaded" + } else if msg.flushed.is_some() { + "L:Flushed" + } else if msg.stored.is_some() { + "L:Stored" + } else if msg.started_commit.is_some() { + "L:StartedCommit" + } else if msg.acknowledged.is_some() { + "L:Acknowledged" + } else if msg.persisted.is_some() { + "L:Persisted" + } else if msg.opened.is_some() { + "L:Opened" + } else if msg.recover.is_some() { + "L:Recover" + } else if msg.stopped.is_some() { + "L:Stopped" + } else { + "(other)" + }; + tracing::debug!(shard_index, kind, "received from shard"); + + Ok(Some(msg)) + } + + /// Synchronously fan out a single leader message to every shard. + fn broadcast(&self, msg: proto::Materialize) { + let (head, tail) = self.shard_tx.split_first().unwrap(); + + for tx in tail { + let _ = tx.send(Ok(msg.clone())); + } + let _ = head.send(Ok(msg)); // Avoid a clone (single-shard common case). + } +} + +fn on_transaction_completed( + max_transactions: u32, + transactions_completed: &mut u32, + stopping: &mut bool, +) { + if max_transactions == 0 || *stopping { + return; + } + + *transactions_completed = transactions_completed.saturating_add(1); + if *transactions_completed >= max_transactions { + tracing::info!( + transactions_completed, + max_transactions, + "materialize transaction limit reached; stopping gracefully", + ); + *stopping = true; + } +} + +fn now_clock() -> uuid::Clock { + let now = tokens::now(); + uuid::Clock::from_unix(now.timestamp() as u64, now.timestamp_subsec_nanos()) +} + +async fn maybe_fut(opt: &mut Option>) -> Option { + match opt.as_mut() { + Some(fut) => Some(fut.await), + None => std::future::pending().await, + } +} + +async fn next_shard_rx( + (shard_index, mut rx): (usize, BoxStream<'static, tonic::Result>), +) -> ( + usize, + Option>, + BoxStream<'static, tonic::Result>, +) { + let msg = rx.next().await; + (shard_index, msg, rx) +} + +#[cfg(test)] +mod tests { + use super::*; + + fn mk_actor( + n_shards: usize, + ) -> ( + Actor, + Vec>>, + ) { + let mut shard_tx = Vec::with_capacity(n_shards); + let mut rxs = Vec::with_capacity(n_shards); + for _ in 0..n_shards { + let (tx, rx) = mpsc::unbounded_channel(); + shard_tx.push(tx); + rxs.push(rx); + } + let task = Task { + binding_collection_names: vec!["test/collection".to_string()], + binding_journal_read_suffixes: vec!["pivot=00".to_string()], + combiner_usage_bytes: 0..u64::MAX, + connector_image: String::new(), + last_close_age: std::time::Duration::ZERO..std::time::Duration::MAX, + max_transactions: 0, + n_shards, + open_duration: std::time::Duration::ZERO..std::time::Duration::MAX, + peers: (0..n_shards).map(|i| format!("shard-{i}")).collect(), + read_bytes: 0..u64::MAX, + read_docs: 0..u64::MAX, + shard_ref: ops::ShardRef::default(), + triggers: None, + }; + let actor = Actor::new( + reqwest::Client::new(), + None, + crate::Publisher::new_preview(), + shard_tx, + task, + ); + (actor, rxs) + } + + #[test] + fn broadcast_fans_out_to_every_shard() { + let (actor, mut rxs) = mk_actor(3); + let msg = proto::Materialize { + store: Some(proto::materialize::Store {}), + ..Default::default() + }; + actor.broadcast(msg); + + for (i, rx) in rxs.iter_mut().enumerate() { + let got = rx + .try_recv() + .unwrap_or_else(|_| panic!("shard {i} should have received broadcast")) + .unwrap(); + assert!(got.store.is_some(), "shard {i}"); + assert!(rx.try_recv().is_err(), "shard {i} should only get one msg"); + } + } + + #[test] + fn broadcast_single_shard_works() { + // Single-shard path uses split_first head/tail; verify nothing odd at N=1. + let (actor, mut rxs) = mk_actor(1); + actor.broadcast(proto::Materialize { + store: Some(proto::materialize::Store {}), + ..Default::default() + }); + rxs[0].try_recv().unwrap().unwrap(); + } + + #[test] + fn on_shard_rx_dispatches_by_message_kind() { + // Outcome: Ok(None) consumed control flag, Ok(Some) data, Err message. + enum Want { + Consumed { stopping: bool, close: bool }, + Passthrough, + Err(&'static str), + } + use Want::*; + + let cases: &[(usize, _, Want)] = &[ + ( + 1, + Some(Ok(proto::Materialize { + stop: Some(proto::Stop {}), + ..Default::default() + })), + Consumed { + stopping: true, + close: false, + }, + ), + ( + 0, + Some(Ok(proto::Materialize { + close_now: Some(proto::CloseNow {}), + ..Default::default() + })), + Consumed { + stopping: false, + close: true, + }, + ), + ( + 0, + Some(Ok(proto::Materialize { + loaded: Some(proto::materialize::Loaded::default()), + ..Default::default() + })), + Passthrough, + ), + // EOF mentions the shard name to localize the failure. + (2, None, Err("unexpected EOF")), + (2, None, Err("shard-2")), + ( + 1, + Some(Result::<_, _>::Err(tonic::Status::aborted("peer aborted"))), + Err("peer aborted"), + ), + ]; + + for (shard_index, input, want) in cases { + let (actor, _rxs) = mk_actor(3); + let mut close_requested = false; + let mut stopping = false; + let result = actor.on_shard_rx( + &mut close_requested, + &mut stopping, + *shard_index, + input.clone(), + ); + match want { + Consumed { stopping: s, close } => { + assert!(result.as_ref().unwrap().is_none()); + assert_eq!(stopping, *s); + assert_eq!(close_requested, *close); + } + Passthrough => { + let msg = result.unwrap().expect("passthrough"); + assert!(msg.loaded.is_some()); + assert!(!stopping && !close_requested); + } + Err(needle) => { + let err = result.unwrap_err(); + let s = format!("{err:?}"); + assert!(s.contains(needle), "missing {needle:?} in {s}"); + } + } + } + } + + #[test] + fn transaction_limit_one_stops_after_one_completion() { + let mut completed = 0; + let mut stopping = false; + + on_transaction_completed(1, &mut completed, &mut stopping); + + assert_eq!(completed, 1); + assert!(stopping); + } + + #[test] + fn transaction_limit_multiple_stops_after_exact_target() { + let mut completed = 0; + let mut stopping = false; + + on_transaction_completed(3, &mut completed, &mut stopping); + assert_eq!(completed, 1); + assert!(!stopping); + + on_transaction_completed(3, &mut completed, &mut stopping); + assert_eq!(completed, 2); + assert!(!stopping); + + on_transaction_completed(3, &mut completed, &mut stopping); + assert_eq!(completed, 3); + assert!(stopping); + } + + #[test] + fn transaction_limit_zero_is_unlimited() { + let mut completed = 0; + let mut stopping = false; + + for _ in 0..3 { + on_transaction_completed(0, &mut completed, &mut stopping); + } + + assert_eq!(completed, 0); + assert!(!stopping); + } + + #[test] + fn external_stop_is_not_overridden_by_transaction_limit() { + let mut completed = 0; + let mut stopping = true; + + on_transaction_completed(1, &mut completed, &mut stopping); + + assert_eq!(completed, 0); + assert!(stopping); + } +} diff --git a/crates/runtime-next/src/leader/materialize/frontier_mapping.rs b/crates/runtime-next/src/leader/materialize/frontier_mapping.rs new file mode 100644 index 00000000000..75714216f39 --- /dev/null +++ b/crates/runtime-next/src/leader/materialize/frontier_mapping.rs @@ -0,0 +1,410 @@ +//! Mapping between legacy `consumer.Checkpoint` and `shuffle::Frontier`. +use proto_gazette::consumer; +use proto_gazette::uuid; +use std::collections::BTreeMap; + +/// Errors produced by `checkpoint_to_frontier`. +#[derive(Debug, thiserror::Error)] +pub enum Error { + #[error("Checkpoint source key {source_key:?} has no ';' suffix separator")] + MissingSourceKeySuffix { source_key: String }, + #[error("Checkpoint producer id for source {source_key:?} is {got} bytes, want 6")] + InvalidProducerIdLength { source_key: String, got: usize }, + #[error("Checkpoint producer begin for source {source_key:?} is {begin} (must be >= -1)")] + InvalidProducerBegin { source_key: String, begin: i64 }, + #[error("Frontier validation failed")] + FrontierValidation(#[from] shuffle::frontier::Error), +} + +/// Convert `Checkpoint.sources` to `shuffle::Frontier` journal entries. +/// +/// `binding_suffixes` indexes binding index → `journal_read_suffix`. +/// Source keys without a `;` separator are an error. The returned +/// `Frontier` has empty `flushed_lsn` — the legacy checkpoint has no +/// analogous per-shard log barrier. +pub fn checkpoint_to_frontier( + sources: &BTreeMap, + journal_read_suffix_index: &[(&str, usize)], +) -> Result { + let mut journals: Vec = Vec::with_capacity(sources.len()); + + for (source_key, source) in sources { + let Some((journal, suffix)) = source_key.split_once(';') else { + return Err(Error::MissingSourceKeySuffix { + source_key: source_key.clone(), + }); + }; + + let Ok(index) = + journal_read_suffix_index.binary_search_by(|(cursor, _)| (*cursor).cmp(suffix)) + else { + continue; + }; + let binding = journal_read_suffix_index[index].1 as u16; + + let mut producers: Vec = + Vec::with_capacity(source.producers.len()); + for entry in &source.producers { + let id: [u8; 6] = + entry + .id + .as_ref() + .try_into() + .map_err(|_| Error::InvalidProducerIdLength { + source_key: source_key.clone(), + got: entry.id.len(), + })?; + let state = entry.state.unwrap_or_default(); + + let offset = if state.begin >= 0 { + state.begin + } else if state.begin == -1 { + -source.read_through + } else { + return Err(Error::InvalidProducerBegin { + source_key: source_key.clone(), + begin: state.begin, + }); + }; + + producers.push(shuffle::ProducerFrontier { + producer: uuid::Producer(id), + last_commit: uuid::Clock::from_u64(state.last_ack), + hinted_commit: uuid::Clock::zero(), + offset, + }); + } + producers.sort_by_key(|p| p.producer); + + journals.push(shuffle::JournalFrontier { + journal: journal.into(), + binding, + producers, + bytes_read_delta: 0, + bytes_behind_delta: 0, + }); + } + + journals.sort_by(|a, b| { + a.journal + .as_ref() + .cmp(b.journal.as_ref()) + .then(a.binding.cmp(&b.binding)) + }); + + Ok(shuffle::Frontier::new(journals, vec![])?) +} + +/// Merge `Frontier` journal entries into `Checkpoint.sources`, keyed by +/// `"{journal};{suffix}"`, while minimizing re-allocation. Producer entries of +/// both `frontier` and `checkpoint` are expected to already be sorted on +/// producer ID. Any journals or producers in `checkpoint.sources` which are not +/// in `frontier` are left alone. +pub fn merge_frontier_into_checkpoint>( + frontier: &shuffle::Frontier, + checkpoint: &mut consumer::Checkpoint, + binding_journal_read_suffixes: &[S], +) { + let mut scratch = Vec::new(); + + for jf in &frontier.journals { + let Some(suffix) = binding_journal_read_suffixes.get(jf.binding as usize) else { + continue; // Reachable only if shuffle service provides bad binding indices. + }; + let source_key = format!("{};{}", jf.journal, suffix.as_ref()); + + let mut source = checkpoint.sources.remove(&source_key).unwrap_or_default(); + scratch.append(&mut source.producers); // Move via copy; source.producers is now empty. + + let mut existing = scratch.drain(..).peekable(); + let mut frontier = jf.producers.iter().peekable(); + + fn pf_to_pe( + pf: &shuffle::ProducerFrontier, + read_through: &mut i64, + ) -> consumer::checkpoint::source::ProducerEntry { + consumer::checkpoint::source::ProducerEntry { + id: bytes::Bytes::copy_from_slice(pf.producer.as_bytes()), + state: Some(pf_to_ps(pf, read_through)), + } + } + + fn update_pe( + entry: &mut consumer::checkpoint::source::ProducerEntry, + pf: &shuffle::ProducerFrontier, + read_through: &mut i64, + ) { + entry.state = Some(pf_to_ps(pf, read_through)); + } + + fn pf_to_ps( + pf: &shuffle::ProducerFrontier, + read_through: &mut i64, + ) -> consumer::checkpoint::ProducerState { + let offset = pf.offset.checked_abs().unwrap_or_default(); + *read_through = (*read_through).max(offset); + + consumer::checkpoint::ProducerState { + last_ack: pf.last_commit.as_u64(), + begin: if pf.offset >= 0 { pf.offset } else { -1 }, + } + } + + loop { + match (existing.peek(), frontier.peek()) { + (Some(a), Some(b)) => match a.id.as_ref().cmp(b.producer.as_bytes()) { + std::cmp::Ordering::Less => source.producers.push(existing.next().unwrap()), + std::cmp::Ordering::Greater => { + source + .producers + .push(pf_to_pe(frontier.next().unwrap(), &mut source.read_through)); + } + std::cmp::Ordering::Equal => { + let mut entry = existing.next().unwrap(); + let pf = frontier.next().unwrap(); + update_pe(&mut entry, pf, &mut source.read_through); + source.producers.push(entry); + } + }, + (Some(_), None) => { + source.producers.extend(existing); + break; + } + (None, Some(_)) => { + for pf in frontier { + let entry = pf_to_pe(pf, &mut source.read_through); + source.producers.push(entry); + } + break; + } + (None, None) => break, + } + } + + checkpoint.sources.insert(source_key, source); + } +} + +/// Project a verbatim `FH:` Frontier into hinted form: each producer's +/// `last_commit` is promoted to `hinted_commit`, and `last_commit` / +/// `offset` are zeroed. The result is reduced with the recovered +/// committed Frontier when composing a session's resume Frontier. +pub fn project_hinted(mut frontier: shuffle::Frontier) -> shuffle::Frontier { + for jf in &mut frontier.journals { + for pf in &mut jf.producers { + pf.hinted_commit = pf.last_commit; + pf.last_commit = uuid::Clock::zero(); + pf.offset = 0; + } + } + frontier +} + +/// Synthetic producer ID under which `encode_committed_close` records the +/// close Clock. Coexisting with real producer IDs in the same source is +/// safe in practice: real IDs are randomly drawn 6-byte values, and a +/// collision is vanishingly unlikely. This is a hack — but an acceptable one. +const COMMITTED_CLOSE_PRODUCER_ID: &[u8; 6] = b"\x01\x00\x00\x00\x00\x00"; + +/// Encode the close Clock of a committing transaction for inclusion in +/// consumer::Checkpoint::sources. +/// +/// This is a re-purpose of the Checkpoint structure (pervasive among Estuary +/// connectors and the Gazette ecosystem), using it to convey a single close +/// Clock which can later be inspected to determine a commit outcome. +pub fn encode_committed_close(clock: uuid::Clock) -> (String, consumer::checkpoint::Source) { + let key = str::from_utf8(crate::shard::recovery::KEY_COMMITTED_CLOSE).unwrap(); + + ( + key.to_string(), + consumer::checkpoint::Source { + read_through: 0, + producers: vec![consumer::checkpoint::source::ProducerEntry { + id: bytes::Bytes::from_static(COMMITTED_CLOSE_PRODUCER_ID), + state: Some(consumer::checkpoint::ProducerState { + last_ack: clock.as_u64(), + begin: 0, + }), + }], + }, + ) +} + +/// Extract a close Clock of a committed transaction from a consumer::Checkpoint, if present. +/// This is the inverse of `encode_committed_close`. +pub fn extract_committed_close(checkpoint: &consumer::Checkpoint) -> Option { + let key = str::from_utf8(crate::shard::recovery::KEY_COMMITTED_CLOSE).unwrap(); + + let Some(source) = checkpoint.sources.get(key) else { + return None; + }; + let Some(producer) = source.producers.get(0) else { + return None; + }; + let Some(state) = &producer.state else { + return None; + }; + Some(uuid::Clock::from_u64(state.last_ack)) +} + +#[cfg(test)] +mod tests { + use super::*; + use bytes::Bytes; + + fn producer(tag: u8) -> uuid::Producer { + uuid::Producer::from_bytes([0x01, tag, 0, 0, 0, 0]) + } + + fn producer_entry( + tag: u8, + last_ack: u64, + begin: i64, + ) -> consumer::checkpoint::source::ProducerEntry { + consumer::checkpoint::source::ProducerEntry { + id: Bytes::copy_from_slice(producer(tag).as_bytes()), + state: Some(consumer::checkpoint::ProducerState { last_ack, begin }), + } + } + + fn source( + read_through: i64, + producers: Vec, + ) -> consumer::checkpoint::Source { + consumer::checkpoint::Source { + read_through, + producers, + } + } + + fn producer_frontier(tag: u8, last_commit: u64, offset: i64) -> shuffle::ProducerFrontier { + shuffle::ProducerFrontier { + producer: producer(tag), + last_commit: uuid::Clock::from_u64(last_commit), + hinted_commit: uuid::Clock::zero(), + offset, + } + } + + fn journal_frontier( + journal: &str, + binding: u16, + producers: Vec, + ) -> shuffle::JournalFrontier { + shuffle::JournalFrontier { + journal: journal.into(), + binding, + producers, + bytes_read_delta: 0, + bytes_behind_delta: 0, + } + } + + #[test] + fn merge_frontier_preserves_sources_and_sorted_merges_producers() { + let mut checkpoint = consumer::Checkpoint::default(); + let mut source_a_producers = Vec::with_capacity(4); + source_a_producers.push(producer_entry(0x10, 1, 11)); + source_a_producers.push(producer_entry(0x30, 3, -1)); + let source_a_vec_ptr = source_a_producers.as_ptr(); + let producer_30_id_ptr = source_a_producers[1].id.as_ptr(); + + checkpoint.sources.insert( + "journal/a;suffix-a".to_string(), + source(90, source_a_producers), + ); + checkpoint.sources.insert( + "unrelated;suffix-z".to_string(), + source(7, vec![producer_entry(0x50, 5, -1)]), + ); + checkpoint + .ack_intents + .insert("ack/journal".to_string(), Bytes::from_static(b"ack")); + + let frontier = shuffle::Frontier::new( + vec![ + journal_frontier( + "journal/a", + 0, + vec![ + producer_frontier(0x20, 20, 70), + producer_frontier(0x30, 30, -150), + ], + ), + journal_frontier("journal/b", 1, vec![producer_frontier(0x40, 40, -10)]), + ], + vec![], + ) + .unwrap(); + + merge_frontier_into_checkpoint( + &frontier, + &mut checkpoint, + &["suffix-a".to_string(), "suffix-b".to_string()], + ); + + assert_eq!( + checkpoint.ack_intents.get("ack/journal").unwrap().as_ref(), + b"ack" + ); + assert!(checkpoint.sources.contains_key("unrelated;suffix-z")); + + let source_a = checkpoint.sources.get("journal/a;suffix-a").unwrap(); + assert_eq!(source_a.producers.as_ptr(), source_a_vec_ptr); + assert_eq!(source_a.read_through, 150); + let ids: Vec<_> = source_a + .producers + .iter() + .map(|p| p.id.as_ref()[1]) + .collect(); + assert_eq!(ids, vec![0x10, 0x20, 0x30]); + assert_eq!(source_a.producers[0].state.unwrap().last_ack, 1); + assert_eq!(source_a.producers[1].state.unwrap().last_ack, 20); + assert_eq!(source_a.producers[1].state.unwrap().begin, 70); + assert_eq!(source_a.producers[2].id.as_ptr(), producer_30_id_ptr); + assert_eq!(source_a.producers[2].state.unwrap().last_ack, 30); + assert_eq!(source_a.producers[2].state.unwrap().begin, -1); + + let source_b = checkpoint.sources.get("journal/b;suffix-b").unwrap(); + assert_eq!(source_b.read_through, 10); + assert_eq!(source_b.producers[0].state.unwrap().last_ack, 40); + assert_eq!(source_b.producers[0].state.unwrap().begin, -1); + } + + #[test] + fn merge_frontier_round_trips_through_checkpoint_mapping() { + let frontier = shuffle::Frontier::new( + vec![ + journal_frontier("journal/a", 0, vec![producer_frontier(0x10, 10, 25)]), + journal_frontier("journal/b", 1, vec![producer_frontier(0x20, 20, -50)]), + ], + vec![], + ) + .unwrap(); + let mut checkpoint = consumer::Checkpoint::default(); + + merge_frontier_into_checkpoint( + &frontier, + &mut checkpoint, + &["suffix-a".to_string(), "suffix-b".to_string()], + ); + + let recovered = + checkpoint_to_frontier(&checkpoint.sources, &[("suffix-a", 0), ("suffix-b", 1)]) + .unwrap(); + + assert_eq!(recovered.journals.len(), frontier.journals.len()); + for (got, want) in recovered.journals.iter().zip(frontier.journals.iter()) { + assert_eq!(got.journal, want.journal); + assert_eq!(got.binding, want.binding); + assert_eq!(got.producers.len(), want.producers.len()); + + for (got, want) in got.producers.iter().zip(want.producers.iter()) { + assert_eq!(got.producer, want.producer); + assert_eq!(got.last_commit, want.last_commit); + assert_eq!(got.hinted_commit, want.hinted_commit); + assert_eq!(got.offset, want.offset); + } + } + } +} diff --git a/crates/runtime-next/src/leader/materialize/fsm.rs b/crates/runtime-next/src/leader/materialize/fsm.rs new file mode 100644 index 00000000000..41391a96a82 --- /dev/null +++ b/crates/runtime-next/src/leader/materialize/fsm.rs @@ -0,0 +1,2165 @@ +//! HeadFSM and TailFSM: the materialize Leader's pipelined transaction FSMs. +//! +//! HeadFSM drives the currently-open transaction toward commit: +//! Idle → Extend → Flush → (Persist) → Store → Stored → WriteStats +//! → StartCommit → Persist → {Rotate | Stop} +//! +//! TailFSM drives post-commit work for the prior transaction: +//! Begin → Acknowledge → (Persist) → WriteIntents → (Trigger) +//! → (Persist) → Done +//! +//! Head and Tail are pipelined. Tail may spend a long time in the connector's +//! post-commit phases (Acknowledge, Trigger), so Head may keep preparing a next +//! transaction while Tail finishes. When stopping, Head exits only once it is +//! idle with Tail already done, or after its next durable commit. Any post- +//! commit work for that last transaction is recovered and resumed by the next +//! leader session. +use super::{Task, frontier_mapping}; +use crate::proto; +use gazette::consumer; +use proto_gazette::uuid; +use std::collections::{BTreeMap, HashMap}; + +/// Per-transaction aggregated state threaded through the HeadFSM. +#[derive(Debug, Default)] +pub struct Extents { + // Clock at which the transaction started (first applied ready frontier). + open: uuid::Clock, + // Clock at which the transaction began to close. + close: uuid::Clock, + // Frontier delta processed by this transaction. + frontier: shuffle::Frontier, + // Sparse per-binding map of bindings having changed extents in this transaction. + bindings: HashMap, +} + +#[derive(Debug, Default)] +pub struct BindingExtents { + max_key_delta: bytes::Bytes, + // Maximum source clock (flow_published_at) read by this binding. + max_source_clock: uuid::Clock, + // Minimum source clock (flow_published_at) read by this binding. + min_source_clock: uuid::Clock, + // Measures of documents read from source journals. + sourced: ops::proto::stats::DocsAndBytes, + // Measures of loaded documents from the materialized endpoint. + loaded: ops::proto::stats::DocsAndBytes, + // Measures of stored documents into the materialized endpoint. + stored: ops::proto::stats::DocsAndBytes, +} + +/// Delta state gathered from shard responses, and staged for emission +/// at later protocol points. +#[derive(Debug, Default)] +pub struct PendingDeltas { + /// ACK Intents to write post-Acknowledge, keyed by journal. + pub ack_intents: BTreeMap, + /// Per-binding-index max-loaded-key deltas, for the next Persist. + pub max_key_deltas: BTreeMap, + /// Queued connector state patches for the next Persist. + pub persist_patches: Vec, + /// Queued connector state patches for the next shards broadcast. + pub shard_patches: Vec, + /// Parameters for the post-Acknowledge trigger. + pub trigger_params: bytes::Bytes, +} + +#[derive(Debug)] +pub enum Head { + Idle(HeadIdle), + Extend(HeadExtend), + Flush(HeadFlush), + Persist(HeadPersist), + Store(HeadStore), + WriteStats(HeadWriteStats), + StartCommit(HeadStartCommit), + Stop, +} + +#[derive(Debug)] +pub enum Tail { + Begin(TailBegin), + Acknowledge(TailAcknowledge), + WriteIntents(TailWriteIntents), + Trigger(TailTrigger), + Persist(TailPersist), + Done(TailDone), +} + +/// `Action` is the next outgoing IO, or an actor-loop control edge. +/// Every non-terminator maps to exactly one IO primitive in the Actor's `dispatch()`. +#[derive(Debug)] +pub enum Action { + /// Park until new IO arrives. + Idle, + /// Park with a precise deadline. + Sleep { + wake_after: std::time::Duration, + }, + + /// Broadcast a `L:Load` Frontier. + Load { + frontier: shuffle::Frontier, + }, + /// Broadcast `L:Flush`. + Flush { + // Prior transaction's C:Acknowledged patches. + connector_patches: bytes::Bytes, + }, + /// Broadcast `L:Store`. + Store, + /// Broadcast `L:StartCommit` with this txn's C:Flushed patches. + StartCommit { + connector_patches: bytes::Bytes, + connector_checkpoint: consumer::Checkpoint, + }, + /// Publish a stats document as CONTINUE_TXN to the ops stats journal. + // NOTE: when mapping this pattern into derivations, pass gathered ACK + // intents from shards to the Actor from this Action variant, to pick up + // later from `stats_write_idle`. + WriteStats { + stats: ops::proto::Stats, + }, + /// Persist one `proto::Persist` WriteBatch to shard zero. + Persist { + persist: proto::Persist, + }, + /// Write ACK intents to their journals. + WriteIntents { + ack_intents: BTreeMap, + }, + /// Broadcast `L:Acknowledge` with this txn's aggregated StartedCommit patches. + Acknowledge { + // This committed transaction's C:StartedCommit patches. + connector_patches: bytes::Bytes, + }, + /// Start calling the trigger. + /// Actor sets `trigger_done = false` upon dispatch. + CallTrigger { + trigger_params: bytes::Bytes, + }, + + Rotate { + pending: PendingDeltas, + }, +} + +impl Head { + /// Dispatch to the current sub-state's `step()`. + pub fn step( + self, + binding_bytes_behind: &mut [i64], + close_requested: &mut bool, + legacy_checkpoint: &mut Option<(shuffle::Frontier, consumer::Checkpoint)>, + now: uuid::Clock, + ready_frontier: &mut Option, + shard_rx: &mut Option<(usize, proto::Materialize)>, + stats_write_idle: Option<&mut BTreeMap>, + stopping: bool, + tail: &mut Tail, + task: &Task, + ) -> (Action, Head) { + match self { + Head::Idle(s) => s.step(now, close_requested, ready_frontier, stopping, tail, task), + Head::Extend(s) => s.step( + now, + close_requested, + ready_frontier, + shard_rx, + stopping, + tail, + task, + ), + Head::Flush(s) => s.step(now, shard_rx, task), + Head::Persist(s) => s.step(shard_rx), + Head::Store(s) => s.step(binding_bytes_behind, shard_rx, task), + Head::WriteStats(s) => s.step(legacy_checkpoint, stats_write_idle, task), + Head::StartCommit(s) => s.step(legacy_checkpoint, now, shard_rx, stopping), + Head::Stop => panic!("HeadFSM::Stop observed at step boundary"), + } + } +} + +impl Tail { + /// Dispatch to the current sub-state's `step()`. + pub fn step( + self, + intents_write_idle: bool, + now: uuid::Clock, + shard_rx: &mut Option<(usize, proto::Materialize)>, + stopping: bool, + task: &Task, + trigger_call_running: bool, + ) -> (Action, Tail) { + match self { + Tail::Begin(s) => s.step(stopping, task), + Tail::WriteIntents(s) => s.step(intents_write_idle), + Tail::Acknowledge(s) => s.step(now, shard_rx), + Tail::Trigger(s) => s.step(now, trigger_call_running), + Tail::Persist(s) => s.step(shard_rx), + Tail::Done(_) => (Action::Idle, self), + } + } +} + +/// HeadIdle awaits a first ready Frontier that begins a transaction. +#[derive(Debug, Default)] +pub struct HeadIdle { + /// Do we expect the next transaction to replay recovered transaction extents? + pub idempotent_replay: bool, + /// Close Clock of the last transaction, which may be recovered from a + /// prior session, or zero. + pub last_close: uuid::Clock, +} + +impl HeadIdle { + pub fn step( + self, + now: uuid::Clock, + close_requested: &mut bool, + ready_frontier: &mut Option, + stopping: bool, + tail: &Tail, + task: &Task, + ) -> (Action, Head) { + // If Tail is Done and Head is Idle, stopping can complete without + // starting another transaction. Otherwise Head may still pipeline + // a next transaction while Tail finishes post-commit work. + if stopping && matches!(tail, Tail::Done(_)) { + return (Action::Idle, Head::Stop); + } + + // A close requested during the prior transaction's tail must not + // immediately close the next one we're about to open. + *close_requested = false; + + let Some(frontier) = ready_frontier.take() else { + return (Action::Idle, Head::Idle(self)); + }; + + // A frontier is ready, and we begin the transaction. + let Self { + idempotent_replay, + last_close, + } = self; + + let unresolved_hints = frontier.unresolved_hints != 0; + let action = Action::Load { + frontier: frontier.clone(), + }; + let extents = Extents { + open: now, + frontier, + ..Default::default() + }; + let state = HeadExtend { + extents, + combiner_usage_bytes: vec![0; task.n_shards], + idempotent_replay, + last_close, + shard_loaded: vec![false; task.n_shards], + unresolved_hints, + }; + (action, Head::Extend(state)) + } +} + +/// HeadExtend drives ready frontiers into Load/Loaded cycles that +/// extend transaction Extents, until we begin to close. +#[derive(Debug)] +pub struct HeadExtend { + pub extents: Extents, + + /// Running disk usage usage of per-shard combiners. + pub combiner_usage_bytes: Vec, + /// Are we replaying recovered transaction extents? + /// When true, we MUST stop extending as soon as no unresolved hints remain. + pub idempotent_replay: bool, + /// Close Clock of the prior transaction (which may be from a prior session), or zero. + pub last_close: uuid::Clock, + /// Per-shard tracking of Loaded response receipt. + pub shard_loaded: Vec, + /// Did the last-extended Frontier have unresolved causal hints? + /// When true, we MUST extend rather than close. + pub unresolved_hints: bool, +} + +impl HeadExtend { + pub fn step( + mut self, + now: uuid::Clock, + close_requested: &mut bool, + ready_frontier: &mut Option, + shard_rx: &mut Option<(usize, proto::Materialize)>, + stopping: bool, + tail: &mut Tail, + task: &Task, + ) -> (Action, Head) { + // Did we receive an expected Loaded response? + if let Some(( + shard_index, + proto::Materialize { + loaded: Some(loaded), + .. + }, + )) = shard_rx + && self.shard_loaded.get(*shard_index) == Some(&false) + { + let proto::materialize::Loaded { + bindings, + combiner_usage_bytes, + } = std::mem::take(loaded); + + for crate::proto::materialize::loaded::Binding { + index, + max_key_delta, + max_source_clock, + min_source_clock, + sourced_bytes_total, + sourced_docs_total, + } in bindings + { + let min_source_clock = uuid::Clock::from_u64(min_source_clock); + let max_source_clock = uuid::Clock::from_u64(max_source_clock); + let extent = self.extents.bindings.entry(index).or_default(); + + if !max_key_delta.is_empty() { + extent.max_key_delta = max_key_delta; + } + if extent.sourced.docs_total == 0 { + extent.max_source_clock = max_source_clock; + extent.min_source_clock = min_source_clock; + } else { + extent.max_source_clock = extent.max_source_clock.max(max_source_clock); + extent.min_source_clock = extent.min_source_clock.min(min_source_clock); + } + extent.sourced.bytes_total += sourced_bytes_total; + extent.sourced.docs_total += sourced_docs_total; + } + self.combiner_usage_bytes[*shard_index] = combiner_usage_bytes; + + // Mark received and consume `shard_rx`. + self.shard_loaded[*shard_index] = true; + shard_rx.take(); + + if self.shard_loaded.iter().all(|b| *b) { + self.shard_loaded.clear(); // All received. + } + } + + if !self.shard_loaded.is_empty() { + return (Action::Idle, Head::Extend(self)); + } + // We've received all expected Loaded responses. + + // Measures used to evaluate extend and close policy. + let open_age = uuid::Clock::delta(now, self.extents.open); + let last_age = uuid::Clock::delta(now, self.last_close); + let max_combiner = *self.combiner_usage_bytes.iter().max().unwrap(); + let (read_docs, read_bytes) = self + .extents + .bindings + .values() + .map(|extents| (extents.sourced.docs_total, extents.sourced.bytes_total)) + .fold((0, 0), |(a1, a2), (b1, b2)| (a1 + b1, a2 + b2)); + + let CloseDecision { + may_extend, + may_close, + } = decide_close_policy( + CloseInputs { + close_requested: *close_requested, + idempotent_replay: self.idempotent_replay, + last_age, + max_combiner, + open_age, + read_bytes, + read_docs, + stopping, + tail_done: matches!(tail, Tail::Done(_)), + unresolved_hints: self.unresolved_hints, + }, + task, + ); + + // Should we extend with a ready checkpoint? + if may_extend && let Some(frontier) = ready_frontier.take() { + self.unresolved_hints = frontier.unresolved_hints != 0; + self.extents.frontier = self.extents.frontier.reduce(frontier.clone()); + self.shard_loaded.resize(task.n_shards, false); + return (Action::Load { frontier }, Head::Extend(self)); + } + + // Should we begin to close the transaction? + if may_close { + *close_requested = false; + let Self { mut extents, .. } = self; + + extents.close = now; + + // Take C:Acknowledged patches of the prior transaction. + let connector_patches = match tail { + Tail::Done(done) => std::mem::take(&mut done.shard_patches), + _ => unreachable!("may_close requires TailFSM::Done"), + }; + + let max_keys = extents + .bindings + .iter() + .filter_map(|(binding_index, extent)| { + if extent.max_key_delta.is_empty() { + None + } else { + Some((*binding_index, extent.max_key_delta.clone())) + } + }) + .collect(); + + let pending = PendingDeltas { + max_key_deltas: max_keys, + ..Default::default() + }; + + return ( + Action::Flush { connector_patches }, + Head::Flush(HeadFlush { + extents, + pending, + shard_flushed: vec![false; task.n_shards], + }), + ); + } + + // Compute next sleep deadline. + let wake_after = [ + task.open_duration.start.checked_sub(open_age), + task.open_duration.end.checked_sub(open_age), + task.last_close_age.start.checked_sub(last_age), + task.last_close_age.end.checked_sub(last_age), + ] + .into_iter() + .filter_map(|s| s) + .min(); + + if let Some(wake_after) = wake_after { + (Action::Sleep { wake_after }, Head::Extend(self)) + } else { + (Action::Idle, Head::Extend(self)) + } + } +} + +/// HeadFlush awaits Flushed responses from all shards. +#[derive(Debug)] +pub struct HeadFlush { + pub extents: Extents, + pub pending: PendingDeltas, + + /// Per-shard tracking of Flushed response receipt. + pub shard_flushed: Vec, +} + +impl HeadFlush { + pub fn step( + mut self, + now: uuid::Clock, + shard_rx: &mut Option<(usize, proto::Materialize)>, + task: &Task, + ) -> (Action, Head) { + // Did we receive an expected Flushed response? + if let Some(( + shard_index, + proto::Materialize { + flushed: Some(flushed), + .. + }, + )) = shard_rx + && self.shard_flushed.get(*shard_index) == Some(&false) + { + let proto::materialize::Flushed { + bindings, + connector_patches_json, + } = std::mem::take(flushed); + + for crate::proto::materialize::flushed::Binding { + index, + loaded_bytes_total, + loaded_docs_total, + } in bindings + { + let extent = self.extents.bindings.entry(index).or_default(); + extent.loaded.bytes_total += loaded_bytes_total; + extent.loaded.docs_total += loaded_docs_total; + } + extend_patches(&mut self.pending, &connector_patches_json); + + // Mark received and consume `shard_rx`. + self.shard_flushed[*shard_index] = true; + shard_rx.take(); + + if self.shard_flushed.iter().all(|b| *b) { + self.shard_flushed.clear(); // All received. + } + } + + if !self.shard_flushed.is_empty() { + return (Action::Idle, Head::Flush(self)); + } + // We've received all expected Flushed responses. + + let Self { + extents, + mut pending, + .. + } = self; + + // Persist extents for idempotent transaction replay. + let persist = proto::Persist { + nonce: now.as_u64(), + connector_patches_json: take_patches(&mut pending.persist_patches), + delete_hinted_frontier: true, + hinted_close_clock: extents.close.as_u64(), + hinted_frontier: Some(shuffle::JournalFrontier::encode(&extents.frontier.journals)), + max_keys: std::mem::take(&mut pending.max_key_deltas), + ..Default::default() + }; + + // Chain Store after the Persisted response. + let store_state = HeadStore { + extents, + pending, + shard_stored: vec![false; task.n_shards], + }; + let persist_state = HeadPersist { + nonce: persist.nonce, + next_action: Action::Store, + next_state: Box::new(Head::Store(store_state)), + }; + + (Action::Persist { persist }, Head::Persist(persist_state)) + } +} + +/// HeadPersist awaits a Persisted response from shard zero, +/// and chains its contained action and state. +#[derive(Debug)] +pub struct HeadPersist { + pub nonce: u64, + pub next_action: Action, + pub next_state: Box, +} + +impl HeadPersist { + pub fn step(self, shard_rx: &mut Option<(usize, proto::Materialize)>) -> (Action, Head) { + if let Some(( + 0, + proto::Materialize { + persisted: Some(proto::Persisted { nonce }), + .. + }, + )) = shard_rx + && *nonce == self.nonce + { + shard_rx.take(); + + let Self { + nonce: _, + next_action, + next_state, + } = self; + + return (next_action, *next_state); + } + + (Action::Idle, Head::Persist(self)) + } +} + +/// HeadStore awaits Stored responses from all shards. +#[derive(Debug)] +pub struct HeadStore { + pub extents: Extents, + pub pending: PendingDeltas, + + /// Per-shard tracking of Stored response receipt. + pub shard_stored: Vec, +} + +impl HeadStore { + pub fn step( + mut self, + binding_bytes_behind: &mut [i64], + shard_rx: &mut Option<(usize, proto::Materialize)>, + task: &Task, + ) -> (Action, Head) { + // Did we receive an expected Stored response? + if let Some(( + shard_index, + proto::Materialize { + stored: Some(stored), + .. + }, + )) = shard_rx + && self.shard_stored.get(*shard_index) == Some(&false) + { + let proto::materialize::Stored { bindings } = std::mem::take(stored); + + for crate::proto::materialize::stored::Binding { + index, + stored_bytes_total, + stored_docs_total, + } in bindings + { + let extent = self.extents.bindings.entry(index).or_default(); + extent.stored.bytes_total += stored_bytes_total; + extent.stored.docs_total += stored_docs_total; + } + + // Mark received and consume `shard_rx`. + self.shard_stored[*shard_index] = true; + shard_rx.take(); + + if self.shard_stored.iter().all(|b| *b) { + self.shard_stored.clear(); // All received. + } + } + + if !self.shard_stored.is_empty() { + return (Action::Idle, Head::Store(self)); + } + // We've received all expected Stored responses. + + let Self { + extents, + mut pending, + .. + } = self; + + // Fold deltas from the extents Frontier into per-binding "bytes behind" gauges. + for jf in &extents.frontier.journals { + let Some(entry) = binding_bytes_behind.get_mut(jf.binding as usize) else { + continue; // Reachable if shuffle service reports invalid binding indices. + }; + *entry += jf.bytes_behind_delta; + } + + // Compose the trigger payload now that we have a complete txn-wide view. + if task.triggers.is_some() { + let collection_names: Vec = extents + .bindings + .keys() + .filter_map(|idx| task.binding_collection_names.get(*idx as usize).cloned()) + .collect(); + + let mut it = extents + .bindings + .values() + .map(|extents| (extents.min_source_clock, extents.max_source_clock)); + let init = it.next().unwrap_or_default(); + let (min, max) = it.fold(init, |(min, max), (a, b)| (min.min(a), max.max(b))); + + pending.trigger_params = serde_json::to_vec(&models::TriggerVariables { + collection_names, + connector_image: task.connector_image.clone(), + materialization_name: task.shard_ref.name.clone(), + flow_published_at_min: tokens::DateTime::from(min.to_time()).to_rfc3339(), + flow_published_at_max: tokens::DateTime::from(max.to_time()).to_rfc3339(), + run_id: tokens::DateTime::from(extents.open.to_time()).to_rfc3339(), + }) + .unwrap() + .into(); + } + + let action = Action::WriteStats { + stats: build_stats_doc(task, &extents, binding_bytes_behind), + }; + let state = HeadWriteStats { extents, pending }; + + (action, Head::WriteStats(state)) + } +} + +/// HeadWriteStats awaits the completion of a stats document append and flush. +#[derive(Debug)] +pub struct HeadWriteStats { + pub extents: Extents, + pub pending: PendingDeltas, +} + +impl HeadWriteStats { + pub fn step( + self, + legacy_checkpoint: &mut Option<(shuffle::Frontier, consumer::Checkpoint)>, + stats_write_idle: Option<&mut BTreeMap>, + task: &Task, + ) -> (Action, Head) { + let ack_intents = match stats_write_idle { + Some(ack_intents) => std::mem::take(ack_intents), + None => return (Action::Idle, Head::WriteStats(self)), + }; + // We've finished publishing to ops stats. + + let Self { + extents, + mut pending, + } = self; + + // We use the existing consumer.Checkpoint `sources` structure to + // piggyback the close Clock of this transaction under a special key. + // This is compatible with deployed connectors which may parse and + // re-serialize Checkpoints en-route to being stored in the endpoint, + // whereas a new field would require more roll-out coordination. + let (committed_close_key, committed_close_source) = + frontier_mapping::encode_committed_close(extents.close); + + // Build the consumer checkpoint which will be threaded into StartCommit. + // It must carry ACK intents because it may commit ahead of our own + // recovery log (remote-authoritative pattern). + // + // If `legacy_checkpoint`, then we're preserving a rollback capability + // to the V1 runtime. We reduce our delta Frontier extents into + // `full_frontier`, map the result into `full_checkpoint`, + // and then extend `connector_checkpoint` with `full_checkpoint`. + let connector_checkpoint = if let Some((full_frontier, full_checkpoint)) = legacy_checkpoint + { + *full_frontier = std::mem::take(full_frontier).reduce(extents.frontier.clone()); + + frontier_mapping::merge_frontier_into_checkpoint( + full_frontier, + full_checkpoint, + &task.binding_journal_read_suffixes, + ); + full_checkpoint + .sources + .insert(committed_close_key, committed_close_source); + + full_checkpoint.ack_intents = ack_intents.clone(); + full_checkpoint.clone() + } else { + consumer::Checkpoint { + ack_intents: ack_intents.clone(), + sources: [(committed_close_key, committed_close_source)].into(), + } + }; + + // Track for future Persist and post-Acknowledge writes. + pending.ack_intents = ack_intents; + + let action = Action::StartCommit { + connector_checkpoint, + connector_patches: take_patches(&mut pending.shard_patches), + }; + let state = HeadStartCommit { + extents, + pending, + shard_started_commit: vec![false; task.n_shards], + }; + + (action, Head::StartCommit(state)) + } +} + +/// HeadStartCommit awaits StartedCommit responses from all shards. +#[derive(Debug)] +pub struct HeadStartCommit { + pub extents: Extents, + pub pending: PendingDeltas, + + /// Per-shard tracking of StartedCommit response receipt. + pub shard_started_commit: Vec, +} + +impl HeadStartCommit { + pub fn step( + mut self, + legacy_checkpoint: &Option<(shuffle::Frontier, consumer::Checkpoint)>, + now: uuid::Clock, + shard_rx: &mut Option<(usize, proto::Materialize)>, + stopping: bool, + ) -> (Action, Head) { + // Did we receive an expected StartedCommit response? + if let Some(( + shard_index, + proto::Materialize { + started_commit: Some(started_commit), + .. + }, + )) = shard_rx + && self.shard_started_commit.get(*shard_index) == Some(&false) + { + let proto::materialize::StartedCommit { + connector_patches_json, + } = std::mem::take(started_commit); + + extend_patches(&mut self.pending, &connector_patches_json); + + // Mark received and consume `shard_rx`. + self.shard_started_commit[*shard_index] = true; + shard_rx.take(); + + if self.shard_started_commit.iter().all(|b| *b) { + self.shard_started_commit.clear(); // All received. + } + } + + if !self.shard_started_commit.is_empty() { + return (Action::Idle, Head::StartCommit(self)); + } + // We've received all expected StartedCommit responses. + + let Self { + extents, + mut pending, + .. + } = self; + + let Extents { + close, frontier, .. + } = extents; + + // If `legacy_checkpoint` is Some, then persist the legacy "checkpoint" + // key to maintain a rollback capability to the V1 runtime. + // The full Frontier was already merged into `full_checkpoint` during + // HeadWriteStats::step(). + let legacy_checkpoint = legacy_checkpoint + .as_ref() + .map(|(_full_frontier, full_checkpoint)| full_checkpoint.clone()); + + let persist = proto::Persist { + nonce: now.as_u64(), + ack_intents: pending.ack_intents.clone(), + committed_close_clock: close.as_u64(), + committed_frontier: Some(shuffle::JournalFrontier::encode(&frontier.journals)), + connector_patches_json: take_patches(&mut pending.persist_patches), + delete_ack_intents: true, + legacy_checkpoint, + max_keys: std::mem::take(&mut pending.max_key_deltas), + trigger_params_json: pending.trigger_params.clone(), + ..Default::default() + }; + + // If we're `stopping`, then transition to Stop after Persist. + let (next_action, next_state) = if stopping { + // By construction, we know Tail is Done and all post-commit activity + // of the *prior* transaction is completed. We halt after Persist + // (commit) without starting any post-commit activity: that's left + // for the next session, which will recover our commit state and + // resume from Tail::Begin. + (Action::Idle, Head::Stop) + } else { + // Rotate to begin a next transaction. `idempotent_replay` + // is one-shot — only the first transaction of a session may replay + // recovered hints, so post-Rotate HeadIdle is always non-replay. + ( + Action::Rotate { pending }, + Head::Idle(HeadIdle { + idempotent_replay: false, + last_close: close, + }), + ) + }; + + let state = HeadPersist { + nonce: persist.nonce, + next_action, + next_state: Box::new(next_state), + }; + let action = Action::Persist { persist }; + + (action, Head::Persist(state)) + } +} + +/// TailBegin is the initial state of the Tail FSM. +/// The transaction has committed, but isn't confirmed to have been acknowledged. +#[derive(Debug)] +pub struct TailBegin { + pub pending: PendingDeltas, +} + +impl TailBegin { + pub fn step(self, stopping: bool, task: &Task) -> (Action, Tail) { + let Self { mut pending } = self; + + // `stopping` can be true here only if it: + // a) arrived after commit Persist was emitted, but before Persisted, + // in which case Head emitted Rotate and not Stop, or + // b) because `on_transaction_completed` tripped on `max_transactions` + // being reached. + if stopping { + let action = Action::Idle; + let state = TailDone { + shard_patches: bytes::Bytes::new(), + }; + (action, Tail::Done(state)) + } else { + let action = Action::Acknowledge { + connector_patches: take_patches(&mut pending.shard_patches), + }; + let state = TailAcknowledge { + pending, + shard_acknowledged: vec![false; task.n_shards], + }; + + (action, Tail::Acknowledge(state)) + } + } +} + +/// TailAcknowledge awaits Acknowledged responses from all shards. +#[derive(Debug)] +pub struct TailAcknowledge { + pub pending: PendingDeltas, + + /// Per-shard tracking of Acknowledged response receipt. + pub shard_acknowledged: Vec, +} + +impl TailAcknowledge { + pub fn step( + mut self, + now: uuid::Clock, + shard_rx: &mut Option<(usize, proto::Materialize)>, + ) -> (Action, Tail) { + // Did we receive an expected Acknowledged response? + if let Some(( + shard_index, + proto::Materialize { + acknowledged: Some(acknowledged), + .. + }, + )) = shard_rx + && self.shard_acknowledged.get(*shard_index) == Some(&false) + { + let proto::materialize::Acknowledged { + connector_patches_json, + } = std::mem::take(acknowledged); + + extend_patches(&mut self.pending, &connector_patches_json); + + // Mark received and consume `shard_rx`. + self.shard_acknowledged[*shard_index] = true; + shard_rx.take(); + + if self.shard_acknowledged.iter().all(|b| *b) { + self.shard_acknowledged.clear(); // All received. + } + } + + if !self.shard_acknowledged.is_empty() { + return (Action::Idle, Tail::Acknowledge(self)); + } + // We've received all expected Acknowledged responses. + + let Self { + pending: + PendingDeltas { + ack_intents, + max_key_deltas: max_keys, + mut persist_patches, + mut shard_patches, + trigger_params, + }, + shard_acknowledged: _, + } = self; + + assert!(max_keys.is_empty()); + let persist_patches = take_patches(&mut persist_patches); + let shard_patches = take_patches(&mut shard_patches); + + // Base: call the trigger if needed, else go straight to Done. + let (mut action, mut state) = if trigger_params.is_empty() { + (Action::Idle, Tail::Done(TailDone { shard_patches })) + } else { + ( + Action::CallTrigger { trigger_params }, + Tail::Trigger(TailTrigger { shard_patches }), + ) + }; + + // Wrap with WriteIntents, so journal ACKs are appended immediately after + // a post-Acknowledge Persist completes (if one is required). + state = Tail::WriteIntents(TailWriteIntents { + next_action: action, + next_state: Box::new(state), + }); + action = Action::WriteIntents { ack_intents }; + + // If Acknowledged returned patches, wrap with a Persist that runs first. + if !persist_patches.is_empty() { + let nonce = now.as_u64(); + + state = Tail::Persist(TailPersist { + nonce, + next_action: action, + next_state: Box::new(state), + }); + action = Action::Persist { + persist: proto::Persist { + nonce, + connector_patches_json: persist_patches, + ..Default::default() + }, + }; + } + + (action, state) + } +} + +/// TailWriteIntents awaits the completion of ACK intent append and flush. +#[derive(Debug)] +pub struct TailWriteIntents { + pub next_action: Action, + pub next_state: Box, +} + +impl TailWriteIntents { + pub fn step(self, intents_write_idle: bool) -> (Action, Tail) { + if !intents_write_idle { + return (Action::Idle, Tail::WriteIntents(self)); + } + + let Self { + next_action, + next_state, + } = self; + + (next_action, *next_state) + } +} + +/// TailTrigger awaits the completion of a trigger call +#[derive(Debug)] +pub struct TailTrigger { + pub shard_patches: bytes::Bytes, +} + +impl TailTrigger { + pub fn step(self, now: uuid::Clock, trigger_call_running: bool) -> (Action, Tail) { + if trigger_call_running { + return (Action::Idle, Tail::Trigger(self)); + } + + let Self { shard_patches } = self; + + let nonce = now.as_u64(); + let action = Action::Persist { + persist: proto::Persist { + nonce, + delete_trigger_params: true, + ..Default::default() + }, + }; + let state = TailPersist { + nonce, + next_action: Action::Idle, + next_state: Box::new(Tail::Done(TailDone { shard_patches })), + }; + + (action, Tail::Persist(state)) + } +} + +/// TailPersist awaits a Persisted response from shard zero, +/// and chains its contained action and state. +#[derive(Debug)] +pub struct TailPersist { + pub nonce: u64, + pub next_action: Action, + pub next_state: Box, +} + +impl TailPersist { + pub fn step(self, shard_rx: &mut Option<(usize, proto::Materialize)>) -> (Action, Tail) { + if let Some(( + 0, + proto::Materialize { + persisted: Some(proto::Persisted { nonce }), + .. + }, + )) = shard_rx + && *nonce == self.nonce + { + shard_rx.take(); + + let Self { + nonce: _, + next_action, + next_state, + } = self; + + return (next_action, *next_state); + } + + (Action::Idle, Tail::Persist(self)) + } +} + +#[derive(Debug, Default)] +pub struct TailDone { + pub shard_patches: bytes::Bytes, +} + +/// Aggregated measures and flags driving an extend-vs-close evaluation. +#[derive(Debug, Clone, Copy)] +pub struct CloseInputs { + pub close_requested: bool, + pub idempotent_replay: bool, + pub last_age: std::time::Duration, + pub max_combiner: u64, + pub open_age: std::time::Duration, + pub read_bytes: u64, + pub read_docs: u64, + pub stopping: bool, + pub tail_done: bool, + pub unresolved_hints: bool, +} + +/// Outcome of an extend-vs-close evaluation. Both flags may be true: the +/// caller extends if a Frontier is ready and otherwise closes. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct CloseDecision { + pub may_extend: bool, + pub may_close: bool, +} + +/// Evaluate whether an open transaction may extend, close, or hold. +/// +/// Threshold policy with a hysteresis band per dimension: +/// - `policy_extend` while every measure is below its `range.end`. +/// - `policy_close` once every measure is above its `range.start`. +/// Usage-based measures saturate below `start` once `policy_extend` is false +/// (otherwise we'd live-lock because the threshold cannot be reached). +/// +/// Overrides: +/// - `close_requested`, or `idempotent_replay && !unresolved_hints`: force close. +/// - `unresolved_hints`: forces extend; suppresses close until hints resolve. +/// - `idempotent_replay`: suppresses extend (replay is one-shot). +/// - `stopping` with `may_close=true`: suppresses extend so Head can stop after +/// the next commit. With Tail still draining, extend is permitted to keep the +/// pipeline full. +/// - `!tail_done`: suppresses close (must hold open while Tail finishes). +pub fn decide_close_policy(inputs: CloseInputs, task: &Task) -> CloseDecision { + let CloseInputs { + open_age, + last_age, + max_combiner, + read_bytes, + read_docs, + close_requested, + idempotent_replay, + unresolved_hints, + stopping, + tail_done, + } = inputs; + + let policy_extend = open_age < task.open_duration.end + && last_age < task.last_close_age.end + && max_combiner < task.combiner_usage_bytes.end + && read_bytes < task.read_bytes.end + && read_docs < task.read_docs.end; + + let mut policy_close = open_age > task.open_duration.start + && last_age > task.last_close_age.start + && (!policy_extend || max_combiner > task.combiner_usage_bytes.start) + && (!policy_extend || read_bytes > task.read_bytes.start) + && (!policy_extend || read_docs > task.read_docs.start); + policy_close |= idempotent_replay && !unresolved_hints; + policy_close |= close_requested; + + let may_close = policy_close && !unresolved_hints && tail_done; + let may_extend = + (!idempotent_replay && policy_extend && (!stopping || !may_close)) || unresolved_hints; + + CloseDecision { + may_extend, + may_close, + } +} + +// Extend separate accrued patches for a future Persist vs future shard broadcast, +// into `pending` from `src`. +pub fn extend_patches(pending: &mut PendingDeltas, src: &[u8]) { + crate::patches::extend_state_patches(&mut pending.shard_patches, src); + crate::patches::extend_state_patches(&mut pending.persist_patches, src); +} + +// Take patches from `src`, leaving it empty, and freeze into Bytes. +pub fn take_patches(src: &mut Vec) -> bytes::Bytes { + bytes::Bytes::from(std::mem::take(src)) +} + +/// Build an `ops::Stats` document snapshotting this transaction's extents. +fn build_stats_doc( + task: &Task, + extents: &Extents, + binding_bytes_behind: &[i64], +) -> ops::proto::Stats { + let mut materialize: BTreeMap = BTreeMap::new(); + + for (binding_index, extents) in &extents.bindings { + let Some(collection_name) = task.binding_collection_names.get(*binding_index as usize) + else { + continue; // Reachable if shards report invalid binding indices. + }; + let entry = materialize.entry(collection_name.clone()).or_default(); + + // It's possible that multiple bindings source from the same collection. + // We accumulate when reporting by-collection. + entry.bytes_behind = entry.bytes_behind.saturating_add_signed( + binding_bytes_behind + .get(*binding_index as usize) + .copied() + .unwrap_or_default(), + ); + entry.last_source_published_at = extents.max_source_clock.to_pb_json_timestamp(); + + ops::merge_docs_and_bytes(&extents.sourced, &mut entry.left); + ops::merge_docs_and_bytes(&extents.loaded, &mut entry.right); + ops::merge_docs_and_bytes(&extents.stored, &mut entry.out); + } + + let open_seconds_total = uuid::Clock::delta(extents.close, extents.open).as_secs_f64(); + + ops::proto::Stats { + meta: Some(ops::proto::Meta { + uuid: String::new(), // Stamped by Publisher::enqueue() + }), + shard: Some(task.shard_ref.clone()), + timestamp: extents.open.to_pb_json_timestamp(), + open_seconds_total, + txn_count: 1, + materialize, + capture: Default::default(), // N/A. + derive: None, // N/A. + interval: None, // N/A. + } +} + +#[cfg(test)] +mod tests { + use super::*; + use bytes::Bytes; + use gazette::consumer; + use std::collections::BTreeMap; + use std::time::Duration; + + /// Aggregates the Actor's per-iteration locals so step_head / step_tail + /// can be driven without recreating the actor's IO scaffolding. + struct Ctx { + binding_bytes_behind: Vec, + close_requested: bool, + intents_idle: bool, + legacy_checkpoint: Option<(shuffle::Frontier, consumer::Checkpoint)>, + now: uuid::Clock, + pending_ack_intents: BTreeMap, + ready_frontier: Option, + shard_rx: Option<(usize, proto::Materialize)>, + stats_idle: bool, + stopping: bool, + task: Task, + trigger_running: bool, + } + + impl Ctx { + fn step_head(&mut self, head: Head, tail: &mut Tail) -> (Action, Head) { + self.now.tick(); + head.step( + &mut self.binding_bytes_behind, + &mut self.close_requested, + &mut self.legacy_checkpoint, + self.now, + &mut self.ready_frontier, + &mut self.shard_rx, + self.stats_idle.then_some(&mut self.pending_ack_intents), + self.stopping, + tail, + &self.task, + ) + } + + fn step_tail(&mut self, tail: Tail) -> (Action, Tail) { + self.now.tick(); + tail.step( + self.intents_idle, + self.now, + &mut self.shard_rx, + self.stopping, + &self.task, + self.trigger_running, + ) + } + } + + fn mk_task(n_shards: usize) -> Task { + // Wide thresholds so `policy_extend` is always true and `policy_close` + // only trips via `close_requested`. This keeps the test free of + // policy-driven close timing. + Task { + binding_collection_names: vec!["test/collection".to_string()], + binding_journal_read_suffixes: vec!["pivot=00".to_string()], + combiner_usage_bytes: 0..u64::MAX, + connector_image: String::new(), + last_close_age: Duration::ZERO..Duration::MAX, + max_transactions: 0, + n_shards, + open_duration: Duration::ZERO..Duration::MAX, + peers: (0..n_shards).map(|i| format!("shard-{i}")).collect(), + read_bytes: 0..u64::MAX, + read_docs: 0..u64::MAX, + shard_ref: ops::ShardRef::default(), + triggers: Some(std::sync::Arc::new( + super::super::triggers::CompiledTriggers::compile(vec![]).unwrap(), + )), + } + } + + fn mk_loaded(shard: usize) -> (usize, proto::Materialize) { + ( + shard, + proto::Materialize { + loaded: Some(proto::materialize::Loaded { + bindings: vec![proto::materialize::loaded::Binding { + index: 0, + min_source_clock: uuid::Clock::from_unix(1_700_000_005, 0).as_u64(), + max_source_clock: uuid::Clock::from_unix(1_700_000_010, 0).as_u64(), + sourced_docs_total: 3, + sourced_bytes_total: 300, + max_key_delta: Bytes::from_static(b"\x05\x06\x07"), + }], + combiner_usage_bytes: 0, + }), + ..Default::default() + }, + ) + } + + fn mk_flushed(shard: usize, patches: &'static [u8]) -> (usize, proto::Materialize) { + ( + shard, + proto::Materialize { + flushed: Some(proto::materialize::Flushed { + bindings: vec![proto::materialize::flushed::Binding { + index: 0, + loaded_docs_total: 2, + loaded_bytes_total: 200, + }], + connector_patches_json: Bytes::from_static(patches), + }), + ..Default::default() + }, + ) + } + + fn mk_stored(shard: usize) -> (usize, proto::Materialize) { + ( + shard, + proto::Materialize { + stored: Some(proto::materialize::Stored { + bindings: vec![proto::materialize::stored::Binding { + index: 0, + stored_docs_total: 4, + stored_bytes_total: 400, + }], + }), + ..Default::default() + }, + ) + } + + fn mk_started_commit(shard: usize, patches: &'static [u8]) -> (usize, proto::Materialize) { + ( + shard, + proto::Materialize { + started_commit: Some(proto::materialize::StartedCommit { + connector_patches_json: Bytes::from_static(patches), + }), + ..Default::default() + }, + ) + } + + fn mk_acknowledged(shard: usize, patches: &'static [u8]) -> (usize, proto::Materialize) { + ( + shard, + proto::Materialize { + acknowledged: Some(proto::materialize::Acknowledged { + connector_patches_json: Bytes::from_static(patches), + }), + ..Default::default() + }, + ) + } + + fn mk_head_persisted(head: &Head) -> (usize, proto::Materialize) { + let nonce = match head { + Head::Persist(p) => p.nonce, + other => panic!("expected Head::Persist, got {other:?}"), + }; + ( + 0, + proto::Materialize { + persisted: Some(proto::Persisted { nonce }), + ..Default::default() + }, + ) + } + + fn mk_tail_persisted(tail: &Tail) -> (usize, proto::Materialize) { + let nonce = match tail { + Tail::Persist(p) => p.nonce, + other => panic!("expected Tail::Persist, got {other:?}"), + }; + ( + 0, + proto::Materialize { + persisted: Some(proto::Persisted { nonce }), + ..Default::default() + }, + ) + } + + /// Table-driven coverage of `decide_close_policy`. The task's hysteresis + /// bands are 1..5 (s/bytes/docs); `mid` sits in-band on every dimension. + #[test] + fn close_policy_table() { + let task = Task { + combiner_usage_bytes: 1..5, + last_close_age: Duration::from_secs(1)..Duration::from_secs(5), + open_duration: Duration::from_secs(1)..Duration::from_secs(5), + read_bytes: 1..5, + read_docs: 1..5, + // Unused by `decide_close_policy`. + binding_collection_names: vec![], + binding_journal_read_suffixes: vec![], + connector_image: String::new(), + max_transactions: 0, + n_shards: 1, + peers: vec![], + shard_ref: ops::ShardRef::default(), + triggers: None, + }; + + // `mid` is permissive across dimensions and flags: every measure is + // inside its band, no overrides are active, and Tail is done. From + // here, individual cases nudge one or two fields to exercise each + // policy / override branch. + let mid = CloseInputs { + open_age: Duration::from_secs(3), + last_age: Duration::from_secs(3), + max_combiner: 3, + read_bytes: 3, + read_docs: 3, + close_requested: false, + idempotent_replay: false, + unresolved_hints: false, + stopping: false, + tail_done: true, + }; + + struct Case { + name: &'static str, + inputs: CloseInputs, + want: CloseDecision, + } + + let want = |may_extend, may_close| CloseDecision { + may_extend, + may_close, + }; + + let cases = [ + Case { + name: "in-band: may extend or close", + inputs: mid, + want: want(true, true), + }, + Case { + name: "below all minima: extend only", + inputs: CloseInputs { + open_age: Duration::ZERO, + last_age: Duration::ZERO, + max_combiner: 0, + read_bytes: 0, + read_docs: 0, + ..mid + }, + want: want(true, false), + }, + Case { + name: "saturated combiner: close only", + inputs: CloseInputs { + max_combiner: 10, + ..mid + }, + want: want(false, true), + }, + Case { + name: "saturated combiner but open_age below min: hold", + inputs: CloseInputs { + open_age: Duration::ZERO, + max_combiner: 10, + ..mid + }, + want: want(false, false), + }, + Case { + name: "close_requested overrides policy floor", + inputs: CloseInputs { + open_age: Duration::ZERO, + last_age: Duration::ZERO, + read_bytes: 0, + read_docs: 0, + max_combiner: 0, + close_requested: true, + ..mid + }, + want: want(true, true), + }, + Case { + name: "close_requested but tail still busy: hold open", + inputs: CloseInputs { + close_requested: true, + tail_done: false, + ..mid + }, + want: want(true, false), + }, + Case { + name: "close_requested but unresolved hints: extend forced, close suppressed", + inputs: CloseInputs { + close_requested: true, + unresolved_hints: true, + ..mid + }, + want: want(true, false), + }, + Case { + name: "idempotent_replay with hints resolved: close only", + inputs: CloseInputs { + open_age: Duration::ZERO, + idempotent_replay: true, + ..mid + }, + want: want(false, true), + }, + Case { + name: "idempotent_replay with unresolved hints: extend forced", + inputs: CloseInputs { + open_age: Duration::ZERO, + idempotent_replay: true, + unresolved_hints: true, + ..mid + }, + want: want(true, false), + }, + Case { + name: "stopping with may_close: extend suppressed", + inputs: CloseInputs { + close_requested: true, + stopping: true, + ..mid + }, + want: want(false, true), + }, + Case { + name: "stopping with tail busy: keep pipeline full", + inputs: CloseInputs { + stopping: true, + tail_done: false, + ..mid + }, + want: want(true, false), + }, + Case { + name: "unresolved hints: extend forced, close suppressed", + inputs: CloseInputs { + unresolved_hints: true, + ..mid + }, + want: want(true, false), + }, + ]; + + for case in cases { + let got = decide_close_policy(case.inputs, &task); + assert_eq!( + got, case.want, + "case `{}` failed: inputs={:?}", + case.name, case.inputs, + ); + } + } + + /// Walks Head and Tail through two pipelined transactions and a graceful + /// stop. No IO; each step mutates Ctx fields and reads back the + /// (Action, State) tuple. + /// + /// Phase 1: txn 1 opens, extends once, closes on `close_requested`, and + /// drives the full commit sequence ending in Action::Rotate. + /// Phase 2: rotation hands `pending` to Tail::Begin. Head opens txn 2 + /// (one Load); while Head awaits the second Loaded, Tail's full + /// post-acknowledge sequence runs interleaved: Acknowledged x2 + /// (with patches) → Persist → Persisted → WriteIntents. Head + /// then receives Loaded(1) and extends txn 2 with another Load + /// round. + /// Phase 3: `stopping` is set; Tail drains WriteIntents → CallTrigger → + /// Persist → Persisted → Done. + /// Phase 4: Head commits txn 2; with `stopping=true` HeadStartCommit + /// chains into (Action::Idle, Head::Stop) instead of Rotate. + #[test] + fn happy_path_two_transactions_then_stop() { + let task = mk_task(2); + let mut ctx = Ctx { + binding_bytes_behind: vec![0; task.binding_collection_names.len()], + close_requested: false, + intents_idle: true, + legacy_checkpoint: None, + now: uuid::Clock::from_unix(1_700_000_000, 0), + pending_ack_intents: BTreeMap::new(), + ready_frontier: None, + shard_rx: None, + stats_idle: false, + stopping: false, + task, + trigger_running: false, + }; + let mut head = Head::Idle(HeadIdle::default()); + let mut tail = Tail::Done(TailDone::default()); + + // ===== Phase 1: txn 1 lifecycle ===== + + // HeadIdle observes a ready Frontier and broadcasts L:Load. + ctx.ready_frontier = Some(shuffle::Frontier::default()); + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(action, Action::Load { .. })); + assert!(matches!(head, Head::Extend(_))); + + // Loaded responses arrive from each shard. After both have landed + // HeadExtend rests (policy allows extend-or-close, neither input is + // provided) — the action is Sleep/Idle, which we don't assert. + for s in 0..2 { + ctx.shard_rx = Some(mk_loaded(s)); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(ctx.shard_rx.is_none(), "Loaded was consumed"); + } + assert!(matches!(head, Head::Extend(_))); + + // Extend the open transaction with a second ready Frontier. + ctx.ready_frontier = Some(shuffle::Frontier::default()); + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(action, Action::Load { .. })); + + for s in 0..2 { + ctx.shard_rx = Some(mk_loaded(s)); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + } + + // Close: with Tail::Done and no unresolved hints, `close_requested` + // forces may_close and we begin L:Flush. + ctx.close_requested = true; + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(action, Action::Flush { .. })); + assert!(matches!(head, Head::Flush(_))); + + // Flushed x2 with distinct connector state patches → idempotency + // Persist that carries the merged Flushed patches. + ctx.shard_rx = Some(mk_flushed(0, b"[{\"phase\":\"flushed\",\"shard\":0}\n]")); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + + ctx.shard_rx = Some(mk_flushed(1, b"[{\"phase\":\"flushed\",\"shard\":1}\n]")); + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(head, Head::Persist(_))); + let persist = match action { + Action::Persist { persist } => persist, + other => panic!("expected Action::Persist, got {other:?}"), + }; + insta::assert_debug_snapshot!( + (&persist.connector_patches_json, &persist.max_keys), + @r#" + ( + b"[{\"phase\":\"flushed\",\"shard\":0}\n,{\"phase\":\"flushed\",\"shard\":1}\n]", + { + 0: b"\x05\x06\x07", + }, + ) + "#); + + // Persisted (shard 0) → Store. + ctx.shard_rx = Some(mk_head_persisted(&head)); + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(action, Action::Store)); + assert!(matches!(head, Head::Store(_))); + + // Stored x2 → WriteStats. Capture the stats action of the second + // step for an inline snapshot of the resulting stats document. + let mut write_stats_action = None; + for s in 0..2 { + ctx.shard_rx = Some(mk_stored(s)); + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + if s == 1 { + write_stats_action = Some(action); + } + } + assert!(matches!(head, Head::WriteStats(_))); + let stats = match write_stats_action.unwrap() { + Action::WriteStats { stats } => stats, + other => panic!("expected Action::WriteStats, got {other:?}"), + }; + insta::assert_json_snapshot!(stats, @r#" + { + "_meta": {}, + "shard": {}, + "ts": "2023-11-14T22:13:20.000000004+00:00", + "openSecondsTotal": 0.000000024, + "txnCount": 1, + "materialize": { + "test/collection": { + "left": { + "docsTotal": 12, + "bytesTotal": 1200 + }, + "right": { + "docsTotal": 4, + "bytesTotal": 400 + }, + "out": { + "docsTotal": 8, + "bytesTotal": 800 + }, + "lastSourcePublishedAt": "2023-11-14T22:13:30+00:00" + } + } + } + "#); + + // Stats publish completes; ACK intents become available → StartCommit. + ctx.pending_ack_intents + .insert("ops/journal".to_string(), Bytes::from_static(b"intent-1")); + ctx.stats_idle = true; + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + ctx.stats_idle = false; + assert!(matches!(action, Action::StartCommit { .. })); + assert!(matches!(head, Head::StartCommit(_))); + + // StartedCommit x2 with distinct connector state patches → committing + // Persist that carries the merged StartedCommit patches. + ctx.shard_rx = Some(mk_started_commit( + 0, + b"[{\"phase\":\"committed\",\"shard\":0}\n]", + )); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + + ctx.shard_rx = Some(mk_started_commit( + 1, + b"[{\"phase\":\"committed\",\"shard\":1}\n]", + )); + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(head, Head::Persist(_))); + let persist = match action { + Action::Persist { persist } => persist, + other => panic!("expected Action::Persist, got {other:?}"), + }; + insta::assert_debug_snapshot!( + (&persist.connector_patches_json, &persist.trigger_params_json), + @r#" + ( + b"[{\"phase\":\"committed\",\"shard\":0}\n,{\"phase\":\"committed\",\"shard\":1}\n]", + b"{\"collection_names\":[\"test/collection\"],\"connector_image\":\"\",\"materialization_name\":\"\",\"flow_published_at_min\":\"2023-11-14T22:13:25+00:00\",\"flow_published_at_max\":\"2023-11-14T22:13:30+00:00\",\"run_id\":\"2023-11-14T22:13:20.000000004+00:00\"}", + ) + "#); + + // Final Persisted → Action::Rotate (since !stopping). Head returns to + // Idle. The Actor's Rotate dispatch transitions Tail::Begin(pending). + ctx.shard_rx = Some(mk_head_persisted(&head)); + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + let pending = match action { + Action::Rotate { pending } => pending, + other => panic!("expected Action::Rotate, got {other:?}"), + }; + assert!(matches!(head, Head::Idle(_))); + tail = Tail::Begin(TailBegin { pending }); + + // ===== Phase 2: pipeline txn 2; Tail's post-acknowledge sequence + // interleaves between Head's Load and second Loaded ===== + + // TailBegin → Acknowledge. + let (action, t) = ctx.step_tail(tail); + tail = t; + assert!(matches!(action, Action::Acknowledge { .. })); + assert!(matches!(tail, Tail::Acknowledge(_))); + + // Head opens txn 2 via a fresh ready Frontier — pipelined with Tail. + ctx.ready_frontier = Some(shuffle::Frontier::default()); + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(action, Action::Load { .. })); + assert!(matches!(head, Head::Extend(_))); + + // Head receives Loaded from shard 0 (one of two). + ctx.shard_rx = Some(mk_loaded(0)); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(head, Head::Extend(_))); + + // --- Interleave: Tail's post-acknowledge work between Loaded(0) + // and Loaded(1) of Head's txn 2 Load round --- + + // Acknowledged from shard 0 carries connector patches. + ctx.shard_rx = Some(mk_acknowledged(0, b"[{\"phase\":\"acked\",\"shard\":0}\n]")); + let (action, t) = ctx.step_tail(tail); + tail = t; + assert!(matches!(action, Action::Idle)); + assert!(matches!(tail, Tail::Acknowledge(_))); + + // Acknowledged from shard 1 carries no patches; Tail has now seen all + // Acknowledged. Because shard 0's patches are non-empty, the chain + // wraps with TailPersist and emits Action::Persist *first*. The + // WriteIntents and CallTrigger steps will fire after Persisted. + ctx.shard_rx = Some(mk_acknowledged(1, b"")); + let (action, t) = ctx.step_tail(tail); + tail = t; + assert!(matches!(tail, Tail::Persist(_))); + let persist = match action { + Action::Persist { persist } => persist, + other => panic!("expected Action::Persist, got {other:?}"), + }; + insta::assert_debug_snapshot!(&persist.connector_patches_json, @r#"b"[{\"phase\":\"acked\",\"shard\":0}\n]""#); + + // Persisted (post-Acknowledge) → chained next_action = WriteIntents. + ctx.shard_rx = Some(mk_tail_persisted(&tail)); + let (action, t) = ctx.step_tail(tail); + tail = t; + assert!(matches!(action, Action::WriteIntents { .. })); + assert!(matches!(tail, Tail::WriteIntents(_))); + + // --- End interleave; Head receives Loaded(1) to complete the round. --- + + ctx.shard_rx = Some(mk_loaded(1)); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + + // Extend txn 2 with another ready Frontier (Tail still in WriteIntents). + ctx.ready_frontier = Some(shuffle::Frontier::default()); + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(action, Action::Load { .. })); + + for s in 0..2 { + ctx.shard_rx = Some(mk_loaded(s)); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + } + + // ===== Phase 3: stop signal; drain Tail through trigger to Done ===== + + ctx.stopping = true; + + // WriteIntents → CallTrigger (intents publish completed; task has + // triggers configured so trigger_params is non-empty). + let (action, t) = ctx.step_tail(tail); + tail = t; + assert!(matches!(action, Action::CallTrigger { .. })); + assert!(matches!(tail, Tail::Trigger(_))); + + // Trigger call completes (trigger_running=false) → final Persist + // (with delete_trigger_params=true). + let (action, t) = ctx.step_tail(tail); + tail = t; + assert!(matches!(action, Action::Persist { .. })); + assert!(matches!(tail, Tail::Persist(_))); + + // Persisted → Tail::Done. + ctx.shard_rx = Some(mk_tail_persisted(&tail)); + let (action, t) = ctx.step_tail(tail); + tail = t; + assert!(matches!(action, Action::Idle)); + assert!(matches!(tail, Tail::Done(_))); + + // ===== Phase 4: commit txn 2 under stopping; observe Head::Stop ===== + + // Drive close via policy this time (Phase 1 covered `close_requested`). + // Shrinking `open_duration.end` below the current `open_age` flips + // `policy_extend` to false, which lets `policy_close` trip and (under + // `stopping`) suppresses extend so Head closes on the next step. + ctx.task.open_duration.end = Duration::from_nanos(1); + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(action, Action::Flush { .. })); + + for s in 0..2 { + ctx.shard_rx = Some(mk_flushed(s, b"")); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + } + ctx.shard_rx = Some(mk_head_persisted(&head)); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(head, Head::Store(_))); + + for s in 0..2 { + ctx.shard_rx = Some(mk_stored(s)); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + } + assert!(matches!(head, Head::WriteStats(_))); + + ctx.stats_idle = true; + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + ctx.stats_idle = false; + assert!(matches!(action, Action::StartCommit { .. })); + + for s in 0..2 { + ctx.shard_rx = Some(mk_started_commit(s, b"")); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + } + + // Final Persisted under stopping: HeadStartCommit chained + // (next_action, next_state) = (Idle, Head::Stop) — no Rotate. + ctx.shard_rx = Some(mk_head_persisted(&head)); + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(action, Action::Idle)); + assert!(matches!(head, Head::Stop)); + assert!(matches!(tail, Tail::Done(_))); + } + + /// Fuzz Head and Tail by perturbing every Ctx field at each step. + /// Random shard responses, frontiers, and idle/stopping flags drive + /// arbitrary state transitions; the test asserts no panics. The FSMs + /// are expected to handle malformed or out-of-order inputs gracefully + /// (ignore unexpected responses, hold their current state) rather than + /// crashing — most random sequences therefore never advance to commit, + /// but they also never trip an `unwrap`/`unreachable!`/index panic. + #[test] + fn fuzz_head_tail_no_panics() { + use rand::{Rng, SeedableRng, rngs::SmallRng}; + + // Synthesize a Materialize message of a randomly chosen variant. The + // `expected_nonce` is plumbed through so Persisted occasionally matches + // the in-progress nonce and lets HeadPersist / TailPersist actually + // chain forward — without it, fuzz traces would rarely leave Persist. + fn random_message( + shard: usize, + expected_nonce: u64, + rng: &mut SmallRng, + ) -> (usize, proto::Materialize) { + let mut msg = proto::Materialize::default(); + // Cap accumulator inputs to keep `+= bytes_total` etc. far from + // u64 overflow over the fuzz length (Rust panics on debug overflow). + match rng.random_range(0..6) { + 0 => { + msg.loaded = Some(proto::materialize::Loaded { + bindings: vec![proto::materialize::loaded::Binding { + index: rng.random_range(0..3), + min_source_clock: rng.random(), + max_source_clock: rng.random(), + sourced_bytes_total: rng.random_range(0..1_000), + sourced_docs_total: rng.random_range(0..100), + max_key_delta: Bytes::from_static(b"\x01\x02\x03"), + }], + combiner_usage_bytes: rng.random_range(0..1_000_000), + }); + } + 1 => { + msg.flushed = Some(proto::materialize::Flushed { + bindings: vec![proto::materialize::flushed::Binding { + index: rng.random_range(0..3), + loaded_bytes_total: rng.random_range(0..1_000), + loaded_docs_total: rng.random_range(0..100), + }], + connector_patches_json: Bytes::from_static(b"[{\"f\":1}\n]"), + }); + } + 2 => { + msg.stored = Some(proto::materialize::Stored { + bindings: vec![proto::materialize::stored::Binding { + index: rng.random_range(0..3), + stored_bytes_total: rng.random_range(0..1_000), + stored_docs_total: rng.random_range(0..100), + }], + }); + } + 3 => { + msg.started_commit = Some(proto::materialize::StartedCommit { + connector_patches_json: Bytes::from_static(b"[{\"sc\":1}\n]"), + }); + } + 4 => { + msg.acknowledged = Some(proto::materialize::Acknowledged { + connector_patches_json: Bytes::from_static(b"[{\"ack\":1}\n]"), + }); + } + _ => { + // Most of the time, target the in-progress Persist's nonce so + // the FSM can actually chain forward; otherwise emit garbage. + let nonce = if rng.random_bool(0.9) { + expected_nonce + } else { + rng.random() + }; + msg.persisted = Some(proto::Persisted { nonce }); + } + } + (shard, msg) + } + + // Pick a "best-guess" nonce to hand to `random_message`. When Head or + // Tail is awaiting Persisted we surface its nonce so the message is + // sometimes accepted; otherwise return random noise. + fn pick_nonce(head: &Head, tail: &Tail, rng: &mut SmallRng) -> u64 { + if let Head::Persist(p) = head { + return p.nonce; + } + if let Tail::Persist(p) = tail { + return p.nonce; + } + rng.random() + } + + fn perturb(ctx: &mut Ctx, head: &Head, tail: &Tail, rng: &mut SmallRng) { + ctx.now.tick(); + + // Independently flip each Boolean knob with low probability so a + // run typically spans many distinct (close_requested, stopping, + // *_idle, trigger_running) combinations. + if rng.random_bool(0.20) { + ctx.close_requested = !ctx.close_requested; + } + if rng.random_bool(0.20) { + ctx.intents_idle = !ctx.intents_idle; + } + if rng.random_bool(0.20) { + ctx.stats_idle = !ctx.stats_idle; + } + // `stopping` is stickier: flipping rarely lets fuzz traces actually + // reach Head::Stop instead of toggling out of it on the next step. + if rng.random_bool(0.05) { + ctx.stopping = !ctx.stopping; + } + if rng.random_bool(0.20) { + ctx.trigger_running = !ctx.trigger_running; + } + + // Inject a Frontier with a randomized `unresolved_hints` so we + // cover the unresolved-hints branch of the close policy. Journals + // are kept empty to avoid Frontier validation invariants. + if rng.random_bool(0.30) { + ctx.ready_frontier = Some(shuffle::Frontier { + unresolved_hints: if rng.random_bool(0.7) { + 0 + } else { + rng.random_range(1..3) + }, + ..Default::default() + }); + } + + // Inject a shard response. Allow shard index up to n_shards + // (sometimes out-of-range) to exercise bounds handling. + if rng.random_bool(0.50) { + let shard = rng.random_range(0..=ctx.task.n_shards); + let nonce = pick_nonce(head, tail, rng); + ctx.shard_rx = Some(random_message(shard, nonce, rng)); + } + + // Add an ACK intent occasionally; HeadWriteStats drains them. + if rng.random_bool(0.10) { + ctx.pending_ack_intents.insert( + format!("ops/journal-{}", rng.random_range(0..4)), + Bytes::from_static(b"intent"), + ); + } + + // Toggle `legacy_checkpoint` to cover the V1-rollback merge branch. + if rng.random_bool(0.05) { + ctx.legacy_checkpoint = if ctx.legacy_checkpoint.is_some() { + None + } else { + Some(( + shuffle::Frontier::default(), + consumer::Checkpoint::default(), + )) + }; + } + } + + fn prop(seed: u64) -> bool { + let mut rng = SmallRng::seed_from_u64(seed); + let n_shards = rng.random_range(1..=4); + + // Narrow the close-policy thresholds (vs `mk_task`'s wide ranges) + // so `policy_extend` flips false after a single typical Loaded + // response, which lets `policy_close` trip frequently and drives + // fuzz traces through Flush / Store / Persist / Rotate. Without + // this, Head spends almost the entire trace in Extend. + let mut task = mk_task(n_shards); + task.combiner_usage_bytes = 0..10_000; + task.read_bytes = 0..500; + task.read_docs = 0..20; + + let mut ctx = Ctx { + binding_bytes_behind: vec![0; 3], + close_requested: false, + intents_idle: false, + legacy_checkpoint: None, + now: uuid::Clock::from_unix(1_700_000_000, 0), + pending_ack_intents: BTreeMap::new(), + ready_frontier: None, + shard_rx: None, + stats_idle: false, + stopping: false, + task, + trigger_running: false, + }; + let mut head = Head::Idle(HeadIdle::default()); + let mut tail = Tail::Done(TailDone::default()); + + for _ in 0..256 { + perturb(&mut ctx, &head, &tail, &mut rng); + + if rng.random_bool(0.5) { + // Head::Stop panics at the step boundary by contract, so + // skip stepping it. The Actor analogously stops dispatching + // once Head reaches Stop. + if !matches!(head, Head::Stop) { + let (action, h) = ctx.step_head(head, &mut tail); + head = h; + // Mirror the Actor's Rotate dispatch: hand `pending` to + // Tail::Begin so fuzz traces actually exercise Tail's + // Acknowledge / WriteIntents / Trigger paths after a + // Head commit, instead of leaving Tail wedged in Done. + if let Action::Rotate { pending } = action { + tail = Tail::Begin(TailBegin { pending }); + } + } + } else { + let (_action, t) = ctx.step_tail(tail); + tail = t; + } + } + true + } + + quickcheck::QuickCheck::new() + .tests(200) + .max_tests(400) + .quickcheck(prop as fn(u64) -> bool); + } +} diff --git a/crates/runtime-next/src/leader/materialize/handler.rs b/crates/runtime-next/src/leader/materialize/handler.rs new file mode 100644 index 00000000000..c6795da175f --- /dev/null +++ b/crates/runtime-next/src/leader/materialize/handler.rs @@ -0,0 +1,197 @@ +use super::{actor, fsm, startup}; +use crate::{leader, proto}; +use futures::StreamExt; +use tokio::sync::mpsc; + +pub(crate) async fn serve( + service: crate::Service, + mut request_rx: R, + response_tx: mpsc::UnboundedSender>, +) -> anyhow::Result<()> +where + R: futures::Stream> + Send + Unpin + 'static, +{ + let verify = crate::verify("Materialize", "Join", "shard"); + + // Read the Join request. + let join = match verify.not_eof(request_rx.next().await)? { + proto::Materialize { + join: Some(join), .. + } => join, + request => return Err(verify.fail_msg(request)), + }; + let task_name = leader::validate_join(&join)?.to_string(); + + tracing::info!( + %task_name, + shards = join.shards.len(), + shard_index = join.shard_index, + etcd_mod_revision = join.etcd_mod_revision, + shuffle_directory = %join.shuffle_directory, + "received Join", + ); + + // Scope `guard` to prove it's not held across await points. + let outcome = { + let mut guard = service.materialize_joins.lock().unwrap(); + + let outcome = guard.entry(task_name.to_string()).or_default().register( + join, + request_rx.boxed(), + response_tx, + ); + if !matches!(&outcome, leader::JoinOutcome::Pending { .. }) { + guard.remove(&task_name); + } + outcome + }; + + let slots = match outcome { + leader::JoinOutcome::Pending { filled, target } => { + tracing::debug!( + %task_name, + filled, + target, + "registered pending Join", + ); + return Ok(()); + } + leader::JoinOutcome::Disagreement(slots) => { + let max_etcd_revision = slots + .iter() + .map(|s| s.join.etcd_mod_revision) + .max() + .unwrap(); + + tracing::info!( + %task_name, + max_etcd_revision, + retrying = slots.len(), + "broadcasting retry due to topology disagreement", + ); + let retry = proto::Materialize { + joined: Some(proto::Joined { max_etcd_revision }), + ..Default::default() + }; + for slot in slots { + let _ = slot.response_tx.send(Ok(retry.clone())); + } + return Ok(()); + } + + leader::JoinOutcome::Consensus(slots) => slots, + }; + + tracing::info!( + task_name, + shards = slots.len(), + "consensus reached; starting session", + ); + + let mut build = String::new(); + let mut reactors: Vec = Vec::new(); + let mut shard_rx = Vec::with_capacity(slots.len()); + let mut shard_tx = Vec::with_capacity(slots.len()); + let mut shard_ids = Vec::with_capacity(slots.len()); + let mut shard_shuffles: Vec = Vec::with_capacity(slots.len()); + + for slot in slots { + let leader::JoinSlot { + join: + proto::Join { + etcd_mod_revision: _, + shards: mut slot_shards, + shard_index, + shuffle_directory: directory, + shuffle_endpoint: endpoint, + leader_endpoint: _, + }, + request_rx: slot_rx, + response_tx: slot_tx, + } = slot; + + let proto::join::Shard { + id, + labeling, + reactor, + etcd_create_revision: _, + } = std::mem::take(&mut slot_shards[shard_index as usize]); + + let labeling = labeling.unwrap_or_default(); + + reactors.push(reactor.unwrap_or_default().suffix); + shard_rx.push(slot_rx); + shard_tx.push(slot_tx); + shard_ids.push(id); + shard_shuffles.push(shuffle::proto::Shard { + range: labeling.range, + directory, + endpoint, + }); + build = labeling.build; + } + + let error_tx = shard_tx.clone(); + + // Run startup, and then the Actor transaction loop. + let result = async move { + let startup::Startup { + committed_close, + committed_frontier, + idempotent_replay, + pending_ack_intents, + pending_trigger_params, + publisher, + session, + task, + } = startup::run( + build, + reactors, + &mut shard_rx, + &mut shard_tx, + &service, + shard_ids, + shard_shuffles, + ) + .await?; + + let head = fsm::Head::Idle(fsm::HeadIdle { + last_close: committed_close, + idempotent_replay, + }); + let pending = fsm::PendingDeltas { + ack_intents: pending_ack_intents, + trigger_params: pending_trigger_params, + ..Default::default() + }; + let tail = fsm::Tail::Begin(fsm::TailBegin { pending }); + + // TODO: Make this toggle-able for dropping rollback support. + let legacy_checkpoint = Some(committed_frontier); + + let mut actor = actor::Actor::new( + service.http_client.clone(), + legacy_checkpoint, + publisher, + shard_tx, + task, + ); + actor.serve(head, tail, session, shard_rx).await + } + .await; + + let Err(err) = result else { + return Ok(()); + }; + + // Best-effort broadcast of terminal error to all shards. + let status = match err.downcast_ref::() { + Some(status) => status.clone(), + None => tonic::Status::unknown(format!("{err:?}")), + }; + for tx in error_tx { + let _ = tx.send(Err(status.clone())); + } + + Err(err) +} diff --git a/crates/runtime-next/src/leader/materialize/mod.rs b/crates/runtime-next/src/leader/materialize/mod.rs new file mode 100644 index 00000000000..6caee52a03d --- /dev/null +++ b/crates/runtime-next/src/leader/materialize/mod.rs @@ -0,0 +1,43 @@ +mod actor; +mod frontier_mapping; +mod fsm; +mod handler; +mod startup; +mod task; +mod triggers; + +pub(crate) use handler::serve; + +// Task configuration, as understood by the leader. +// +// Several fields express "close" vs "extend" policy thresholds: +// - A transaction may close once `min` is met on every dimension. +// - A transaction may be extended if below `max` on every dimension. +struct Task { + // Collection name of each materialization binding. + binding_collection_names: Vec, + // Journal read suffix of each materialization binding. + binding_journal_read_suffixes: Vec, + // Min/max desired combiner disk byte utilization. + combiner_usage_bytes: std::ops::Range, + // OCI image of the connector, or empty if not an image connector. + connector_image: String, + // Min/max desired age of the last transaction (elapsed since last txn close). + last_close_age: std::ops::Range, + // Maximum completed transactions before graceful stop. Zero means unlimited. + max_transactions: u32, + // Number of shards lead by this leader. + n_shards: usize, + // Min/max desired duration of an open transaction (elapsed since first ready checkpoint). + open_duration: std::ops::Range, + // Descriptive peer names of each shard, for logging and errors. + peers: Vec, + // Min/max desired bytes read in a transaction. + read_bytes: std::ops::Range, + // Min/max desired documents read in a transaction. + read_docs: std::ops::Range, + // ShardRef embedded in every stats document. + shard_ref: ops::ShardRef, + // Compiled triggers, or None if the task has no triggers configured. + triggers: Option>, +} diff --git a/crates/runtime-next/src/leader/materialize/snapshots/runtime_next__leader__materialize__triggers__test__rendered-template.snap b/crates/runtime-next/src/leader/materialize/snapshots/runtime_next__leader__materialize__triggers__test__rendered-template.snap new file mode 100644 index 00000000000..a75a268106b --- /dev/null +++ b/crates/runtime-next/src/leader/materialize/snapshots/runtime_next__leader__materialize__triggers__test__rendered-template.snap @@ -0,0 +1,17 @@ +--- +source: crates/runtime-next/src/leader/materialize/triggers.rs +assertion_line: 262 +expression: parsed +--- +{ + "auth": "Bearer my-secret", + "collections": [ + "acmeCo/example/collection" + ], + "connector": "ghcr.io/estuary/materialize-example:v1", + "event": "materialization_transaction_completed", + "flow_published_at_max": "2024-01-01T00:01:00Z", + "flow_published_at_min": "2024-01-01T00:00:00Z", + "materialization": "acmeCo/example/materialization", + "run_id": "2024-01-01T00:00:00.000Z" +} diff --git a/crates/runtime-next/src/leader/materialize/startup.rs b/crates/runtime-next/src/leader/materialize/startup.rs new file mode 100644 index 00000000000..90760f81cca --- /dev/null +++ b/crates/runtime-next/src/leader/materialize/startup.rs @@ -0,0 +1,834 @@ +use super::{Task, frontier_mapping}; +use crate::proto; +use anyhow::Context; +use bytes::Bytes; +use futures::{StreamExt, stream::BoxStream}; +use prost::Message; +use proto_flow::flow; +use proto_gazette::uuid; +use std::collections::BTreeMap; +use tokio::sync::mpsc; + +/// Outcomes of the leader protocol startup phase. +pub(super) struct Startup { + // Clock at which the last-committed transaction closed. + pub committed_close: uuid::Clock, + // Fully committed Frontier. + pub committed_frontier: shuffle::Frontier, + // Is the first transaction an idempotent replay of a recovered hinted Frontier? + pub idempotent_replay: bool, + // Recovered ACK intents of the last transaction. + pub pending_ack_intents: BTreeMap, + // Recovered variables for the task. + pub pending_trigger_params: Bytes, + // Publisher for writing stats and ACK intents. + pub publisher: crate::Publisher, + // Initiated shuffle session for the task and topology. + pub session: shuffle::SessionClient, + // Task definition. + pub task: Task, +} + +#[tracing::instrument( + level = "debug", + err(Debug, level = "warn"), + skip_all, + fields(shard_zero = %shard_ids[0], shards = shard_ids.len()) +)] +pub(super) async fn run( + build: String, + reactors: Vec, + shard_rx: &mut Vec>>, + shard_tx: &Vec>>, + service: &crate::Service, + shard_ids: Vec, + shard_shuffles: Vec, +) -> anyhow::Result { + let n_shards = reactors.len(); + assert_eq!(n_shards, shard_rx.len()); + assert_eq!(n_shards, shard_tx.len()); + assert_eq!(n_shards, shard_ids.len()); + assert_eq!(n_shards, shard_shuffles.len()); + + let peers: Vec = shard_ids + .iter() + .zip(reactors.iter()) + .map(|(shard_id, reactor)| format!("{shard_id}@{reactor}")) + .collect(); + + // Send L:Joined response to all shards. + for tx in shard_tx { + let _ = tx.send(Ok(proto::Materialize { + joined: Some(proto::Joined { + max_etcd_revision: 0, // Success. + }), + ..Default::default() + })); + } + + // Receive L:Task from shard zero. + let verify = crate::verify("Materialize", "Open", &peers[0]); + let task = match verify.not_eof(shard_rx[0].next().await)? { + proto::Materialize { + task: Some(task), .. + } => task, + other => return Err(verify.fail_msg(other)), + }; + + // Build task definition. + let proto::Task { + ops_stats_journal, + ops_stats_spec, + preview, + max_transactions, + spec: spec_bytes, + } = task; + + let spec = flow::MaterializationSpec::decode(spec_bytes.as_ref()) + .context("invalid Task materialization")?; + let task = Task::new(build, &spec, max_transactions, peers) + .await + .context("building task definition")?; + + // Initialize publisher. + let publisher = if preview { + crate::Publisher::new_preview() + } else { + let ops_stats_spec = ops_stats_spec.as_ref().context("missing ops stats spec")?; + + crate::Publisher::new_real( + shard_ids[0].clone(), // Shard zero is AuthZ subject. + &service.publisher_factory, + &ops_stats_journal, + ops_stats_spec, + [], // No additional bindings. + ) + .context("creating publisher")? + }; + + // Receive Recover fan-in. + let proto::Recover { + ack_intents: mut pending_ack_intents, + committed_close_clock: committed_close, + committed_frontier, + mut connector_state_json, + hinted_close_clock: hinted_close, + hinted_frontier, + last_applied, + legacy_checkpoint, + max_keys, + trigger_params_json: pending_trigger_params, + } = recv_recovers(shard_rx, &task.peers) + .await + .context("receiving Recover fan-in")?; + + let mut committed_close = uuid::Clock::from_u64(committed_close); + let hinted_close = uuid::Clock::from_u64(hinted_close); + let legacy_checkpoint = legacy_checkpoint.unwrap_or_default(); + + let mut hinted_frontier = shuffle::Frontier::decode(hinted_frontier.unwrap_or_default()) + .context("validating hinted Frontier")?; + let mut committed_frontier = shuffle::Frontier::decode(committed_frontier.unwrap_or_default()) + .context("validating committed Frontier")?; + + tracing::debug!( + ?committed_close, + ?committed_frontier, + connector_state_bytes = connector_state_json.len(), + ?hinted_close, + ?hinted_frontier, + last_applied = !last_applied.is_empty(), + ?legacy_checkpoint, + ?max_keys, + ?pending_trigger_params, + "collected Recover from all shards", + ); + + // Run Apply on shard zero until convergence. + apply_loop( + &mut shard_rx[0], + &shard_tx[0], + &task.peers[0], + &last_applied, + &spec_bytes, + &task.shard_ref.build, + &mut connector_state_json, + ) + .await?; + + // Open connectors across all shards. + for (tx, shard) in shard_tx.iter().zip(shard_shuffles.iter()) { + let _ = tx.send(Ok(proto::Materialize { + open: Some(proto::Open { + spec: spec_bytes.clone(), + version: task.shard_ref.build.clone(), + range: shard.range.clone(), + connector_state_json: connector_state_json.clone(), + max_keys: max_keys.clone(), + }), + ..Default::default() + })); + } + + // Receive Opened fan-in. + let proto::materialize::Opened { + container: _, // Not sent to leader. + connector_checkpoint, + } = recv_opened(shard_rx, &task.peers) + .await + .context("receiving Opened fan-in")?; + let connector_checkpoint = connector_checkpoint.unwrap_or_default(); + + // Build sorted index on journal_read_suffix => binding index, for frontier mapping. + let mut journal_read_suffix_index: Vec<(&str, usize)> = task + .binding_journal_read_suffixes + .iter() + .enumerate() + .map(|(i, b)| (b.as_str(), i)) + .collect(); + journal_read_suffix_index.sort(); + + // Handle migration from `legacy_checkpoint`. + if !legacy_checkpoint.sources.is_empty() { + let clock = frontier_mapping::extract_committed_close(&legacy_checkpoint); + + if clock == Some(committed_close) { + tracing::debug!( + ?committed_close, + "legacy_checkpoint present but matches Recover::committed_close (ignoring)" + ); + } else if let Some(clock) = clock { + // Implementation error: these update together and should always sync. + anyhow::bail!( + "legacy_checkpoint has clock {clock:?} that doesn't match Recover's committed_close ({committed_close:?})" + ); + } else { + tracing::debug!( + ?committed_close, + ?legacy_checkpoint, + "legacy_checkpoint doesn't contain committed-close-clock; treating as authoritative" + ); + committed_frontier = frontier_mapping::checkpoint_to_frontier( + &legacy_checkpoint.sources, + &journal_read_suffix_index, + ) + .context("mapping recovered legacy checkpoint into Frontier")?; + + pending_ack_intents = legacy_checkpoint.ack_intents; + } + } else { + tracing::debug!("no legacy_checkpoint present"); + } + + // Handle a `connector_checkpoint` from remote-authoritative connectors. + // It may be *ahead* of `committed_frontier`, which is detect as its embedded + // committed-close Clock matching our recovered `hinted_close`. + if !connector_checkpoint.sources.is_empty() { + let clock = frontier_mapping::extract_committed_close(&connector_checkpoint); + + if clock == Some(committed_close) { + tracing::debug!( + ?committed_close, + "connector_checkpoint present but matches Recover::committed_close (ignoring)" + ); + } else if clock == Some(hinted_close) { + // Connector declares that the hinted txn did in fact commit. + tracing::debug!( + ?committed_close, + ?hinted_close, + ?hinted_frontier, + "connector_checkpoint present and matches Recover::hinted_close; applying delta" + ); + committed_close = hinted_close; + committed_frontier = committed_frontier.reduce(std::mem::take(&mut hinted_frontier)); + + pending_ack_intents = connector_checkpoint.ack_intents; + } else if let Some(clock) = clock { + // Implementation error: these update together and should always sync. + anyhow::bail!( + "connector_checkpoint has clock {clock:?} which doesn't match Recover's\ + committed_close ({committed_close:?}) or hinted_close ({hinted_close:?})" + ); + } else { + tracing::debug!( + ?committed_close, + ?connector_checkpoint, + "connector_checkpoint doesn't contain committed-close-clock; treating as authoritative" + ); + + committed_frontier = frontier_mapping::checkpoint_to_frontier( + &connector_checkpoint.sources, + &journal_read_suffix_index, + ) + .context("mapping recovered connector checkpoint into Frontier")?; + + pending_ack_intents = connector_checkpoint.ack_intents; + } + } else { + tracing::debug!("no connector_checkpoint present"); + } + + // Compose the session resume Frontier: project the recovered hinted + // Frontier into hinted form (last_commit -> hinted_commit, zero + // last_commit/offset) and reduce with the committed Frontier. + let resume_frontier = + frontier_mapping::project_hinted(hinted_frontier).reduce(committed_frontier.clone()); + + // If we recovered a producer frontier with an unapplied hinted commit, + // then the first transaction must be an idempotent replay of the hinted frontier. + let idempotent_replay = resume_frontier.unresolved_hints != 0; + + // Open the shuffle Session with the recovered resume Frontier. + let shuffle_task = shuffle::proto::Task { + task: Some(shuffle::proto::task::Task::Materialization(spec)), + }; + let session = shuffle::SessionClient::open( + &service.shuffle_service, + shuffle_task, + shard_shuffles, + resume_frontier, + ) + .await + .context("opening shuffle Session")?; + + Ok(Startup { + committed_close, + committed_frontier, + idempotent_replay, + pending_ack_intents, + pending_trigger_params, + publisher, + session, + task, + }) +} + +async fn recv_recovers( + request_rxs: &mut [BoxStream<'static, tonic::Result>], + peers: &[String], +) -> anyhow::Result { + let mut recovers = futures::future::try_join_all(request_rxs.into_iter().enumerate().map( + |(shard_index, rx)| async move { + let verify = crate::verify("Materialize", "Recover", &peers[shard_index]); + match verify.not_eof(rx.next().await)? { + proto::Materialize { + recover: Some(recover), + .. + } if shard_index == 0 || recover == proto::Recover::default() => { + Ok::<_, anyhow::Error>(recover) + } + other => Err(verify.fail_msg(other)), + } + }, + )) + .await?; + + Ok(recovers.swap_remove(0)) +} + +// The apply loop's persistent state machine is `(last_applied, +// connector_state_json)`. Each iteration may persist new connector state +// patches; `last_applied` is bumped only on the FINAL iteration once the +// connector returns no further patches. A crash mid-loop therefore resumes +// with the OLD `last_applied` against the partially-advanced state, +// requiring the connector's Apply to be idempotent across repeated +// invocations of the same target spec. +async fn apply_loop( + rx: &mut BoxStream<'static, tonic::Result>, + tx: &mpsc::UnboundedSender>, + peer: &str, + last_applied: &Bytes, + next_applied: &Bytes, + next_version: &str, + connector_state_json: &mut Bytes, +) -> anyhow::Result<()> { + let verify_applied = crate::verify("Materialize", "Applied", peer); + let verify_persisted = crate::verify("Materialize", "Persisted", peer); + let last_version = if last_applied.is_empty() { + String::new() + } else { + let last_spec = flow::MaterializationSpec::decode(last_applied.as_ref()) + .context("invalid recovered last-applied spec")?; + labels_build_for(&last_spec) + }; + + const MAX_APPLY_ITERATIONS: u64 = 3; + + for iteration in 1..=MAX_APPLY_ITERATIONS { + // Send Apply carrying the current reduced connector state. + // Sends are best-effort: a closed peer will surface on the next rx. + let _ = tx.send(Ok(proto::Materialize { + apply: Some(proto::Apply { + spec: next_applied.clone(), + version: next_version.to_string(), + last_spec: last_applied.clone(), + last_version: last_version.clone(), + connector_state_json: connector_state_json.clone(), + }), + ..Default::default() + })); + + // Receive Applied. + let applied_patches_json = match verify_applied.not_eof(rx.next().await)? { + proto::Materialize { + applied: + Some(proto::Applied { + action_description, + connector_patches_json, + }), + .. + } => { + tracing::info!( + iteration, + action_description, + patches = ?ops::DebugJson( + serde_json::from_slice::(&connector_patches_json) + .unwrap_or_default() + ), + "connector Apply completed" + ); + connector_patches_json + } + other => return Err(verify_applied.fail_msg(other)), + }; + + if applied_patches_json.is_empty() { + tracing::debug!(iteration, "apply loop complete"); + + if last_applied == next_applied { + return Ok(()); + } + + let _ = tx.send(Ok(proto::Materialize { + persist: Some(proto::Persist { + nonce: iteration, + last_applied: next_applied.clone(), + ..Default::default() + }), + ..Default::default() + })); + + match verify_persisted.not_eof(rx.next().await)? { + proto::Materialize { + persisted: Some(proto::Persisted { nonce }), + .. + } if nonce == iteration => {} + other => return Err(verify_persisted.fail_msg(other)), + } + + return Ok(()); + } + + // Fold the iteration's patches into the running reduced state so + // subsequent Apply iterations and Open observe the newly-applied state. + *connector_state_json = + crate::patches::apply_state_patches(connector_state_json, &applied_patches_json)?; + + // Persist the iteration's patches to shard zero. + let _ = tx.send(Ok(proto::Materialize { + persist: Some(proto::Persist { + nonce: iteration, // End-of-sequence. + connector_patches_json: applied_patches_json, + ..Default::default() + }), + ..Default::default() + })); + + // Receive Persisted. + match verify_persisted.not_eof(rx.next().await)? { + proto::Materialize { + persisted: Some(proto::Persisted { nonce }), + .. + } if nonce == iteration => {} + other => return Err(verify_persisted.fail_msg(other)), + } + } + + anyhow::bail!( + "apply loop did not converge after {MAX_APPLY_ITERATIONS} iterations; \ + connector continues to return state patches" + ); +} + +fn labels_build_for(spec: &flow::MaterializationSpec) -> String { + let Some(template) = spec.shard_template.as_ref() else { + return String::new(); + }; + let Some(set) = template.labels.as_ref() else { + return String::new(); + }; + + labels::expect_one(set, labels::BUILD) + .unwrap_or_default() + .to_string() +} + +async fn recv_opened( + request_rxs: &mut [BoxStream<'static, tonic::Result>], + peers: &[String], +) -> anyhow::Result { + let mut openeds = futures::future::try_join_all(request_rxs.iter_mut().enumerate().map( + |(shard_index, rx)| async move { + let verify = crate::verify("Materialize", "Opened", &peers[shard_index]); + match verify.not_eof(rx.next().await)? { + proto::Materialize { + opened: Some(opened), + .. + } if shard_index == 0 || opened == proto::materialize::Opened::default() => { + Ok::<_, anyhow::Error>(opened) + } + other => Err(verify.fail_msg(other)), + } + }, + )) + .await?; + + Ok(openeds.swap_remove(0)) +} + +#[cfg(test)] +mod tests { + use super::*; + use tokio_stream::wrappers::UnboundedReceiverStream; + + // Build a peer/leader pair of channels, returning the BoxStream + // the apply_loop would read from and the receiver of leader-side + // sends, plus a peer-side sender used to inject responses. + fn channel_pair() -> ( + BoxStream<'static, tonic::Result>, + mpsc::UnboundedSender>, + mpsc::UnboundedSender>, + mpsc::UnboundedReceiver>, + ) { + let (peer_tx, peer_rx) = mpsc::unbounded_channel(); + let (leader_tx, leader_rx) = mpsc::unbounded_channel(); + let stream = UnboundedReceiverStream::new(peer_rx).boxed(); + (stream, peer_tx, leader_tx, leader_rx) + } + + fn applied(patches: &'static [u8]) -> proto::Materialize { + proto::Materialize { + applied: Some(proto::Applied { + action_description: String::new(), + connector_patches_json: Bytes::from_static(patches), + }), + ..Default::default() + } + } + + fn persisted(nonce: u64) -> proto::Materialize { + proto::Materialize { + persisted: Some(proto::Persisted { nonce }), + ..Default::default() + } + } + + #[tokio::test] + async fn apply_loop_no_op_when_last_eq_next_and_no_patches() { + // When last_applied == next_applied and the connector returns no + // patches, we issue a single Apply and return without Persist. + let (mut rx, peer_tx, leader_tx, mut leader_rx) = channel_pair(); + peer_tx.send(Ok(applied(b""))).unwrap(); + + let same = Bytes::new(); + let mut state = Bytes::from_static(b"{\"k\":1}"); + apply_loop(&mut rx, &leader_tx, "p", &same, &same, "v1", &mut state) + .await + .unwrap(); + + let m = leader_rx.try_recv().unwrap().unwrap(); + let apply = m.apply.expect("Apply was sent"); + assert!(apply.last_spec.is_empty()); + assert!(apply.spec.is_empty()); + assert_eq!(apply.version, "v1"); + // No Persist since spec is unchanged. + assert!(leader_rx.try_recv().is_err()); + // State is unchanged. + assert_eq!(state.as_ref(), b"{\"k\":1}"); + } + + #[tokio::test] + async fn apply_loop_persists_last_applied_when_no_patches_but_spec_changed() { + // No patches but next != last: loop sends Apply, then Persist + // marking next_applied as the new last_applied with matching nonce. + let (mut rx, peer_tx, leader_tx, mut leader_rx) = channel_pair(); + peer_tx.send(Ok(applied(b""))).unwrap(); + peer_tx.send(Ok(persisted(1))).unwrap(); + + let last = Bytes::new(); + let next = Bytes::from_static(b"new-spec-bytes"); + let mut state = Bytes::from_static(b"{}"); + apply_loop(&mut rx, &leader_tx, "p", &last, &next, "v2", &mut state) + .await + .unwrap(); + + let m1 = leader_rx.try_recv().unwrap().unwrap(); + let apply = m1.apply.unwrap(); + assert_eq!(apply.spec, next); + assert!(apply.last_spec.is_empty()); + assert_eq!(apply.last_version, ""); + + let m2 = leader_rx.try_recv().unwrap().unwrap(); + let p = m2.persist.unwrap(); + assert_eq!(p.nonce, 1); + assert_eq!(p.last_applied, next); + assert!(p.connector_patches_json.is_empty()); + + assert!(leader_rx.try_recv().is_err()); + } + + #[tokio::test] + async fn apply_loop_accumulates_patches_across_iterations() { + // Initial state has nested objects, a key the patches will overwrite, + // and a key the patches will delete (RFC 7396 null sentinel). Across + // two patching iterations we should observe: deep merge of `nested`, + // replacement of `keep`, deletion of `drop`, and addition of `added`. + let (mut rx, peer_tx, leader_tx, mut leader_rx) = channel_pair(); + + let patch1 = b"[{\"nested\":{\"a\":1},\"keep\":\"v1\"}\n]"; + let patch2 = b"[{\"nested\":{\"b\":2},\"keep\":\"v2\",\"drop\":null,\"added\":true}\n]"; + peer_tx.send(Ok(applied(patch1))).unwrap(); + peer_tx.send(Ok(persisted(1))).unwrap(); + peer_tx.send(Ok(applied(patch2))).unwrap(); + peer_tx.send(Ok(persisted(2))).unwrap(); + peer_tx.send(Ok(applied(b""))).unwrap(); + peer_tx.send(Ok(persisted(3))).unwrap(); + + let last = Bytes::new(); + let next = Bytes::from_static(b"spec"); + let mut state = Bytes::from_static(br#"{"nested":{"a":0},"keep":"v0","drop":"x"}"#); + apply_loop(&mut rx, &leader_tx, "p", &last, &next, "v2", &mut state) + .await + .unwrap(); + + // Apply (iter 1) — connector observes the original state. + let apply1 = leader_rx.try_recv().unwrap().unwrap().apply.unwrap(); + let s1: serde_json::Value = serde_json::from_slice(&apply1.connector_state_json).unwrap(); + assert_eq!( + s1, + serde_json::json!({"nested":{"a":0},"keep":"v0","drop":"x"}) + ); + // Persist iter 1 carries the connector's patches but no last_applied. + let p1 = leader_rx.try_recv().unwrap().unwrap().persist.unwrap(); + assert_eq!(p1.nonce, 1); + assert!(p1.last_applied.is_empty()); + assert_eq!(p1.connector_patches_json.as_ref(), patch1); + + // Apply (iter 2) — `keep` replaced, `nested.a` retained from iter 1. + let apply2 = leader_rx.try_recv().unwrap().unwrap().apply.unwrap(); + let s2: serde_json::Value = serde_json::from_slice(&apply2.connector_state_json).unwrap(); + assert_eq!( + s2, + serde_json::json!({"nested":{"a":1},"keep":"v1","drop":"x"}), + ); + let p2 = leader_rx.try_recv().unwrap().unwrap().persist.unwrap(); + assert_eq!(p2.nonce, 2); + assert!(p2.last_applied.is_empty()); + assert_eq!(p2.connector_patches_json.as_ref(), patch2); + + // Apply (iter 3) — `drop` removed via null, `nested` deep-merged, `added` added. + let apply3 = leader_rx.try_recv().unwrap().unwrap().apply.unwrap(); + let s3: serde_json::Value = serde_json::from_slice(&apply3.connector_state_json).unwrap(); + assert_eq!( + s3, + serde_json::json!({"nested":{"a":1,"b":2},"keep":"v2","added":true}), + ); + // Final Persist promotes spec and carries no patches. + let p3 = leader_rx.try_recv().unwrap().unwrap().persist.unwrap(); + assert_eq!(p3.nonce, 3); + assert_eq!(p3.last_applied, next); + assert!(p3.connector_patches_json.is_empty()); + + // Final reduced state escapes via &mut, observable to caller. + let final_state: serde_json::Value = serde_json::from_slice(&state).unwrap(); + assert_eq!( + final_state, + serde_json::json!({"nested":{"a":1,"b":2},"keep":"v2","added":true}), + ); + } + + #[tokio::test] + async fn apply_loop_error_paths() { + struct Case { + name: &'static str, + // Build the peer-side responses; closure receives the peer tx. + seed: fn(&mpsc::UnboundedSender>), + expect: &'static str, + } + let cases = [ + Case { + // Connector returns patches forever; we cap at MAX_APPLY_ITERATIONS. + name: "no_convergence", + seed: |tx| { + for nonce in 1..=4 { + tx.send(Ok(applied(b"[{\"x\":1}\n]"))).unwrap(); + tx.send(Ok(persisted(nonce))).unwrap(); + } + }, + expect: "did not converge", + }, + Case { + // Peer returns Persisted with a wrong nonce — protocol error. + name: "persisted_nonce_mismatch", + seed: |tx| { + tx.send(Ok(applied(b"[{\"x\":1}\n]"))).unwrap(); + tx.send(Ok(persisted(99))).unwrap(); + }, + expect: "expected Persisted", + }, + Case { + // Peer sends a non-Applied message in response to Apply. + name: "unexpected_message_kind", + seed: |tx| { + tx.send(Ok(proto::Materialize { + opened: Some(proto::materialize::Opened::default()), + ..Default::default() + })) + .unwrap(); + }, + expect: "expected Applied", + }, + Case { + // Peer closes the stream without sending Applied — surfaces as EOF. + name: "eof", + seed: |_tx| {}, + expect: "unexpected EOF", + }, + ]; + + for case in cases { + let (mut rx, peer_tx, leader_tx, _leader_rx) = channel_pair(); + (case.seed)(&peer_tx); + drop(peer_tx); + + let last = Bytes::new(); + let next = Bytes::from_static(b"spec"); + let mut state = Bytes::from_static(b"{}"); + let err = apply_loop(&mut rx, &leader_tx, "p", &last, &next, "v2", &mut state) + .await + .unwrap_err(); + let s = format!("{err:?}"); + assert!( + s.contains(case.expect), + "{}: missing {:?} in {s}", + case.name, + case.expect, + ); + } + } + + fn make_streams( + per_shard: Vec>>, + ) -> Vec>> { + per_shard + .into_iter() + .map(|msgs| { + let (tx, rx) = mpsc::unbounded_channel(); + for m in msgs { + tx.send(m).unwrap(); + } + drop(tx); + UnboundedReceiverStream::new(rx).boxed() + }) + .collect() + } + + fn recover_msg(recover: proto::Recover) -> tonic::Result { + Ok(proto::Materialize { + recover: Some(recover), + ..Default::default() + }) + } + + #[tokio::test] + async fn recv_recovers_returns_shard_zero_value() { + let zero = proto::Recover { + committed_close_clock: 42, + ..Default::default() + }; + let mut streams = make_streams(vec![ + vec![recover_msg(zero.clone())], + vec![recover_msg(proto::Recover::default())], + vec![recover_msg(proto::Recover::default())], + ]); + let peers = vec!["s0".into(), "s1".into(), "s2".into()]; + let got = recv_recovers(&mut streams, &peers).await.unwrap(); + assert_eq!(got, zero); + } + + #[tokio::test] + async fn recv_recovers_error_paths() { + let cases = [ + ( + "non_default_from_non_zero_shard", + vec![ + vec![recover_msg(proto::Recover::default())], + vec![recover_msg(proto::Recover { + committed_close_clock: 1, + ..Default::default() + })], + ], + vec!["expected Recover", "from s1"], + ), + ( + "wrong_message_kind", + vec![vec![Ok(proto::Materialize { + opened: Some(proto::materialize::Opened::default()), + ..Default::default() + })]], + vec!["expected Recover"], + ), + ("eof", vec![vec![]], vec!["unexpected EOF"]), + ]; + + for (name, per_shard, needles) in cases { + let mut streams = make_streams(per_shard); + let peers: Vec = (0..streams.len()).map(|i| format!("s{i}")).collect(); + let err = recv_recovers(&mut streams, &peers).await.unwrap_err(); + let s = format!("{err:?}"); + for n in needles { + assert!(s.contains(n), "{name}: missing {n:?} in {s}"); + } + } + } + + #[tokio::test] + async fn recv_opened_returns_shard_zero_value_and_rejects_others() { + let zero = proto::materialize::Opened { + container: None, + connector_checkpoint: Some(proto_gazette::consumer::Checkpoint::default()), + }; + let mut streams = make_streams(vec![ + vec![Ok(proto::Materialize { + opened: Some(zero.clone()), + ..Default::default() + })], + vec![Ok(proto::Materialize { + opened: Some(proto::materialize::Opened::default()), + ..Default::default() + })], + ]); + let peers = vec!["s0".into(), "s1".into()]; + let got = recv_opened(&mut streams, &peers).await.unwrap(); + assert_eq!(got, zero); + + // Now a non-zero shard sends a populated Opened — error. + let mut streams = make_streams(vec![ + vec![Ok(proto::Materialize { + opened: Some(proto::materialize::Opened::default()), + ..Default::default() + })], + vec![Ok(proto::Materialize { + opened: Some(proto::materialize::Opened { + container: None, + connector_checkpoint: Some(proto_gazette::consumer::Checkpoint::default()), + }), + ..Default::default() + })], + ]); + let err = recv_opened(&mut streams, &peers).await.unwrap_err(); + let s = format!("{err:?}"); + assert!(s.contains("expected Opened")); + assert!(s.contains("from s1")); + } +} diff --git a/crates/runtime-next/src/leader/materialize/task.rs b/crates/runtime-next/src/leader/materialize/task.rs new file mode 100644 index 00000000000..8db1c29b722 --- /dev/null +++ b/crates/runtime-next/src/leader/materialize/task.rs @@ -0,0 +1,137 @@ +use super::{Task, triggers}; +use anyhow::Context; +use proto_flow::flow; +use proto_gazette::consumer; + +impl Task { + pub async fn new( + build: String, + spec: &flow::MaterializationSpec, + max_transactions: u32, + peers: Vec, + ) -> anyhow::Result { + let flow::MaterializationSpec { + name, + shard_template, + bindings, + triggers_json, + connector_type, + config_json, + .. + } = spec; + + let consumer::ShardSpec { + min_txn_duration, + max_txn_duration, + .. + } = shard_template.as_ref().context("missing shard template")?; + + let mut binding_collection_names = Vec::with_capacity(bindings.len()); + let mut binding_journal_read_suffixes = Vec::with_capacity(bindings.len()); + + for binding in bindings { + let flow::materialization_spec::Binding { + collection, + journal_read_suffix, + .. + } = binding; + + let flow::CollectionSpec { + name: collection_name, + .. + } = collection.as_ref().context("missing collection")?; + + binding_collection_names.push(collection_name.clone()); + binding_journal_read_suffixes.push(journal_read_suffix.clone()); + } + + // Extract the connector image from the config, if applicable. + let connector_image = + if *connector_type == flow::materialization_spec::ConnectorType::Image as i32 { + serde_json::from_slice::(&config_json) + .context("parsing connector config")? + .image + } else { + String::new() + }; + + let triggers = if triggers_json.is_empty() { + None + } else { + Some(std::sync::Arc::new( + decrypt_and_compile_triggers(&triggers_json).await?, + )) + }; + + let min_txn_duration = min_txn_duration + .context("missing min_txn_duration")? + .try_into()?; + let max_txn_duration = max_txn_duration + .context("missing max_txn_duration")? + .try_into()?; + + let shard_ref = ops::ShardRef { + kind: ops::TaskType::Materialization as i32, + name: name.clone(), + key_begin: labels::KEY_BEGIN_MIN.to_string(), + r_clock_begin: labels::RCLOCK_BEGIN_MIN.to_string(), + build, + }; + + // Close-policy thresholds, many with placeholder defaults. + // TODO: thread these through from the spec once they're supported there. + let open_duration: std::ops::Range = + min_txn_duration..max_txn_duration; + let last_close_age = std::time::Duration::ZERO..std::time::Duration::from_secs(300); + let combiner_usage_bytes = 0..(30 * 1024 * 1024 * 1024); + let read_docs = 0..u64::MAX; + let read_bytes = 0..u64::MAX; + + Ok(Self { + binding_collection_names, + binding_journal_read_suffixes, + combiner_usage_bytes, + connector_image, + last_close_age, + max_transactions, + n_shards: peers.len(), + open_duration, + peers, + read_bytes, + read_docs, + shard_ref, + triggers, + }) + } +} + +async fn decrypt_and_compile_triggers( + triggers: &[u8], +) -> anyhow::Result { + let mut triggers: models::Triggers = + serde_json::from_slice(triggers).context("parsing triggers JSON")?; + + // Strip HMAC-excluded fields before decryption (they were stripped + // during encryption so SOPS HMAC doesn't cover them), then restore. + let originals = models::triggers::strip_hmac_excluded_fields(&mut triggers); + + let stripped = models::RawValue::from_string( + serde_json::to_string(&triggers).expect("triggers always serialize"), + ) + .expect("trigger serialization is JSON"); + + let mut decrypted: models::Triggers = serde_json::from_str( + unseal::decrypt_sops(&stripped) + .await + .context("decrypting triggers_json")? + .get(), + ) + .context("parsing decrypted triggers JSON")?; + + models::triggers::restore_hmac_excluded_fields(&mut decrypted, originals); + + let compiled = triggers::CompiledTriggers::compile(decrypted.config) + .context("compiling trigger templates")?; + + Ok(compiled) +} diff --git a/crates/runtime-next/src/leader/materialize/triggers.rs b/crates/runtime-next/src/leader/materialize/triggers.rs new file mode 100644 index 00000000000..9e4975f8b18 --- /dev/null +++ b/crates/runtime-next/src/leader/materialize/triggers.rs @@ -0,0 +1,361 @@ +use anyhow::Context; +use models::TriggerVariables; + +/// Pre-compiled trigger templates and their associated configs. +pub struct CompiledTriggers { + pub configs: Vec, + registry: handlebars::Handlebars<'static>, +} + +impl CompiledTriggers { + /// Compile all trigger payload templates into a shared Handlebars registry. + pub fn compile(configs: Vec) -> anyhow::Result { + let mut registry = handlebars::Handlebars::new(); + registry.set_strict_mode(true); + registry.register_escape_fn(handlebars::no_escape); + + for (index, config) in configs.iter().enumerate() { + registry + .register_template_string(&Self::template_name(index), &config.payload_template) + .with_context(|| format!("compiling trigger {index} template"))?; + } + + Ok(Self { configs, registry }) + } + + /// Render the template for trigger `index` with the given context. + pub fn render(&self, index: usize, context: &serde_json::Value) -> anyhow::Result { + self.registry + .render(&Self::template_name(index), context) + .with_context(|| format!("rendering trigger {index} template")) + } + + fn template_name(index: usize) -> String { + format!("trigger_{index}") + } +} + +/// Fire all configured triggers using the given variables. +pub async fn fire_pending_triggers( + compiled: &CompiledTriggers, + variables: &TriggerVariables, + client: &reqwest::Client, +) -> anyhow::Result<()> { + let started_at = std::time::Instant::now(); + + send_webhooks( + compiled, + variables, + client, + std::time::Duration::from_secs(1), + ) + .await + .context("trigger webhook delivery failed")?; + + tracing::info!( + num_triggers = compiled.configs.len(), + elapsed_ms = started_at.elapsed().as_millis() as u64, + "trigger webhooks delivered successfully", + ); + + Ok(()) +} + +/// Render and send all configured trigger webhooks concurrently. +pub async fn send_webhooks( + compiled: &CompiledTriggers, + variables: &TriggerVariables, + client: &reqwest::Client, + base_backoff: std::time::Duration, +) -> anyhow::Result<()> { + if compiled.configs.is_empty() { + return Ok(()); + } + + let rendered: Vec = compiled + .configs + .iter() + .enumerate() + .map(|(index, trigger)| { + let context = models::build_template_context(variables, &trigger.headers); + compiled.render(index, &context) + }) + .collect::>()?; + + let futures: Vec<_> = compiled + .configs + .iter() + .zip(rendered) + .enumerate() + .map(|(index, (trigger, body))| { + send_single_webhook(index, trigger, body, client, base_backoff) + }) + .collect(); + + let results = futures::future::join_all(futures).await; + + let errors: Vec = results + .into_iter() + .filter_map(|r| r.err().map(|e| e.to_string())) + .collect(); + + if errors.is_empty() { + Ok(()) + } else { + anyhow::bail!("{} trigger(s) failed: {}", errors.len(), errors.join("; ")) + } +} + +async fn send_single_webhook( + index: usize, + trigger: &models::TriggerConfig, + body: String, + client: &reqwest::Client, + base_backoff: std::time::Duration, +) -> anyhow::Result<()> { + let total_attempts = trigger.max_attempts.max(1); + + let mut last_err = String::new(); + + for attempt in 0..total_attempts { + if attempt > 0 { + // Exponential backoff capped at 30 seconds. + let delay = base_backoff + .saturating_mul(2u32.saturating_pow(attempt - 1)) + .min(std::time::Duration::from_secs(30)); + tokio::time::sleep(delay).await; + } + + let method = match trigger.method { + models::HttpMethod::POST => reqwest::Method::POST, + models::HttpMethod::PUT => reqwest::Method::PUT, + models::HttpMethod::PATCH => reqwest::Method::PATCH, + }; + let has_content_type = trigger + .headers + .keys() + .any(|k| k.eq_ignore_ascii_case("content-type")); + + let mut request = client + .request(method, &trigger.url) + .timeout(trigger.timeout) + .body(body.clone()); + + if !has_content_type { + request = request.header(reqwest::header::CONTENT_TYPE, "application/json"); + } + for (name, value) in &trigger.headers { + request = request.header(name, value); + } + + match request.send().await { + Ok(response) if response.status().is_success() => { + return Ok(()); + } + Ok(response) => { + let status = response.status(); + let response_body = response + .text() + .await + .unwrap_or_else(|_| "".to_string()); + + last_err = format!("HTTP {status}: {response_body}"); + + // 4xx errors (other than 408 Request Timeout and 429 Too Many + // Requests) indicate a client-side problem that won't resolve + // on retry — fail immediately. + if status.is_client_error() + && status != reqwest::StatusCode::REQUEST_TIMEOUT + && status != reqwest::StatusCode::TOO_MANY_REQUESTS + { + anyhow::bail!( + "trigger {index} ({}) received non-retryable {status}: {response_body}", + trigger.url, + ); + } + + tracing::warn!( + trigger_index = index, + url = %trigger.url, + %status, + attempt = attempt + 1, + total_attempts, + "trigger webhook received non-success response, will retry" + ); + } + Err(err) => { + last_err = err.to_string(); + tracing::warn!( + trigger_index = index, + url = %trigger.url, + error = %err, + attempt = attempt + 1, + total_attempts, + "trigger webhook request failed, will retry" + ); + } + } + } + + anyhow::bail!( + "trigger {index} ({}) exhausted {total_attempts} attempts: {last_err}", + trigger.url, + ) +} + +#[cfg(test)] +mod test { + use super::*; + use models::TriggerVariables; + use std::sync::Arc; + use std::sync::atomic::{AtomicU32, Ordering}; + + async fn start_mock_server( + app: axum::Router, + ) -> (std::net::SocketAddr, tokio::task::JoinHandle<()>) { + let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap(); + let addr = listener.local_addr().unwrap(); + let handle = tokio::spawn(async move { + axum::serve(listener, app).await.ok(); + }); + (addr, handle) + } + + fn make_trigger_with_url(url: &str, template: &str) -> models::TriggerConfig { + models::TriggerConfig { + url: url.to_string(), + method: models::HttpMethod::POST, + headers: Default::default(), + payload_template: template.to_string(), + timeout: std::time::Duration::from_secs(5), + max_attempts: 3, + } + } + + #[test] + fn render_template() { + let mut trigger = make_trigger_with_url( + "https://example.com/webhook", + r#"{ + "event": "materialization_transaction_completed", + "connector": "{{connector_image}}", + "collections": [{{#each collection_names}}"{{this}}"{{#unless @last}}, {{/unless}}{{/each}}], + "materialization": "{{materialization_name}}", + "flow_published_at_min": "{{flow_published_at_min}}", + "flow_published_at_max": "{{flow_published_at_max}}", + "run_id": "{{run_id}}", + "auth": "{{headers.Authorization}}" +}"#, + ); + trigger + .headers + .insert("Authorization".to_string(), "Bearer my-secret".to_string()); + let compiled = CompiledTriggers::compile(vec![trigger.clone()]).unwrap(); + let context = + models::build_template_context(&TriggerVariables::placeholder(), &trigger.headers); + let rendered = compiled.render(0, &context).unwrap(); + let parsed: serde_json::Value = serde_json::from_str(&rendered).unwrap(); + insta::assert_json_snapshot!("rendered-template", parsed); + } + + #[tokio::test] + async fn webhook_retry_behavior() { + struct Case { + status: u16, + fail_times: u32, + max_attempts: u32, + expect_success: bool, + expect_calls: u32, + } + + let cases = vec![ + // Happy path — succeeds immediately. + Case { + status: 200, + fail_times: 0, + max_attempts: 3, + expect_success: true, + expect_calls: 1, + }, + // Retryable — succeeds after 2 failures. + Case { + status: 429, + fail_times: 2, + max_attempts: 3, + expect_success: true, + expect_calls: 3, + }, + Case { + status: 500, + fail_times: 2, + max_attempts: 3, + expect_success: true, + expect_calls: 3, + }, + // Retryable — exhausts retries. + Case { + status: 500, + fail_times: 9, + max_attempts: 2, + expect_success: false, + expect_calls: 2, + }, + // Non-retryable — fails immediately. + Case { + status: 400, + fail_times: 9, + max_attempts: 3, + expect_success: false, + expect_calls: 1, + }, + ]; + + for case in cases { + let call_count = Arc::new(AtomicU32::new(0)); + let count_clone = call_count.clone(); + let fail_times = case.fail_times; + let status = case.status; + + let app = axum::Router::new().route( + "/webhook", + axum::routing::post(move || { + let count = count_clone.clone(); + async move { + if count.fetch_add(1, Ordering::SeqCst) < fail_times { + axum::http::StatusCode::from_u16(status).unwrap() + } else { + axum::http::StatusCode::OK + } + } + }), + ); + + let (addr, _handle) = start_mock_server(app).await; + let mut trigger = + make_trigger_with_url(&format!("http://{addr}/webhook"), r#"{"event": "test"}"#); + trigger.max_attempts = case.max_attempts; + + let compiled = CompiledTriggers::compile(vec![trigger]).unwrap(); + let result = send_webhooks( + &compiled, + &TriggerVariables::placeholder(), + &reqwest::Client::new(), + std::time::Duration::ZERO, + ) + .await; + + assert_eq!( + result.is_ok(), + case.expect_success, + "status {status}: expected success={}, got {result:?}", + case.expect_success + ); + assert_eq!( + call_count.load(Ordering::SeqCst), + case.expect_calls, + "status {status}: expected {} calls", + case.expect_calls + ); + } + } +} diff --git a/crates/runtime-next/src/leader/mod.rs b/crates/runtime-next/src/leader/mod.rs new file mode 100644 index 00000000000..04bfe0cab67 --- /dev/null +++ b/crates/runtime-next/src/leader/mod.rs @@ -0,0 +1,13 @@ +// Protocol primitives for joining over connecting shards. +mod join; +use join::{JoinOutcome, JoinSlot, PendingJoin, validate as validate_join}; + +// gRPC service wiring. +mod service; + +// Task-specific handling. +mod materialize; +// mod derive; // TODO: implement. +// mod capture; // TODO: implement. + +pub use service::Service; diff --git a/crates/runtime-next/src/leader/service.rs b/crates/runtime-next/src/leader/service.rs new file mode 100644 index 00000000000..b92b81cd6b8 --- /dev/null +++ b/crates/runtime-next/src/leader/service.rs @@ -0,0 +1,137 @@ +use crate::proto; +use std::collections::HashMap; +use std::sync::Arc; +use tokio::sync::mpsc; + +/// Service is the implementation of the Leader gRPC service trait. +#[derive(Clone)] +pub struct Service(Arc); + +/// ServiceImpl holds shared implementation state for the Leader gRPC service. +pub struct ServiceImpl { + /// In-progress Materialize session Joins, keyed by task name. + pub(crate) materialize_joins: + std::sync::Mutex>>, + /// Service used by leader sessions to open shuffle Sessions. + pub(crate) shuffle_service: shuffle::Service, + /// Factory for building Gazette clients for publish operations. + pub(crate) publisher_factory: gazette::journal::ClientFactory, + /// Process-wide HTTP client used by the actor to deliver trigger webhooks. + pub(crate) http_client: reqwest::Client, +} + +impl Service { + pub fn new( + shuffle_service: shuffle::Service, + publisher_factory: gazette::journal::ClientFactory, + ) -> Self { + Self(Arc::new(ServiceImpl { + materialize_joins: std::sync::Mutex::new(HashMap::new()), + shuffle_service, + publisher_factory, + http_client: reqwest::Client::new(), + })) + } + + /// Wrap this service in its typed tonic server, for composition + /// with sibling services on a `tonic::transport::Server::builder()`. + pub fn into_tonic_service(self) -> proto_grpc::runtime::leader_server::LeaderServer { + proto_grpc::runtime::leader_server::LeaderServer::new(self) + .max_decoding_message_size(crate::MAX_MESSAGE_SIZE) + .max_encoding_message_size(usize::MAX) + } + + /* + pub fn spawn_derive( + &self, + request_rx: R, + ) -> mpsc::UnboundedReceiver> + where + R: futures::Stream> + Send + Unpin + 'static, + { + let service = self.clone(); + let (response_tx, response_rx) = mpsc::unbounded_channel::>(); + let error_tx = response_tx.clone(); + + tokio::spawn(async move { + if let Err(e) = serve_derive(service, request_rx, response_tx).await { + let _ = error_tx.send(Err(crate::anyhow_to_status(e))); + } + }); + response_rx + } + */ + + pub fn spawn_materialize( + &self, + request_rx: R, + ) -> mpsc::UnboundedReceiver> + where + R: futures::Stream> + Send + Unpin + 'static, + { + let service = self.clone(); + let (response_tx, response_rx) = + mpsc::unbounded_channel::>(); + let error_tx = response_tx.clone(); + + tokio::spawn(async move { + if let Err(e) = super::materialize::serve(service, request_rx, response_tx).await { + let _ = error_tx.send(Err(crate::anyhow_to_status(e))); + } + }); + response_rx + } +} + +impl std::ops::Deref for Service { + type Target = ServiceImpl; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +#[tonic::async_trait] +impl proto_grpc::runtime::leader_server::Leader for Service { + //type DeriveStream = + // tokio_stream::wrappers::UnboundedReceiverStream>; + type MaterializeStream = + tokio_stream::wrappers::UnboundedReceiverStream>; + + /* + async fn derive( + &self, + request: tonic::Request>, + ) -> tonic::Result> { + Ok(tonic::Response::new( + tokio_stream::wrappers::UnboundedReceiverStream::new( + self.spawn_derive(request.into_inner()), + ), + )) + } + */ + + async fn materialize( + &self, + request: tonic::Request>, + ) -> tonic::Result> { + Ok(tonic::Response::new( + tokio_stream::wrappers::UnboundedReceiverStream::new( + self.spawn_materialize(request.into_inner()), + ), + )) + } +} + +/* +async fn serve_derive( + _service: Service, + _request_rx: R, + _response_tx: mpsc::UnboundedSender>, +) -> anyhow::Result<()> +where + R: futures::Stream> + Send + Unpin + 'static, +{ + anyhow::bail!("Leader Derive RPC is not yet implemented") +} +*/ diff --git a/crates/runtime-next/src/lib.rs b/crates/runtime-next/src/lib.rs new file mode 100644 index 00000000000..183742ccbbe --- /dev/null +++ b/crates/runtime-next/src/lib.rs @@ -0,0 +1,264 @@ +//! `runtime-next` hosts both the controller-facing `Shard` gRPC service +//! (per-shard, in `crate::shard`) and the `Leader` gRPC service (sidecar, +//! in `crate::leader`). Each shard's `Shard` stream terminates both the +//! controller-bound and leader-bound `runtime.proto` streams, and +//! translates between them and the connector RPC. The only messages that +//! flow end-to-end unmodified are `Stop` and `CloseNow` +//! (controller → runtime-next → leader). +//! +//! "Controller" here is the peer that drives the shard's lifecycle: the +//! Go runtime in production, an in-process driver such as `flowctl +//! preview`, or a unit-test harness. This crate is agnostic to which. +//! +//! `runtime-next` will eventually replace `runtime`; during the parity +//! period both crates coexist and the controller selects between them +//! per-task at startup. `runtime-next` MUST NOT depend on `runtime` — +//! files shared between the two crates live physically in `runtime/` and +//! are pulled into `runtime-next` via `#[path]`. + +// `runtime` shares files with this crate via `#[path]`. Those shared files +// reference symbols as `runtime_next::*` so the path resolves identically +// from `runtime` (which has runtime-next as a dependency) and from this +// crate compiling them in-tree. +extern crate self as runtime_next; + +pub use ::proto_flow::runtime::Plane; // Re-export. +/// Re-export of `proto_flow::runtime` so that this crate (and its dependents) +/// can refer to protocol message types as `crate::proto::*` / +/// `runtime_next::proto::*`, avoiding the naming conflict between this crate +/// and the protobuf module. +pub use proto_flow::runtime as proto; + +mod container; +mod image_connector; +mod local_connector; +mod tokio_context; + +pub mod leader; +pub mod patches; +pub mod publish; +pub mod shard; +mod task_service; + +pub use container::flow_runtime_protocol; + +pub use leader::Service; +pub use publish::Publisher; +pub use task_service::TaskService; +pub use tokio_context::TokioContext; + +/// Maximum accepted protobuf message size on Shard / Leader streams. +pub const MAX_MESSAGE_SIZE: usize = 1 << 26; // 64MB. + +/// CHANNEL_BUFFER for connector RPC pipelines, shared with `runtime`. +pub const CHANNEL_BUFFER: usize = 16; + +/// Interval at which leader actor event loops tick, ensuring per-loop tracing +/// instrumentation fires periodically even when no other events arrive. +pub(crate) const ACTOR_TICK_INTERVAL: std::time::Duration = std::time::Duration::from_secs(60); + +/// Describes the basic type of runtime protocol. Mirrors `runtime::RuntimeProtocol` +/// so that connector image inspection (Phase F-ported `container::flow_runtime_protocol`) +/// can return a type that's local to this crate. +#[derive(Debug, Clone, Copy, PartialEq, serde::Serialize)] +#[serde(rename_all = "lowercase")] +pub enum RuntimeProtocol { + Capture, + Materialize, + Derive, +} + +impl RuntimeProtocol { + fn from_image_label(value: &str) -> Result { + match value { + "capture" => Ok(RuntimeProtocol::Capture), + "materialize" => Ok(RuntimeProtocol::Materialize), + "derive" => Ok(RuntimeProtocol::Derive), + other => Err(other), + } + } + + /// Returns the appropriate representation for storing in the control plane database. + pub fn database_string_value(&self) -> &'static str { + match self { + RuntimeProtocol::Capture => "capture", + RuntimeProtocol::Materialize => "materialization", + RuntimeProtocol::Derive => "derive", + } + } + + pub fn from_database_string_value(proto: &str) -> Option { + match proto { + "capture" => Some(RuntimeProtocol::Capture), + "materialization" => Some(RuntimeProtocol::Materialize), + "derive" => Some(RuntimeProtocol::Derive), + _ => None, + } + } +} + +// Map an anyhow::Error into a tonic::Status. +// If the error is already a Status, it's downcast. +// Otherwise, an internal error is used to wrap a formatted anyhow::Error chain. +pub fn anyhow_to_status(err: anyhow::Error) -> tonic::Status { + match err.downcast::() { + Ok(status) => status, + Err(err) => tonic::Status::unknown(format!("{err:?}")), + } +} + +// Map a tonic::Status into an anyhow::Error. +// If the status is an internal error, its message is extracted into a dynamic anyhow::Error. +// Otherwise the Status is wrapped by a dynamic anyhow::Error, and may be downcast again. +pub fn status_to_anyhow(status: tonic::Status) -> anyhow::Error { + match status.code() { + // Unwrap Unknown (only), as this code is consistently used for user-facing errors. + // Note that non-Status errors are wrapped with Unknown when mapping back into Status. + tonic::Code::Unknown => anyhow::anyhow!(status.message().to_owned()), + // For all other Status types, pass through the Status in order to preserve a + // capability to lossless-ly downcast back to the Status later. + _ => anyhow::Error::new(status), + } +} + +pub trait LogHandler: Send + Sync + Clone + 'static { + fn log(&self, log: &ops::Log); + + fn as_fn(self) -> impl Fn(&ops::Log) + Send + Sync + 'static { + move |log| self.log(log) + } +} + +impl LogHandler for T { + fn log(&self, log: &ops::Log) { + self(log) + } +} + +struct Accumulator(doc::combine::Accumulator, simd_doc::Parser); + +impl Accumulator { + pub fn new(spec: doc::combine::Spec) -> anyhow::Result { + Ok(Self( + doc::combine::Accumulator::new(spec, tempfile::tempfile()?)?, + simd_doc::Parser::new(), + )) + } + + pub fn memtable(&mut self) -> Result<&doc::combine::MemTable, doc::combine::Error> { + self.0.memtable() + } + + /// On-disk byte usage of the combiner's spill file. + pub fn combiner_byte_usage(&self) -> u64 { + self.0.ranges().last().map(|r| r.end).unwrap_or(0) + } + + /// Parse one JSON document into a HeapNode backed by the Accumulator's + /// current MemTable and Allocator. + pub fn parse_json_doc<'a>( + &'a mut self, + doc_bytes: &[u8], + ) -> anyhow::Result<( + &'a doc::combine::MemTable, + &'a doc::Allocator, + doc::HeapNode<'a>, + )> { + let memtable = self.0.memtable()?; + let alloc = memtable.alloc(); + Ok((memtable, alloc, self.1.parse_one(doc_bytes, alloc)?)) + } + + pub fn into_drainer( + self, + ) -> Result<(doc::combine::Drainer, simd_doc::Parser), doc::combine::Error> { + Ok((self.0.into_drainer()?, self.1)) + } + + pub fn from_drainer( + drainer: doc::combine::Drainer, + parser: simd_doc::Parser, + ) -> Result { + Ok(Self(drainer.into_new_accumulator()?, parser)) + } +} + +// `verify` is a convenience for building protocol error messages in a standard, +// structured way. You call `verify` to establish a `Verify` instance, which +// is then used to assert expectations over protocol requests or responses. +// If an expectation fails, it produces a suitable error message annotated +// with the originating peer. +pub fn verify<'p>(source: &'static str, expect: &'static str, peer: &'p str) -> Verify<'p> { + Verify { + source, + expect, + peer, + } +} + +pub struct Verify<'p> { + source: &'static str, + expect: &'static str, + peer: &'p str, +} + +impl<'p> Verify<'p> { + #[inline] + pub fn ok(&self, t: tonic::Result) -> anyhow::Result { + match t { + Ok(t) => Ok(t), + Err(status) => Err(self.fail_status(status)), + } + } + + #[inline] + pub fn eof(&self, t: Option>) -> anyhow::Result<()> { + match t { + None => Ok(()), + Some(Err(status)) => Err(self.fail_status(status)), + Some(Ok(t)) => Err(self.fail_msg(t)), + } + } + + #[inline] + pub fn not_eof(&self, t: Option>) -> anyhow::Result { + if let Some(t) = t { + Ok(self.ok(t)?) + } else { + Err(self.fail_err(anyhow::anyhow!("unexpected EOF"))) + } + } + + #[must_use] + #[cold] + pub fn fail_msg(&self, msg: T) -> anyhow::Error { + let Self { + source, + expect, + peer, + } = self; + + let mut t = serde_json::to_string(&msg).unwrap(); + t.truncate(4096); + + anyhow::format_err!("{source} protocol error (expected {expect}) from {peer}: {t}") + } + + #[must_use] + #[cold] + pub fn fail_err(&self, err: anyhow::Error) -> anyhow::Error { + let Self { + source, + expect, + peer, + } = self; + + err.context(format!("{source} error (expected {expect}) from {peer}")) + } + + #[must_use] + #[cold] + pub fn fail_status(&self, status: tonic::Status) -> anyhow::Error { + self.fail_err(crate::status_to_anyhow(status)) + } +} diff --git a/crates/runtime-next/src/local_connector.rs b/crates/runtime-next/src/local_connector.rs new file mode 100644 index 00000000000..5a84ab2caea --- /dev/null +++ b/crates/runtime-next/src/local_connector.rs @@ -0,0 +1,39 @@ +use futures::{Stream, StreamExt}; +use std::collections::BTreeMap; +use tokio::sync::mpsc; + +/// Serve a local connector by starting its program and adapting its stdin and stdout. +pub fn serve( + command: Vec, // Connector to run. + env: BTreeMap, // Environment variables. + log_handler: impl crate::LogHandler, // Handler for connector logs. + log_level: ops::LogLevel, // Log-level of the container, if known. + protobuf: bool, // Whether to use protobuf codec. + request_rx: mpsc::Receiver, // Caller's input request stream. +) -> anyhow::Result> + Send> +where + Request: serde::Serialize + prost::Message + Send + Sync + 'static, + Response: prost::Message + for<'de> serde::Deserialize<'de> + Default + 'static, +{ + let codec = if protobuf { + connector_init::Codec::Proto + } else { + connector_init::Codec::Json + }; + + // Invoke the underlying local connector. + let mut connector = connector_init::rpc::new_command(&command); + connector.envs(&env); + + connector.env("LOG_FORMAT", "json"); + connector.env("LOG_LEVEL", log_level.or(ops::LogLevel::Info).as_str_name()); + + let container_rx = connector_init::rpc::bidi::( + connector, + codec, + tokio_stream::wrappers::ReceiverStream::new(request_rx).map(Result::Ok), + log_handler.clone().as_fn(), + )?; + + Ok(container_rx) +} diff --git a/crates/runtime-next/src/patches.rs b/crates/runtime-next/src/patches.rs new file mode 100644 index 00000000000..56c982e476f --- /dev/null +++ b/crates/runtime-next/src/patches.rs @@ -0,0 +1,264 @@ +//! Wire format helpers for connector state patches. +//! +//! Connector state updates move through runtime-next as a framed sequence of +//! JSON merge patches. The empty byte string means "no patches"; otherwise the +//! payload is a JSON array with one patch per line. + +use anyhow::Context; +use bytes::Bytes; + +#[derive(Debug, thiserror::Error)] +#[error("{reason}")] +pub struct MalformedStatePatches { + pub reason: &'static str, +} + +/// Apply a State-Update-Wire-Format payload of JSON merge patches to a +/// JSON-encoded connector state document, returning the updated document. +/// +/// Empty `state_json` is interpreted as an empty object. +pub fn apply_state_patches(state_json: &Bytes, patches_json: &Bytes) -> anyhow::Result { + let mut doc = if state_json.is_empty() { + serde_json::Value::Object(Default::default()) + } else { + serde_json::from_slice(state_json).context("parsing connector state JSON")? + }; + + for patch in split_state_patches(patches_json)? { + let patch = serde_json::from_slice(&patch).context("parsing connector state patch")?; + json_patch::merge(&mut doc, &patch); + } + + Ok(Bytes::from(serde_json::to_vec(&doc)?)) +} + +/// Encode a connector-supplied [`proto_flow::flow::ConnectorState`] into a +/// State-Update-Wire-Format payload accepted by [`extend_state_patches`] / +/// [`split_state_patches`]. +/// +/// `None` and the proto default both yield empty bytes ("zero patches"). When +/// `merge_patch` is false the connector wants its prior state replaced — we +/// encode that as a leading `null` patch (state reset) followed by the +/// connector's `updated_json`. +pub fn encode_connector_state(state: Option) -> Bytes { + let Some(proto_flow::flow::ConnectorState { + merge_patch, + updated_json, + }) = state + else { + return Bytes::new(); + }; + + if updated_json.is_empty() { + return Bytes::new(); + } + + let mut b = Vec::with_capacity(updated_json.len() + 12); + b.push(b'['); + + if !merge_patch { + b.extend_from_slice(b"null\n,"); + } + b.extend_from_slice(&updated_json); + b.extend_from_slice(b"\n]"); + + b.into() +} + +/// Extend a State-Update-Wire-Format payload with another payload. +/// +/// Both `out` and `src` use the framed JSON-array wire form accepted by +/// [`split_state_patches`]. Empty bytes is interpreted as "zero patches". +pub fn extend_state_patches(out: &mut Vec, src: &[u8]) { + if out.is_empty() { + out.extend_from_slice(src); + } else if !src.is_empty() { + out.truncate(out.len() - 1); // Remove trailing ']'. + let src = &src[1..]; // Remove leading '['. + + out.push(b','); // Add separator. + out.extend_from_slice(src); + } +} + +/// Split a State-Update-Wire-Format payload into its individual JSON patches. +/// +/// The wire form is always framed — a JSON array with each patch on its own +/// line, prefixed by `[` (first) or `,` (subsequent) and terminated by `\n`, +/// with a closing `]`. Empty bytes is interpreted as "zero patches". +pub fn split_state_patches(payload: &Bytes) -> Result, MalformedStatePatches> { + if payload.is_empty() { + return Ok(Vec::new()); + } + if payload.first() != Some(&b'[') { + return Err(MalformedStatePatches { + reason: "expected leading `[`", + }); + } + + let mut out = Vec::new(); + let mut cursor = 0usize; + loop { + match payload.get(cursor) { + Some(b'[') | Some(b',') => cursor += 1, + Some(b']') => { + let tail = payload.len() - cursor - 1; + if tail == 0 || (tail == 1 && payload[cursor + 1] == b'\n') { + return Ok(out); + } + return Err(MalformedStatePatches { + reason: "trailing bytes after closing `]`", + }); + } + _ => { + return Err(MalformedStatePatches { + reason: "expected framing `[`, `,`, or `]`", + }); + } + } + + // Handle `[]` (empty array) and guard against a stray trailing comma. + if payload.get(cursor) == Some(&b']') { + if out.is_empty() { + let tail = payload.len() - cursor - 1; + if tail == 0 || (tail == 1 && payload[cursor + 1] == b'\n') { + return Ok(out); + } + return Err(MalformedStatePatches { + reason: "trailing bytes after closing `]`", + }); + } + return Err(MalformedStatePatches { + reason: "trailing comma before `]`", + }); + } + + let newline = + payload[cursor..] + .iter() + .position(|b| *b == b'\n') + .ok_or(MalformedStatePatches { + reason: "missing trailing newline", + })?; + let end = cursor + newline; + out.push(payload.slice(cursor..end)); + cursor = end + 1; + } +} + +#[cfg(test)] +mod test { + use super::*; + use proto_flow::flow; + + #[test] + fn split_state_patches_cases() { + let ok_cases: &[(&[u8], &[&[u8]])] = &[ + // Empty bytes tolerated (proto default) as "zero patches". + (b"", &[]), + // Canonical zero-patches wire form. + (b"[]", &[]), + (b"[]\n", &[]), + (b"[{\"a\":1}\n]", &[b"{\"a\":1}"]), + ( + b"[{\"a\":1}\n,{\"b\":2}\n,{\"c\":3}\n]", + &[b"{\"a\":1}", b"{\"b\":2}", b"{\"c\":3}"], + ), + // Trailing newline after `]` is permitted. + (b"[{\"a\":1}\n]\n", &[b"{\"a\":1}"]), + ]; + for (input, want) in ok_cases { + let got = split_state_patches(&Bytes::copy_from_slice(input)).unwrap(); + let got: Vec<&[u8]> = got.iter().map(|b| b.as_ref()).collect(); + assert_eq!(got, *want, "input {:?}", String::from_utf8_lossy(input)); + } + + let err_cases: &[&[u8]] = &[ + b"{\"a\":1}", // bare single-patch form is no longer valid + b"[{\"a\":1}]", // missing trailing newline + b"[{\"a\":1}\n] extra", // junk after closing + b"[{\"a\":1}\n{\"b\":2}\n]", // missing inter-entry comma + b"[{\"a\":1}\n,]", // trailing comma before `]` + ]; + for input in err_cases { + split_state_patches(&Bytes::copy_from_slice(input)).unwrap_err(); + } + } + + #[test] + fn extend_state_patches_cases() { + let mut out = Vec::new(); + extend_state_patches(&mut out, b""); + assert!(out.is_empty()); + + extend_state_patches(&mut out, b"[{\"a\":1}\n]"); + assert_eq!(out.as_slice(), b"[{\"a\":1}\n]"); + + extend_state_patches(&mut out, b""); + assert_eq!(out.as_slice(), b"[{\"a\":1}\n]"); + + extend_state_patches(&mut out, b"[{\"b\":2}\n,{\"c\":null}\n]"); + assert_eq!(out.as_slice(), b"[{\"a\":1}\n,{\"b\":2}\n,{\"c\":null}\n]"); + + let decoded = split_state_patches(&Bytes::from(out)).unwrap(); + let decoded: Vec<&[u8]> = decoded.iter().map(|b| b.as_ref()).collect(); + assert_eq!( + decoded, + vec![ + b"{\"a\":1}".as_slice(), + b"{\"b\":2}".as_slice(), + b"{\"c\":null}".as_slice() + ] + ); + } + + #[test] + fn encode_connector_state_cases() { + let cases = [ + (None, b"".as_slice(), Vec::<&[u8]>::new()), + ( + Some(flow::ConnectorState::default()), + b"", + Vec::<&[u8]>::new(), + ), + ( + Some(flow::ConnectorState { + updated_json: Bytes::from_static(br#"{"a":1}"#), + merge_patch: true, + }), + b"[{\"a\":1}\n]", + vec![br#"{"a":1}"#.as_slice()], + ), + ( + Some(flow::ConnectorState { + updated_json: Bytes::from_static(br#"{"a":1}"#), + merge_patch: false, + }), + b"[null\n,{\"a\":1}\n]", + vec![b"null".as_slice(), br#"{"a":1}"#.as_slice()], + ), + ]; + + for (state, want_encoded, want_split) in cases { + let encoded = encode_connector_state(state); + assert_eq!(encoded.as_ref(), want_encoded); + + let split = split_state_patches(&encoded).unwrap(); + let split: Vec<&[u8]> = split.iter().map(|b| b.as_ref()).collect(); + assert_eq!(split, want_split); + } + } + + #[test] + fn replacement_state_resets_prior_state() { + let patches = encode_connector_state(Some(flow::ConnectorState { + updated_json: Bytes::from_static(br#"{"kept":1}"#), + merge_patch: false, + })); + let updated = + apply_state_patches(&Bytes::from_static(br#"{"dropped":true}"#), &patches).unwrap(); + + let updated: serde_json::Value = serde_json::from_slice(&updated).unwrap(); + assert_eq!(updated, serde_json::json!({"kept": 1})); + } +} diff --git a/crates/runtime-next/src/publish.rs b/crates/runtime-next/src/publish.rs new file mode 100644 index 00000000000..4ef8a81b4c0 --- /dev/null +++ b/crates/runtime-next/src/publish.rs @@ -0,0 +1,147 @@ +//! Publishing surface used by leader actors. +//! +//! `Publisher` is the unified entry point. Two variants: +//! +//! - `Publisher::Real` wraps a real `publisher::Publisher` and performs +//! Gazette journal IO (stats / logs / ACK intents / future capture & +//! derive collection writes). +//! - `Publisher::Preview` performs no journal IO. Stats and log documents +//! are emitted as `tracing::info!` events. Output documents +//! (capture / derive — TODO when those task types land) will +//! print to stdout in the `["{collection}",{...doc...}]` format used by +//! `flowctl preview`. +//! +//! Construction is decided in `startup::run` based on the presence of +//! ops_logs / ops_stats specs in `L:Open`: present ⇒ `Real`, absent ⇒ +//! `Preview`. The leader actor parks the `Publisher` across IO futures. + +use bytes::Bytes; +use proto_flow::flow; +use proto_gazette::uuid; +use std::collections::BTreeMap; + +/// Publishing entity used by leader sessions and runtime-next shards. +/// `crate::publish::Publisher` is the operative publisher from the +/// perspective of the `leader` and `runtime-next` crates; the inner +/// `publisher::Publisher` is an implementation detail of the `Real` +/// variant. +/// +/// Methods mirror the subset of `publisher::Publisher` the leader actor +/// uses, plus `publish_stats` which factors out the actor's +/// stats-enqueue-then-flush idiom. The `Real` arm delegates to +/// `publisher::Publisher`; the `Preview` arm performs no IO. +pub enum Publisher { + Real(publisher::Publisher), + Preview, +} + +impl Publisher { + /// Build a real `Publisher` backed by a `publisher::Publisher` for + /// `ops_logs_spec` / `ops_stats_spec` journals plus any additional + /// supplied collection specs. + pub fn new_real<'a, I>( + authz_subject: String, + client_factory: &gazette::journal::ClientFactory, + ops_stats_journal: &str, + ops_stats_spec: &flow::CollectionSpec, + collection_specs: I, + ) -> anyhow::Result + where + I: IntoIterator, + { + let mut bindings = Vec::new(); + + bindings.push(publisher::Binding::from_collection_spec( + ops_stats_spec, + Some(ops_stats_journal), + )?); + + for spec in collection_specs { + bindings.push(publisher::Binding::from_collection_spec(spec, None)?); + } + + let mut producer: [u8; 6] = rand::random(); + producer[0] |= 0x01; // Set multicast bit. + let producer = uuid::Producer::from_bytes(producer); + + let mut publisher = publisher::Publisher::new( + authz_subject, + bindings, + client_factory.clone(), + producer, + uuid::Clock::zero(), + ); + publisher.update_clock(); + + Ok(Self::Real(publisher)) + } + + /// Build a preview `Mode` that performs no journal IO. Stats are emitted + /// to `tracing::debug!`; ACK intents are implicitly empty. + pub fn new_preview() -> Self { + Self::Preview + } + + /// Advance the publisher's clock to the current wall-clock time. + /// No-op in preview mode. + pub fn update_clock(&mut self) { + match self { + Self::Real(p) => p.update_clock(), + Self::Preview => {} + } + } + + /// Enqueue and flush a single stats document as a `CONTINUE_TXN`. + /// + /// This consolidates the leader actor's prior "enqueue stats, then + /// flush" pattern into one method so the parking pattern stays + /// symmetric across `Real` and `Preview` arms. + pub async fn publish_stats(&mut self, mut stats: ops::proto::Stats) -> tonic::Result<()> { + match self { + Self::Real(p) => { + p.enqueue( + |uuid| { + // Binding index 0 is ops_stats_spec. + stats.meta.as_mut().unwrap().uuid = uuid.to_string(); + (0, serde_json::to_value(&stats).unwrap()) + }, + uuid::Flags::CONTINUE_TXN, + ) + .await?; + p.flush().await + } + Self::Preview => { + tracing::info!(stats = ?ops::DebugJson(stats), "transaction stats"); + Ok(()) + } + } + } + + /// Snapshot this producer's contribution to the current transaction's + /// ACK intents. In preview mode, returns an empty list — no real + /// publishes happened, so there are no commit positions to encode. + pub fn commit_intents(&mut self) -> Option<(uuid::Producer, uuid::Clock, Vec)> { + match self { + Self::Real(p) => Some(p.commit_intents()), + Self::Preview => None, + } + } + + /// Write per-journal ACK intent documents to their journals. + /// No-op in preview mode (intents are necessarily empty). + pub async fn write_intents( + &mut self, + journal_intents: BTreeMap, + ) -> tonic::Result<()> { + match self { + Self::Real(p) => p.write_intents(journal_intents).await, + Self::Preview => { + debug_assert!( + journal_intents.is_empty(), + "Publisher::Preview received non-empty ACK intents", + ); + Ok(()) + } + } + } +} diff --git a/crates/runtime-next/src/shard/materialize/actor.rs b/crates/runtime-next/src/shard/materialize/actor.rs new file mode 100644 index 00000000000..8ef18bd9c60 --- /dev/null +++ b/crates/runtime-next/src/shard/materialize/actor.rs @@ -0,0 +1,807 @@ +use super::{Binding, LoadKeys, drain, scan}; +use crate::{patches, proto}; +use anyhow::Context; +use bytes::Bytes; +use futures::{FutureExt, StreamExt, future, future::BoxFuture}; +use proto_flow::materialize; +use std::collections::{HashMap, VecDeque}; +use tokio::sync::mpsc; + +use crate::proto::materialize::flushed::Binding as FlushedBinding; + +/// Phase of the shard transaction which are incrementally driven by Actor. +pub(super) enum Phase { + Idle { + accumulator: crate::Accumulator, + shuffle_reader: shuffle::log::Reader, + shuffle_remainders: VecDeque, + }, + Scanning(scan::Scanner), + Draining(drain::Drainer), +} + +/// Shard-side materialization reactor for one joined leader session. +pub(super) struct Actor { + // Task binding specifications. + bindings: Vec, + // FIFO of outbound connector requests, drained head-first into + // `connector_tx` as channel capacity permits. + connector_pending: Vec, + // Bounded channel out to the connector subprocess. + connector_tx: mpsc::Sender, + // RocksDB and binding state keys, when a Persist is not in flight. + db: Option<(crate::shard::RocksDB, Vec)>, + // RocksDB future when a Persist is in flight. + db_persist_fut: Option< + BoxFuture< + 'static, + anyhow::Result<((crate::shard::RocksDB, Vec), proto::Persisted)>, + >, + >, + // When true, don't suppress C:Load for keys less-than `max_keys`. + disable_load_optimization: bool, + // Aggregate active bindings of a pending Flushed response. + flushed: HashMap, + // Channel for sending to the leader. + leader_tx: mpsc::UnboundedSender, + // Keys for which we've sent C:Load in the current transaction. + load_keys: LoadKeys, + // Running maximum observed key (committed, current), one entry per binding. + // The current key ratchets up during frontier scans, and is rotated into + // committed at transaction close (and not before; this lets us filter keys + // above "committed" but below "current" during scans). + max_keys: Vec<(Bytes, Bytes)>, +} + +impl Actor { + pub fn new( + bindings: Vec, + binding_state_keys: Vec, + connector_tx: mpsc::Sender, + db: crate::shard::RocksDB, + disable_load_optimization: bool, + leader_tx: mpsc::UnboundedSender, + max_keys: Vec<(Bytes, Bytes)>, + ) -> Self { + Self { + bindings, + connector_pending: Vec::new(), + connector_tx, + db: Some((db, binding_state_keys)), + db_persist_fut: None, + disable_load_optimization, + flushed: HashMap::new(), + leader_tx, + load_keys: Default::default(), + max_keys, + } + } + + #[tracing::instrument(level = "debug", err(Debug, level = "warn"), skip_all)] + pub async fn serve( + mut self, + accumulator: crate::Accumulator, + connector_rx: &mut C, + controller_rx: &mut R, + leader_rx: &mut L, + shuffle_reader: shuffle::log::Reader, + ) -> anyhow::Result + where + R: futures::Stream> + Send + Unpin + 'static, + C: futures::Stream> + Send + Unpin + 'static, + L: futures::Stream> + Send + Unpin + 'static, + { + let mut phase = Phase::Idle { + accumulator, + shuffle_reader, + shuffle_remainders: VecDeque::new(), + }; + let mut loop_count: u64 = 0; + + let mut ticker = tokio::time::interval(crate::ACTOR_TICK_INTERVAL); + ticker.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + + loop { + loop_count += 1; + + let phase_kind = match &phase { + Phase::Idle { .. } => "idle", + Phase::Scanning(_) => "scanning", + Phase::Draining(_) => "draining", + }; + tracing::trace!( + loop_count, + connector_pending_len = self.connector_pending.len(), + loaded_bindings = self.flushed.len(), + persist_in_flight = self.db_persist_fut.is_some(), + phase = phase_kind, + "shard materialize Actor::serve iteration" + ); + + // Perform non-blocking sends of pending connector requests. + let wake_connector_tx = self.try_connector_tx(); + + // Attempt to drive `phase` forward. On success we loop to + // immediately start IO, falling through to the select below only + // if we can't immediately send to `connector_tx`. + if !self.connector_pending.is_empty() { + // Channel is stuffed -- don't allow further requests to queue. + } else if let Phase::Scanning(mut scanner) = phase { + if scanner.step( + &self.bindings, + &mut self.load_keys, + &mut self.max_keys, + self.disable_load_optimization, + &mut self.connector_pending, + )? { + phase = Phase::Scanning(scanner); + } else { + let (accumulator, shuffle_reader, shuffle_remainders, active) = + scanner.into_parts(); + + _ = self.leader_tx.send(proto::Materialize { + loaded: Some(proto::materialize::Loaded { + bindings: active.into_values().collect(), + combiner_usage_bytes: accumulator.combiner_byte_usage(), + }), + ..Default::default() + }); + phase = Phase::Idle { + accumulator, + shuffle_reader, + shuffle_remainders, + }; + } + continue; + } else if let Phase::Draining(mut drainer) = phase { + if let Some(request) = drainer.step(&self.bindings)? { + self.connector_pending.push(request); + phase = Phase::Draining(drainer); + } else { + let (accumulator, shuffle_reader, shuffle_remainders, active) = + drainer.into_parts()?; + + _ = self.leader_tx.send(proto::Materialize { + stored: Some(proto::materialize::Stored { bindings: active }), + ..Default::default() + }); + phase = Phase::Idle { + accumulator, + shuffle_reader, + shuffle_remainders, + }; + } + continue; + } + + tokio::select! { + biased; + + // Prioritize moving connector messages (high volume). + msg = connector_rx.next() => { + self.on_connector_response(&mut phase, msg)?; + } + // Next, a leader message. + msg = leader_rx.next() => { + let (next, stopped) = self.on_leader_message(phase, msg)?; + phase = next; + + if stopped { + break; + } + } + // Next, a controller message. + msg = controller_rx.next() => { + self.on_controller_request(msg)?; + } + // Next, a Persist completion. + result = maybe_fut(&mut self.db_persist_fut) => { + let (db, persisted) = result?; + self.db = Some(db); + + _ = self.leader_tx.send(proto::Materialize { + persisted: Some(persisted), + ..Default::default() + }); + } + // Next, wait for capacity to send to the connector. + true = wake_connector_tx => {} + // Periodic tick ensures the iteration trace fires even when otherwise idle. + _ = ticker.tick() => {} + } + } + + // After Stopped, the leader's stream must EOF. + let verify = crate::verify("Materialize", "leader EOF after Stopped", "leader"); + verify.eof(leader_rx.next().await)?; + + // Leader-protocol invariant: the leader does not send L:Stopped while a + // Persist is outstanding — every L:Persist is acknowledged with + // L:Persisted before L:Stopped. + let Some((db, _)) = self.db.take() else { + anyhow::bail!("leader Stopped while a Persist is in flight"); + }; + + Ok(db) + } + + fn try_connector_tx(&mut self) -> impl Future + 'static { + // Closure for mapping an OwnedPermit Result to Ok (our "poll again" signal). + let ok = |result: Result<_, _>| result.is_ok(); + // Future which represents an absence of an awake signal. + let idle = future::Either::Right(std::future::ready(false)); + + if self.connector_pending.is_empty() { + return idle; + } else if let Ok(permits) = self + .connector_tx + .try_reserve_many(self.connector_pending.len()) + { + for (request, permit) in self.connector_pending.drain(..).zip(permits) { + permit.send(request); + } + return idle; + } + + // We have only partial capacity to send. In this uncommon case, we pay + // for relocating the tail of `connector_pending`. Note the sender may + // race sends, so account for capacity > connector_pending having opened up. + let n = self + .connector_tx + .capacity() + .min(self.connector_pending.len()); + let permits = self + .connector_tx + .try_reserve_many(n) + .expect("capacity was just observed and we are the sole sender"); + + for (request, permit) in self.connector_pending.drain(..n).zip(permits) { + permit.send(request); + } + // Wake when more capacity becomes available. + future::Either::Left(self.connector_tx.clone().reserve_owned().map(ok)) + } + + fn on_leader_message( + &mut self, + phase: Phase, + msg: Option>, + ) -> anyhow::Result<(Phase, bool)> { + let verify = crate::verify("Materialize", "leader message", "leader"); + let msg = verify.not_eof(msg)?; + + if let Some(proto::Stopped {}) = msg.stopped { + return Ok((phase, true)); + } else if let Some(proto::materialize::Load { + frontier: Some(proto), + }) = msg.load + { + let frontier = + shuffle::Frontier::decode(proto).context("invalid Frontier on L:Load")?; + + let Phase::Idle { + accumulator, + shuffle_reader, + shuffle_remainders, + } = phase + else { + anyhow::bail!("L:Load received while actor is not idle"); + }; + + let scanner = + scan::Scanner::new(accumulator, frontier, shuffle_reader, shuffle_remainders)?; + return Ok((Phase::Scanning(scanner), false)); + } else if let Some(proto::materialize::Flush { + connector_patches_json, + }) = msg.flush + { + self.connector_pending.push(materialize::Request { + flush: Some(materialize::request::Flush { + connector_state_patches_json: connector_patches_json, + }), + ..Default::default() + }); + } else if let Some(proto::materialize::Store {}) = msg.store { + let Phase::Idle { + accumulator, + shuffle_reader, + shuffle_remainders, + } = phase + else { + anyhow::bail!("L:Store received while actor is not idle"); + }; + + // Clear and partially release `load_keys` memory before the next scan. + let load_keys_len = self.load_keys.len(); + self.load_keys.clear(); + self.load_keys.shrink_to(load_keys_len / 2); + + // Rotate a changed `next_max` into `prev_max` for each of `max_keys`. + for (prev_max, next_max) in self.max_keys.iter_mut() { + if !next_max.is_empty() { + *prev_max = std::mem::take(next_max); + } + } + + let drainer = drain::Drainer::new(accumulator, shuffle_reader, shuffle_remainders)?; + return Ok((Phase::Draining(drainer), false)); + } else if let Some(proto::materialize::StartCommit { + connector_checkpoint, + connector_patches_json, + }) = msg.start_commit + { + self.connector_pending.push(materialize::Request { + start_commit: Some(materialize::request::StartCommit { + runtime_checkpoint: connector_checkpoint, + connector_state_patches_json: connector_patches_json, + }), + ..Default::default() + }); + } else if let Some(proto::materialize::Acknowledge { + connector_patches_json, + }) = msg.acknowledge + { + self.connector_pending.push(materialize::Request { + acknowledge: Some(materialize::request::Acknowledge { + connector_state_patches_json: connector_patches_json, + }), + ..Default::default() + }); + } else if let Some(persist) = msg.persist { + let nonce = persist.nonce; + + let (db, binding_state_keys) = self + .db + .take() + .context("received L:Persist while a Persist is already in flight")?; + + self.db_persist_fut = Some( + async move { + let db = db.persist(&persist, &binding_state_keys).await?; + Ok(((db, binding_state_keys), proto::Persisted { nonce })) + } + .boxed(), + ); + } else { + return Err(verify.fail_msg(msg)); + } + + Ok((phase, false)) + } + + fn on_connector_response( + &mut self, + phase: &mut Phase, + resp: Option>, + ) -> anyhow::Result<()> { + let verify = crate::verify("Materialize", "connector response", "connector"); + let resp = verify.not_eof(resp)?; + + if let Some(materialize::response::Loaded { binding, doc_json }) = resp.loaded { + let active = self + .flushed + .entry(binding) + .or_insert_with(|| FlushedBinding { + index: binding, + ..Default::default() + }); + active.loaded_docs_total += 1; + active.loaded_bytes_total += doc_json.len() as u64; + + // C:Loaded responses arrive at the connector's pace, which may + // straddle phase transitions: + // * Phase::Scanning — a common case, mid-scan response. + // * Phase::Idle — the connector is still draining responses for + // prior C:Load requests after the Scanner finished and we sent + // L:Loaded back to the leader (often triggered by C:Flush). + // The Idle and Scanning phases share the same accumulator, + // so we add the doc to it identically. + // * Phase::Draining — must not happen: the leader sends L:Store + // only after every shard's L:Flushed, which we send only after + // C:Flushed, which ends the connector's Loaded phase. + let accumulator = match phase { + Phase::Idle { accumulator, .. } => accumulator, + Phase::Scanning(scanner) => scanner.accumulator(), + Phase::Draining(_) => { + anyhow::bail!("C:Loaded received while actor is draining") + } + }; + let binding_index = binding as usize; + let binding_spec = self + .bindings + .get(binding_index) + .ok_or_else(|| anyhow::anyhow!("Loaded binding {binding_index} out of range"))?; + + let (memtable, _alloc, doc) = + accumulator.parse_json_doc(&doc_json).with_context(|| { + format!("parsing loaded doc for {}", binding_spec.collection_name) + })?; + memtable.add(binding_index as u16, doc, true)?; + } else if let Some(materialize::response::Flushed { state }) = resp.flushed { + let bindings = std::mem::take(&mut self.flushed).into_values().collect(); + _ = self.leader_tx.send(proto::Materialize { + flushed: Some(proto::materialize::Flushed { + bindings, + connector_patches_json: patches::encode_connector_state(state), + }), + ..Default::default() + }); + } else if let Some(materialize::response::StartedCommit { state }) = resp.started_commit { + _ = self.leader_tx.send(proto::Materialize { + started_commit: Some(proto::materialize::StartedCommit { + connector_patches_json: patches::encode_connector_state(state), + }), + ..Default::default() + }); + } else if let Some(materialize::response::Acknowledged { state }) = resp.acknowledged { + _ = self.leader_tx.send(proto::Materialize { + acknowledged: Some(proto::materialize::Acknowledged { + connector_patches_json: patches::encode_connector_state(state), + }), + ..Default::default() + }); + } else { + return Err(verify.fail_msg(resp)); + } + + Ok(()) + } + + fn on_controller_request( + &mut self, + msg: Option>, + ) -> anyhow::Result<()> { + let verify = crate::verify("Materialize", "Stop or CloseNow", "controller"); + let msg = verify.not_eof(msg)?; + + if matches!(msg.stop, Some(proto::Stop {})) { + _ = self.leader_tx.send(proto::Materialize { + stop: Some(proto::Stop {}), + ..Default::default() + }); + } else if matches!(msg.close_now, Some(proto::CloseNow {})) { + _ = self.leader_tx.send(proto::Materialize { + close_now: Some(proto::CloseNow {}), + ..Default::default() + }); + } else { + return Err(verify.fail_msg(msg)); + } + Ok(()) + } +} + +async fn maybe_fut(opt: &mut Option>) -> T { + match opt.as_mut() { + Some(fut) => { + let result = fut.await; + *opt = None; + result + } + None => std::future::pending().await, + } +} + +#[cfg(test)] +mod tests { + use super::*; + use proto_flow::flow; + use proto_flow::materialize::response; + use tokio_stream::wrappers::{ReceiverStream, UnboundedReceiverStream}; + + fn make_idle_phase() -> Phase { + let accumulator = + crate::Accumulator::new(super::super::task::combine_spec(&[]).unwrap()).unwrap(); + let shuffle_reader = shuffle::log::Reader::new(std::path::Path::new("/dev/null"), 0); + Phase::Idle { + accumulator, + shuffle_reader, + shuffle_remainders: VecDeque::new(), + } + } + + fn make_actor() -> ( + Actor, + mpsc::UnboundedReceiver, + mpsc::Receiver, + ) { + let (leader_tx, leader_rx) = mpsc::unbounded_channel(); + let (connector_tx, connector_rx) = mpsc::channel(8); + + ( + Actor { + bindings: Vec::new(), + connector_pending: Vec::new(), + connector_tx, + db: None, + db_persist_fut: None, + disable_load_optimization: false, + leader_tx, + load_keys: Default::default(), + flushed: HashMap::new(), + max_keys: Vec::new(), + }, + leader_rx, + connector_rx, + ) + } + + #[tokio::test] + async fn acknowledge_round_trip_forwards_patches() { + let (mut actor, mut leader_rx, mut connector_rx) = make_actor(); + let patches = Bytes::from_static(br#"[{"ok":true}]"#); + + let (_phase, _stop) = actor + .on_leader_message( + make_idle_phase(), + Some(Ok(proto::Materialize { + acknowledge: Some(proto::materialize::Acknowledge { + connector_patches_json: patches.clone(), + }), + ..Default::default() + })), + ) + .unwrap(); + + // Drive the non-blocking send of the pending connector request. + _ = actor.try_connector_tx(); + + let request = connector_rx.recv().await.unwrap(); + assert_eq!( + request.acknowledge.unwrap().connector_state_patches_json, + patches + ); + + let mut phase = make_idle_phase(); + actor + .on_connector_response( + &mut phase, + Some(Ok(materialize::Response { + acknowledged: Some(response::Acknowledged { + state: Some(flow::ConnectorState { + updated_json: Bytes::from_static(br#"{"done":true}"#), + merge_patch: true, + }), + }), + ..Default::default() + })), + ) + .unwrap(); + + let response = leader_rx.recv().await.unwrap(); + assert_eq!( + response.acknowledged.unwrap().connector_patches_json, + Bytes::from_static( + br#"[{"done":true} +]"# + ) + ); + } + + /// Drive `Actor::serve` end-to-end with mpsc channels acting as the + /// connector, leader, and controller, walking a transaction lifecycle: + /// Acknowledge → Flush → Store (empty drain) → StartCommit → Persist → + /// controller Stop → controller CloseNow → leader Stopped. Each step + /// asserts that the actor correctly translates leader commands into + /// connector requests, fans connector responses back to the leader, + /// forwards controller signals to the leader, and persists state to + /// RocksDB. + #[tokio::test] + async fn full_lifecycle_round_trip() { + // Actor → connector requests; the test reads as a mock connector. + let (actor_to_conn_tx, mut actor_to_conn_rx) = mpsc::channel::(8); + // Mock connector → actor responses. + let (conn_to_actor_tx, conn_to_actor_rx) = + mpsc::channel::>(8); + // Actor → leader; the test reads as a mock leader. + let (actor_to_leader_tx, mut actor_to_leader_rx) = + mpsc::unbounded_channel::(); + // Mock leader → actor commands. + let (leader_to_actor_tx, leader_to_actor_rx) = + mpsc::unbounded_channel::>(); + // Controller → actor; used to drive Stop + CloseNow forwarding below. + let (controller_to_actor_tx, controller_to_actor_rx) = + mpsc::unbounded_channel::>(); + + let conn_stream = ReceiverStream::new(conn_to_actor_rx); + let leader_stream = UnboundedReceiverStream::new(leader_to_actor_rx); + let controller_stream = UnboundedReceiverStream::new(controller_to_actor_rx); + + // A real RocksDB lets the Persist step exercise the full code path. + let db = crate::shard::RocksDB::open(None).await.unwrap(); + + let actor = Actor { + bindings: Vec::new(), + connector_pending: Vec::new(), + connector_tx: actor_to_conn_tx, + db: Some((db, Vec::new())), + db_persist_fut: None, + disable_load_optimization: false, + leader_tx: actor_to_leader_tx, + load_keys: Default::default(), + flushed: HashMap::new(), + max_keys: Vec::new(), + }; + + let accumulator = + crate::Accumulator::new(super::super::task::combine_spec(&[]).unwrap()).unwrap(); + let shuffle_reader = shuffle::log::Reader::new(std::path::Path::new("/dev/null"), 0); + + let serve_handle = tokio::spawn(async move { + let mut conn_stream = conn_stream; + let mut leader_stream = leader_stream; + let mut controller_stream = controller_stream; + actor + .serve( + accumulator, + &mut conn_stream, + &mut controller_stream, + &mut leader_stream, + shuffle_reader, + ) + .await + }); + + // 1) L:Acknowledge → C:Acknowledge → C:Acknowledged → L:Acknowledged. + leader_to_actor_tx + .send(Ok(proto::Materialize { + acknowledge: Some(proto::materialize::Acknowledge { + connector_patches_json: Bytes::from_static(br#"[{"ack":1}]"#), + }), + ..Default::default() + })) + .unwrap(); + + let req = actor_to_conn_rx.recv().await.unwrap(); + assert_eq!( + req.acknowledge.unwrap().connector_state_patches_json, + Bytes::from_static(br#"[{"ack":1}]"#), + ); + + conn_to_actor_tx + .send(Ok(materialize::Response { + acknowledged: Some(response::Acknowledged { + state: Some(flow::ConnectorState { + updated_json: Bytes::from_static(br#"{"ack_done":true}"#), + merge_patch: false, + }), + }), + ..Default::default() + })) + .await + .unwrap(); + + let resp = actor_to_leader_rx.recv().await.unwrap(); + assert!(resp.acknowledged.is_some()); + + // 2) L:Flush → C:Flush → C:Flushed → L:Flushed. + leader_to_actor_tx + .send(Ok(proto::Materialize { + flush: Some(proto::materialize::Flush { + connector_patches_json: Bytes::from_static(br#"[{"f":1}]"#), + }), + ..Default::default() + })) + .unwrap(); + + let req = actor_to_conn_rx.recv().await.unwrap(); + assert_eq!( + req.flush.unwrap().connector_state_patches_json, + Bytes::from_static(br#"[{"f":1}]"#), + ); + + conn_to_actor_tx + .send(Ok(materialize::Response { + flushed: Some(response::Flushed { + state: Some(flow::ConnectorState { + updated_json: Bytes::from_static(br#"{"flushed":true}"#), + merge_patch: false, + }), + }), + ..Default::default() + })) + .await + .unwrap(); + + let resp = actor_to_leader_rx.recv().await.unwrap(); + let flushed = resp.flushed.unwrap(); + assert!(flushed.bindings.is_empty()); + + // 3) L:Store → empty drain → L:Stored. + // With no bindings and an empty memtable, the drainer immediately + // completes and the actor reports zero stored bindings. + leader_to_actor_tx + .send(Ok(proto::Materialize { + store: Some(proto::materialize::Store {}), + ..Default::default() + })) + .unwrap(); + + let resp = actor_to_leader_rx.recv().await.unwrap(); + let stored = resp.stored.unwrap(); + assert!(stored.bindings.is_empty()); + + // 4) L:StartCommit → C:StartCommit → C:StartedCommit → L:StartedCommit. + leader_to_actor_tx + .send(Ok(proto::Materialize { + start_commit: Some(proto::materialize::StartCommit { + connector_checkpoint: Some(proto_gazette::consumer::Checkpoint::default()), + connector_patches_json: Bytes::from_static(br#"[{"sc":1}]"#), + }), + ..Default::default() + })) + .unwrap(); + + let req = actor_to_conn_rx.recv().await.unwrap(); + let sc = req.start_commit.unwrap(); + assert_eq!( + sc.connector_state_patches_json, + Bytes::from_static(br#"[{"sc":1}]"#), + ); + + conn_to_actor_tx + .send(Ok(materialize::Response { + started_commit: Some(response::StartedCommit { + state: Some(flow::ConnectorState { + updated_json: Bytes::from_static(br#"{"sc_done":true}"#), + merge_patch: false, + }), + }), + ..Default::default() + })) + .await + .unwrap(); + + let resp = actor_to_leader_rx.recv().await.unwrap(); + assert!(resp.started_commit.is_some()); + + // 5) L:Persist → RocksDB write → L:Persisted echoes nonce. + leader_to_actor_tx + .send(Ok(proto::Materialize { + persist: Some(proto::Persist { + nonce: 42, + last_applied: Bytes::from_static(b"persisted-spec-bytes"), + ..Default::default() + }), + ..Default::default() + })) + .unwrap(); + + let resp = actor_to_leader_rx.recv().await.unwrap(); + assert_eq!(resp.persisted.unwrap().nonce, 42); + + // 6) Controller Stop + CloseNow → forwarded to the leader. + controller_to_actor_tx + .send(Ok(proto::Materialize { + stop: Some(proto::Stop {}), + ..Default::default() + })) + .unwrap(); + let resp = actor_to_leader_rx.recv().await.unwrap(); + assert!(resp.stop.is_some()); + + controller_to_actor_tx + .send(Ok(proto::Materialize { + close_now: Some(proto::CloseNow {}), + ..Default::default() + })) + .unwrap(); + let resp = actor_to_leader_rx.recv().await.unwrap(); + assert!(resp.close_now.is_some()); + + // 7) L:Stopped + leader EOF → serve completes, returning the DB. + leader_to_actor_tx + .send(Ok(proto::Materialize { + stopped: Some(proto::Stopped {}), + ..Default::default() + })) + .unwrap(); + std::mem::drop(leader_to_actor_tx); + + let db = serve_handle.await.unwrap().unwrap(); + + // Confirm the Persist round-tripped: scan back the last_applied bytes. + let (_db, recover) = db.scan(Vec::new()).await.unwrap(); + assert_eq!(recover.last_applied.as_ref(), b"persisted-spec-bytes"); + } +} diff --git a/crates/runtime-next/src/shard/materialize/connector.rs b/crates/runtime-next/src/shard/materialize/connector.rs new file mode 100644 index 00000000000..8e82716c109 --- /dev/null +++ b/crates/runtime-next/src/shard/materialize/connector.rs @@ -0,0 +1,211 @@ +use anyhow::Context; +use futures::{FutureExt, StreamExt, TryStreamExt, stream::BoxStream}; +use proto_flow::{flow, materialize}; +use tokio::sync::mpsc; +use tokio_stream::wrappers::ReceiverStream; +use unseal; +use zeroize::Zeroize; + +// Start a materialization connector as indicated by the `initial` Request. +// Returns a pair of Streams for sending Requests and receiving Responses, +// plus OpenExtras with decrypted trigger configs and connector metadata. +pub async fn start( + service: &crate::shard::Service, + mut initial: materialize::Request, +) -> anyhow::Result<( + mpsc::Sender, + BoxStream<'static, tonic::Result>, + Option, +)> { + let log_level = initial.get_internal()?.log_level(); + let (endpoint, config_json, connector_type, catalog_name) = extract_endpoint(&mut initial)?; + let (connector_tx, connector_rx) = mpsc::channel(crate::CHANNEL_BUFFER); + + fn start_rpc( + channel: tonic::transport::Channel, + rx: mpsc::Receiver, + ) -> crate::image_connector::StartRpcFuture { + async move { + proto_grpc::materialize::connector_client::ConnectorClient::new(channel) + .max_decoding_message_size(crate::MAX_MESSAGE_SIZE) + .max_encoding_message_size(usize::MAX) + .materialize(ReceiverStream::new(rx)) + .await + } + .boxed() + } + + let (mut connector_rx, container) = match endpoint { + models::MaterializationEndpoint::Connector(models::ConnectorConfig { + image, + config: sealed_config, + }) => { + *config_json = unseal::decrypt_sops(&sealed_config).await?.into(); + + let (rx, container) = crate::image_connector::serve( + image.clone(), + service.log_handler.clone(), + log_level, + &service.container_network, + connector_rx, + start_rpc, + &service.task_name, + ops::TaskType::Materialization, + service.plane, + ) + .await?; + + (rx.boxed(), Some(container)) + } + models::MaterializationEndpoint::Local(_) + if !matches!(service.plane, crate::Plane::Local) => + { + return Err(tonic::Status::failed_precondition( + "Local connectors are not permitted in this context", + ) + .into()); + } + models::MaterializationEndpoint::Local(models::LocalConfig { + command, + config: sealed_config, + env, + protobuf, + }) => { + *config_json = unseal::decrypt_sops(&sealed_config).await?.into(); + + let rx = crate::local_connector::serve( + command, + env, + service.log_handler.clone(), + log_level, + protobuf, + connector_rx, + )? + .boxed(); + + (rx, None) + } + models::MaterializationEndpoint::Dekaf(_) => { + let rx = dekaf_connector::connector(ReceiverStream::new(connector_rx)) + .map_err(crate::anyhow_to_status) + .boxed(); + + (rx, None) + } + }; + + // Send an initial Spec request which may direct us to perform an IAM token exchange. + _ = connector_tx.try_send(materialize::Request { + spec: Some(materialize::request::Spec { + config_json: "{}".into(), + connector_type: connector_type, + }), + ..Default::default() + }); + + let verify = crate::verify("Materialize", "spec response", "connector"); + let spec_response = match verify.not_eof(connector_rx.next().await)? { + materialize::Response { spec: Some(r), .. } => r, + response => return Err(verify.fail_msg(response)), + }; + + if let Ok(Some(iam_config)) = iam_auth::extract_iam_auth_from_connector_config( + config_json, + &spec_response.config_schema_json, + ) { + // Only proceed with IAM auth if we have an actual catalog name + if let Some(task_name) = catalog_name.as_deref() { + let mut tokens = iam_config + .generate_tokens(task_name) + .await + .map_err(crate::anyhow_to_status)?; + + *config_json = tokens.inject_into(config_json)?.to_string().into(); + tokens.zeroize(); + } + } + _ = connector_tx.try_send(initial); + + Ok((connector_tx, connector_rx, container)) +} + +fn extract_endpoint<'r>( + request: &'r mut materialize::Request, +) -> anyhow::Result<( + models::MaterializationEndpoint, + &'r mut bytes::Bytes, + i32, + Option, +)> { + let (connector_type, config_json, catalog_name) = match request { + materialize::Request { + spec: Some(spec), .. + } => (spec.connector_type, &mut spec.config_json, None), + materialize::Request { + validate: Some(validate), + .. + } => ( + validate.connector_type, + &mut validate.config_json, + Some(validate.name.clone()), + ), + materialize::Request { + apply: Some(apply), .. + } => { + let catalog_name = apply.materialization.as_ref().map(|m| m.name.clone()); + let inner = apply + .materialization + .as_mut() + .context("`apply` missing required `materialization`")?; + + (inner.connector_type, &mut inner.config_json, catalog_name) + } + materialize::Request { + open: Some(open), .. + } => { + let catalog_name = open.materialization.as_ref().map(|m| m.name.clone()); + let inner = open + .materialization + .as_mut() + .context("`open` missing required `materialization`")?; + + (inner.connector_type, &mut inner.config_json, catalog_name) + } + request => { + return Err( + crate::verify("Materialize", "valid first request", "controller").fail_msg(request), + ); + } + }; + + if connector_type == flow::materialization_spec::ConnectorType::Image as i32 { + Ok(( + models::MaterializationEndpoint::Connector( + serde_json::from_slice(config_json).context("parsing connector config")?, + ), + config_json, + connector_type, + catalog_name, + )) + } else if connector_type == flow::materialization_spec::ConnectorType::Local as i32 { + Ok(( + models::MaterializationEndpoint::Local( + serde_json::from_slice(config_json).context("parsing local config")?, + ), + config_json, + connector_type, + catalog_name, + )) + } else if connector_type == flow::materialization_spec::ConnectorType::Dekaf as i32 { + Ok(( + models::MaterializationEndpoint::Dekaf( + serde_json::from_slice(config_json).context("parsing local config")?, + ), + config_json, + connector_type, + catalog_name, + )) + } else { + anyhow::bail!("invalid connector type: {connector_type}"); + } +} diff --git a/crates/runtime-next/src/shard/materialize/drain.rs b/crates/runtime-next/src/shard/materialize/drain.rs new file mode 100644 index 00000000000..2d293f94264 --- /dev/null +++ b/crates/runtime-next/src/shard/materialize/drain.rs @@ -0,0 +1,160 @@ +use super::Binding; +use crate::Accumulator; +use anyhow::Context; +use bytes::{BufMut, Bytes}; +use proto_flow::materialize; +use std::collections::VecDeque; +use std::sync::atomic::AtomicBool; + +use crate::proto::materialize::stored::Binding as StoredBinding; + +/// Drainer is a synchronous state machine that drains the accumulator's +/// combined documents and produces `C:Store` requests one at a time. +/// `new` consumes the accumulator and the unused shuffle reader / remainders; +/// `finish` returns them so the actor can restore them on `Phase::Idle`. +pub(super) struct Drainer { + drainer: doc::combine::Drainer, + // Active bindings of this drain, in binding index order. + active: Vec, + // Buffer into which we extract document bytes, key, and values. + buf: bytes::BytesMut, + // Carried through the drain so they can be used in future phases. + parser: simd_doc::Parser, + shuffle_reader: shuffle::log::Reader, + shuffle_remainders: VecDeque, +} + +impl Drainer { + pub fn new( + accumulator: crate::Accumulator, + shuffle_reader: shuffle::log::Reader, + shuffle_remainders: VecDeque, + ) -> anyhow::Result { + let (drainer, parser) = accumulator + .into_drainer() + .context("preparing combiner drain")?; + + Ok(Self { + drainer, + active: Vec::new(), + buf: bytes::BytesMut::new(), + parser, + shuffle_reader, + shuffle_remainders, + }) + } + + /// Drain one document (Some), or returns None when the drain is complete. + pub fn step(&mut self, bindings: &[Binding]) -> anyhow::Result> { + let Some(doc::combine::DrainedDoc { meta, root }) = self.drainer.drain_next()? else { + return Ok(None); + }; + + let binding_index = meta.binding(); + let binding = &bindings[binding_index]; + + let active = if let Some(entry) = self.active.last_mut() + && entry.index as usize == binding_index + { + entry + } else { + self.active.push(StoredBinding { + index: binding_index as u32, + ..Default::default() + }); + self.active.last_mut().unwrap() + }; + + // Track whether truncation occurred. On the order of operations: + // The value extractors may contain a special truncation extractor + // into which will write the value of this variable. Thus, it's critical + // that we extract values last, so it observes truncations of the key + // or the document. + let truncation_indicator = AtomicBool::new(false); + + // Serialize the root document so that we can account for its bytes + // in reported measures. When the binding doesn't store the document, + // count bytes against a sink rather than allocating into `self.buf`. + let serialized = &binding + .ser_policy + .on_owned_with_truncation_indicator(&root, &truncation_indicator); + let doc_json = if binding.store_document { + serde_json::to_writer((&mut self.buf).writer(), serialized) + .expect("document serialization cannot fail"); + active.stored_bytes_total += self.buf.len() as u64; + self.buf.split().freeze() + } else { + let mut counter = ByteCounter(0); + serde_json::to_writer(&mut counter, serialized) + .expect("document serialization cannot fail"); + active.stored_bytes_total += counter.0; + bytes::Bytes::new() + }; + active.stored_docs_total += 1; + + doc::Extractor::extract_all_owned_indicate_truncation( + &root, + &binding.key_extractors, + &mut self.buf, + &truncation_indicator, + ); + let key_packed = self.buf.split().freeze(); + + binding.value_plan.extract_all_owned_indicate_truncation( + &root, + &mut self.buf, + &truncation_indicator, + ); + let values_packed = self.buf.split().freeze(); + + let store = materialize::request::Store { + binding: binding_index as u32, + delete: meta.deleted(), + doc_json, + exists: meta.front(), + key_json: Bytes::new(), + key_packed, + values_json: Bytes::new(), + values_packed, + }; + + Ok(Some(materialize::Request { + store: Some(store), + ..Default::default() + })) + } + + pub fn into_parts( + self, + ) -> anyhow::Result<( + crate::Accumulator, + shuffle::log::Reader, + VecDeque, + Vec, + )> { + let Self { + drainer, + active, + buf: _, + parser, + shuffle_reader, + shuffle_remainders, + } = self; + let accumulator = Accumulator::from_drainer(drainer, parser)?; + Ok((accumulator, shuffle_reader, shuffle_remainders, active)) + } +} + +// `Write` adapter that just counts bytes written, used to size serialized +// documents when the binding doesn't store the document body. +struct ByteCounter(u64); + +impl std::io::Write for ByteCounter { + fn write(&mut self, buf: &[u8]) -> std::io::Result { + self.0 += buf.len() as u64; + Ok(buf.len()) + } + fn flush(&mut self) -> std::io::Result<()> { + Ok(()) + } +} diff --git a/crates/runtime-next/src/shard/materialize/handler.rs b/crates/runtime-next/src/shard/materialize/handler.rs new file mode 100644 index 00000000000..5d06ee2fc2b --- /dev/null +++ b/crates/runtime-next/src/shard/materialize/handler.rs @@ -0,0 +1,228 @@ +use super::{connector, startup}; +use crate::{patches, proto}; +use anyhow::Context; +use futures::StreamExt; +use proto_flow::materialize; +use tokio::sync::mpsc; + +pub(crate) async fn serve( + service: crate::shard::Service, + mut controller_rx: R, + controller_tx: mpsc::UnboundedSender>, +) -> anyhow::Result<()> +where + R: futures::Stream> + Send + Unpin + 'static, +{ + let verify = crate::verify( + "Materialize", + "SessionLoop, Spec, or Validate", + "controller", + ); + while let Some(result) = controller_rx.next().await { + match verify.ok(result)? { + proto::Materialize { + session_loop: Some(session_loop), + .. + } => { + return serve_session_loop( + &service, + &mut controller_rx, + &controller_tx, + session_loop, + ) + .await; + } + + proto::Materialize { + spec: Some(spec), .. + } => { + let request = materialize::Request { + spec: Some(spec), + ..Default::default() + }; + let response = serve_unary(&service, request).await?; + _ = controller_tx.send(Ok(response)); + } + + proto::Materialize { + validate: Some(validate), + .. + } => { + let request = materialize::Request { + validate: Some(validate), + ..Default::default() + }; + let response = serve_unary(&service, request).await?; + _ = controller_tx.send(Ok(response)); + } + + request => return Err(verify.fail_msg(request)), + } + } + Ok(()) +} + +pub async fn serve_unary( + service: &crate::shard::Service, + request: materialize::Request, +) -> anyhow::Result { + let is_spec = request.spec.is_some(); + let is_validate = request.validate.is_some(); + let is_apply = request.apply.is_some(); + + let (connector_tx, mut connector_rx, _container) = connector::start(service, request).await?; + std::mem::drop(connector_tx); // Send EOF. + + // Read connector response, and verify it matches the request type. + let verify = crate::verify("Materialize", "unary response", "connector"); + let response = match verify.not_eof(connector_rx.next().await)? { + materialize::Response { + spec: Some(spec), .. + } if is_spec => proto::Materialize { + spec_response: Some(spec), + ..Default::default() + }, + materialize::Response { + validated: Some(validated), + .. + } if is_validate => proto::Materialize { + validated: Some(validated), + ..Default::default() + }, + materialize::Response { + applied: + Some(materialize::response::Applied { + action_description, + state, + }), + .. + } if is_apply => proto::Materialize { + applied: Some(proto::Applied { + action_description, + connector_patches_json: patches::encode_connector_state(state), + }), + ..Default::default() + }, + response => return Err(verify.fail_msg(response)), + }; + + // Expect EOF after the single response. + () = verify.eof(connector_rx.next().await)?; + + Ok(response) +} + +async fn serve_session_loop( + service: &crate::shard::Service, + controller_rx: &mut R, + controller_tx: &mpsc::UnboundedSender>, + session_loop: proto::SessionLoop, +) -> anyhow::Result<()> +where + R: futures::Stream> + Send + Unpin + 'static, +{ + let proto::SessionLoop { rocksdb_descriptor } = session_loop; + let mut db = crate::shard::RocksDB::open(rocksdb_descriptor).await?; + + let verify = crate::verify("Materialize", "Join", "controller"); + while let Some(result) = controller_rx.next().await { + match verify.ok(result)? { + proto::Materialize { + join: Some(join), .. + } => { + db = serve_session(service, controller_rx, controller_tx, db, join).await?; + } + request => return Err(verify.fail_msg(request)), + }; + } + + Ok(()) +} + +async fn serve_session( + service: &crate::shard::Service, + controller_rx: &mut R, + controller_tx: &mpsc::UnboundedSender>, + db: crate::shard::RocksDB, + join: proto::Join, +) -> anyhow::Result +where + R: futures::Stream> + Send + Unpin + 'static, +{ + let proto::join::Shard { + etcd_create_revision: _, + id: shard_id, + labeling, + reactor: _, + } = join + .shards + .get(join.shard_index as usize) + .context("missing shard for shard index")?; + + let labeling = labeling.as_ref().context("missing shard labeling")?.clone(); + let shard_id = shard_id.clone(); + let shard_index = join.shard_index; + let shuffle_directory = join.shuffle_directory.clone(); + + let (joined, leader_stream) = startup::dial_and_join(join).await?; + + // Forward Joined to controller. + _ = controller_tx.send(Ok(proto::Materialize { + joined: Some(joined), + ..Default::default() + })); + let Some((leader_tx, leader_rx)) = leader_stream else { + return Ok(db); // Controller must retry Join/Joined. + }; + + let startup::Startup { + accumulator, + bindings, + binding_state_keys, + mut connector_rx, + connector_tx, + db, + disable_load_optimization, + mut leader_rx, + leader_tx, + max_keys, + shuffle_reader, + } = startup::run( + controller_rx, + controller_tx, + db, + labeling, + leader_rx, + leader_tx, + service, + shard_id, + shard_index, + shuffle_directory, + ) + .await?; + + let db = super::actor::Actor::new( + bindings, + binding_state_keys, + connector_tx, + db, + disable_load_optimization, + leader_tx, + max_keys, + ) + .serve( + accumulator, + &mut connector_rx, + controller_rx, + &mut leader_rx, + shuffle_reader, + ) + .await?; + + _ = controller_tx.send(Ok(proto::Materialize { + stopped: Some(proto::Stopped {}), + ..Default::default() + })); + + Ok(db) +} diff --git a/crates/runtime-next/src/shard/materialize/mod.rs b/crates/runtime-next/src/shard/materialize/mod.rs new file mode 100644 index 00000000000..54e9f11c869 --- /dev/null +++ b/crates/runtime-next/src/shard/materialize/mod.rs @@ -0,0 +1,39 @@ +mod actor; +mod connector; +mod drain; +mod handler; +mod scan; +mod startup; +mod task; + +pub(crate) use handler::serve; + +#[derive(Debug)] +struct Binding { + collection_name: String, // Source collection. + delta_updates: bool, // Delta updates, or standard? + key_extractors: Vec, // Key extractors for this collection. + read_schema_json: bytes::Bytes, // Read JSON-Schema of collection documents. + ser_policy: doc::SerPolicy, // Serialization policy for this source. + state_key: String, // State key for this binding. + store_document: bool, // Are we storing the root document (often `flow_document`)? + value_plan: doc::ExtractorPlan, +} + +// Set of observed keys, used to de-duplicate sent C:Load requests. +type LoadKeys = std::collections::HashSet>; + +/// Identity hasher over the 16-byte xxh3_128 key digests we insert. +/// We trust the digest to be well-distributed and avoid re-hashing it. +#[derive(Default)] +struct IdentHasher(u64); + +impl std::hash::Hasher for IdentHasher { + fn write(&mut self, bytes: &[u8]) { + self.0 = u64::from_ne_bytes(bytes[..8].try_into().unwrap()); + } + + fn finish(&self) -> u64 { + self.0 + } +} diff --git a/crates/runtime-next/src/shard/materialize/scan.rs b/crates/runtime-next/src/shard/materialize/scan.rs new file mode 100644 index 00000000000..a9ac5cb7544 --- /dev/null +++ b/crates/runtime-next/src/shard/materialize/scan.rs @@ -0,0 +1,177 @@ +use super::{Binding, LoadKeys}; +use anyhow::Context; +use bytes::Buf; +use bytes::{BufMut, Bytes}; +use proto_flow::materialize; +use std::collections::{HashMap, VecDeque}; + +use crate::proto::materialize::loaded::Binding as LoadedBinding; + +/// Scanner is a synchronous state machine that walks a `shuffle::Frontier` +/// one block at a time. Each `step` adds source documents to the +/// accumulator's memtable and produces `C:Load` requests for keys that +/// may exist in the materialization endpoint. +pub(super) struct Scanner { + accumulator: crate::Accumulator, + // FrontierScan underway by this Scanner. + scan: shuffle::log::FrontierScan, + // Buffer into which we extract and hash packed keys. + buf: bytes::BytesMut, + // Active bindings of this scan, indexed on binding index. + active: HashMap, +} + +impl Scanner { + pub fn new( + accumulator: crate::Accumulator, + frontier: shuffle::Frontier, + shuffle_reader: shuffle::log::Reader, + shuffle_remainders: VecDeque, + ) -> anyhow::Result { + let scan = shuffle::log::FrontierScan::new(frontier, shuffle_reader, shuffle_remainders) + .context("failed to begin a FrontierScan")?; + + Ok(Self { + accumulator, + scan, + buf: bytes::BytesMut::new(), + active: HashMap::new(), + }) + } + + pub fn accumulator(&mut self) -> &mut crate::Accumulator { + &mut self.accumulator + } + + /// Process at most one block. Pushes any produced `C:Load` requests onto + /// `out`. Returns `Ok(true)` while there is more work to do, and + /// `Ok(false)` once the frontier has been fully consumed. + pub fn step( + &mut self, + bindings: &[Binding], + load_keys: &mut LoadKeys, + max_keys: &mut [(Bytes, Bytes)], + disable_load_optimization: bool, + out: &mut Vec, + ) -> anyhow::Result { + if !self + .scan + .advance_block() + .context("failed to advance FrontierScan block")? + { + return Ok(false); + } + + let memtable = self + .accumulator + .memtable() + .context("failed to acquire combiner memtable")?; + let alloc = memtable.alloc(); + + for shuffle::log::reader::Entry { + meta, + doc, + journal: _, + producer: _, + } in self.scan.block_iter() + { + // Was the document successfully validated against its JSON schema? + let known_valid = meta.flags.to_native() & shuffle::FLAGS_SCHEMA_VALID != 0; + + let binding_index = meta.binding.to_native() as u32; + let binding = bindings + .get(meta.binding.to_native() as usize) + .context("scan entry has invalid meta.binding")?; + + memtable + .add_embedded( + meta.binding.to_native(), + &doc.packed_key_prefix, + doc.doc.to_heap(alloc), + false, + known_valid, + ) + .context("MemTable::add_embedded failed")?; + + // Encode the binding index followed by the packed key, for hashing. + self.buf.put_u32(binding_index); + doc::Extractor::extract_all(doc.doc.get(), &binding.key_extractors, &mut self.buf); + let key_hash = xxhash_rust::xxh3::xxh3_128(&self.buf); + let mut key_packed = self.buf.split().freeze(); + key_packed.advance(4); // Advance past 4-byte binding index. + + // `next_max` starts empty each transaction and only ratchets to + // larger-than-`prev_max` values. The `gt_prev_max` guard is needed, + // as `key_packed` compares greater than initial (empty) `next_max`. + let (prev_max, next_max) = &mut max_keys[binding_index as usize]; + let gt_prev_max = key_packed > *prev_max; + let gt_next_max = gt_prev_max && key_packed > *next_max; + + let active = self.active.entry(binding_index).or_default(); + + // Accumulate metrics for active bindings of the scan. + let clock = meta.clock.to_native(); + if active.sourced_docs_total == 0 { + active.index = binding_index; + active.max_source_clock = clock; + active.min_source_clock = clock; + } else { + active.max_source_clock = active.max_source_clock.max(clock); + active.min_source_clock = active.min_source_clock.min(clock); + } + active.sourced_docs_total += 1; + active.sourced_bytes_total += doc.source_byte_length.to_native() as u64; + + // Is `key_packed` larger than the largest key previously stored + // to the connector? If so, then it cannot possibly exist. + // We still track the max key even when the optimization is disabled. + if gt_prev_max { + if gt_next_max { + // This is a new high water mark for the largest-stored key. + *next_max = key_packed.clone(); + active.max_key_delta = key_packed.clone(); + } + // Skip the load request unless optimization is disabled. + if !disable_load_optimization { + continue; + } + } + + if binding.delta_updates { + // Delta-update bindings don't load. + } else if load_keys.contains(&key_hash) { + // We already sent a Load request for this key. + } else { + load_keys.insert(key_hash); + + out.push(materialize::Request { + load: Some(materialize::request::Load { + binding: binding_index, + key_json: Bytes::new(), // NOTE(johnny): Unclear if we'll implement this. + key_packed, + }), + ..Default::default() + }); + } + } + + Ok(true) + } + + pub fn into_parts( + self, + ) -> ( + crate::Accumulator, + shuffle::log::Reader, + VecDeque, + HashMap, + ) { + let (_, shuffle_reader, shuffle_remainders) = self.scan.into_parts(); + ( + self.accumulator, + shuffle_reader, + shuffle_remainders, + self.active, + ) + } +} diff --git a/crates/runtime-next/src/shard/materialize/startup.rs b/crates/runtime-next/src/shard/materialize/startup.rs new file mode 100644 index 00000000000..efbca991ecb --- /dev/null +++ b/crates/runtime-next/src/shard/materialize/startup.rs @@ -0,0 +1,318 @@ +use super::Binding; +use crate::proto; +use anyhow::Context; +use futures::StreamExt; +use prost::Message; +use proto_flow::{flow, materialize}; +use tokio::sync::mpsc; + +pub async fn dial_and_join( + join: proto::Join, +) -> anyhow::Result<( + proto::Joined, + Option<( + mpsc::UnboundedSender, + tonic::Streaming, + )>, +)> { + let leader_endpoint = join.leader_endpoint.clone(); + + // First, dial the leader. + let channel = gazette::dial_channel(&leader_endpoint).context("failed to dial leader")?; + let mut leader_client = proto_grpc::runtime::leader_client::LeaderClient::new(channel) + .max_decoding_message_size(crate::MAX_MESSAGE_SIZE) + .max_encoding_message_size(usize::MAX); + + // Start the materialize RPC. We use an unbounded sender because we never + // pump messages to the leader (strictly request / response). + let (leader_tx, leader_rx) = tokio::sync::mpsc::unbounded_channel(); + let mut leader_rx = leader_client + .materialize(tokio_stream::wrappers::UnboundedReceiverStream::new( + leader_rx, + )) + .await + .context("opening leader Materialize stream")? + .into_inner(); + + // Send L:Join, and read L:Joined. + let verify = crate::verify("Materialize", "Joined", "leader"); + + _ = leader_tx.send(proto::Materialize { + join: Some(join), + ..Default::default() + }); + let joined = match verify.not_eof(leader_rx.next().await)? { + proto::Materialize { + joined: Some(joined), + .. + } => joined, + response => return Err(verify.fail_msg(response)), + }; + + // Did leader signal that we need to retry? + if joined.max_etcd_revision != 0 { + // If leader signaled retry, expect it next sends EOF. + () = verify.eof(leader_rx.next().await)?; + + Ok((joined, None)) + } else { + Ok((joined, Some((leader_tx, leader_rx)))) + } +} + +pub(super) struct Startup { + pub accumulator: crate::Accumulator, + pub bindings: Vec, + pub binding_state_keys: Vec, + pub connector_rx: futures::stream::BoxStream<'static, tonic::Result>, + pub connector_tx: mpsc::Sender, + pub db: crate::shard::RocksDB, + pub disable_load_optimization: bool, + pub leader_rx: tonic::Streaming, + pub leader_tx: mpsc::UnboundedSender, + pub max_keys: Vec<(bytes::Bytes, bytes::Bytes)>, + pub shuffle_reader: shuffle::log::Reader, +} + +pub(super) async fn run( + controller_rx: &mut R, + controller_tx: &mpsc::UnboundedSender>, + db: crate::shard::RocksDB, + labeling: ops::proto::ShardLabeling, + mut leader_rx: tonic::Streaming, + leader_tx: mpsc::UnboundedSender, + service: &crate::shard::Service, + shard_id: String, + shard_index: u32, + shuffle_directory: String, +) -> anyhow::Result +where + R: futures::Stream> + Send + Unpin + 'static, +{ + // Receive L:Task from controller. + let verify = crate::verify("Materialize", "Open", "controller"); + let l_task = match verify.not_eof(controller_rx.next().await)? { + proto::Materialize { + task: Some(task), .. + } => task, + other => return Err(verify.fail_msg(other)), + }; + // Shard zero (only) forwards L:Task to leader. + if shard_index == 0 { + _ = leader_tx.send(proto::Materialize { + task: Some(l_task.clone()), + ..Default::default() + }); + } + + let proto::Task { + max_transactions: _, + ops_stats_journal, + ops_stats_spec, + preview, + spec: spec_bytes, + } = l_task; + + // Build task definition. + let spec = flow::MaterializationSpec::decode(spec_bytes.as_ref()) + .context("invalid Task materialization")?; + let (bindings, shard_ref) = + super::task::build_bindings(&spec, &labeling).context("building task definition")?; + // Reserved for future logging; the actor and scan/drain activities + // don't presently need shard identity. + let _ = shard_ref; + + // Initialize publisher. + let publisher = if preview { + crate::Publisher::new_preview() + } else { + let ops_stats_spec = ops_stats_spec.as_ref().context("missing ops stats spec")?; + + crate::Publisher::new_real( + shard_id, // Shard ID is AuthZ subject. + &service.publisher_factory, + &ops_stats_journal, + ops_stats_spec, + [], // No additional bindings. + ) + .context("creating publisher")? + }; + // TODO: Materialization shards don't publish journal documents — the leader + // handles all publishing. We construct the publisher here only as a + // working pattern/template for derivations, which do publish from the + // shard. Once derivations are implemented this construction stays there + // and the call here is removed. + drop(publisher); + + // Scan and send L:Recover state from RocksDB. + let mut sorted_state_keys: Vec<(String, u32)> = bindings + .iter() + .enumerate() + .map(|(i, b)| (b.state_key.clone(), i as u32)) + .collect(); + sorted_state_keys.sort(); + + let (mut db, recover) = db + .scan(sorted_state_keys) + .await + .context("scanning RocksDB")?; + + _ = leader_tx.send(proto::Materialize { + recover: Some(recover), + ..Default::default() + }); + + let binding_state_keys: Vec = bindings.iter().map(|b| b.state_key.clone()).collect(); + + // Read and execute L:Apply and L:Persist from the leader until L:Open. + let open = loop { + let verify = crate::verify("Materialize", "Apply, Persist, or Open", "leader"); + match verify.not_eof(leader_rx.next().await)? { + proto::Materialize { + apply: + Some(proto::Apply { + connector_state_json, + last_spec, + last_version, + spec, + version, + }), + .. + } => { + let last_spec = if last_spec.is_empty() { + None + } else { + Some( + flow::MaterializationSpec::decode(last_spec.as_ref()) + .context("invalid last Apply spec")?, + ) + }; + let spec = flow::MaterializationSpec::decode(spec.as_ref()) + .context("invalid current Apply spec")?; + + let apply = materialize::request::Apply { + materialization: Some(spec), + last_materialization: last_spec, + last_version, + state_json: connector_state_json, + version, + }; + _ = leader_tx.send( + super::handler::serve_unary( + service, + materialize::Request { + apply: Some(apply), + ..Default::default() + }, + ) + .await?, + ); + } + proto::Materialize { + persist: Some(persist), + .. + } => { + db = db + .persist(&persist, &binding_state_keys) + .await + .context("Persist failed")?; + + _ = leader_tx.send(proto::Materialize { + persisted: Some(proto::Persisted { + nonce: persist.nonce, + }), + ..Default::default() + }); + } + proto::Materialize { + open: Some(open), .. + } => break open, + + other => return Err(verify.fail_msg(other)), + } + }; + + // Start the connector and send C:Open. + let proto::Open { + connector_state_json, + max_keys, + range, + spec, + version, + } = open; + + let spec = + flow::MaterializationSpec::decode(spec.as_ref()).context("invalid current Apply spec")?; + + let initial = materialize::Request { + open: Some(materialize::request::Open { + materialization: Some(spec), + version, + state_json: connector_state_json, + range, + }), + ..Default::default() + }; + let (connector_tx, mut connector_rx, container) = + super::connector::start(service, initial).await?; + + // Read C:Opened from the connector. + let verify = crate::verify("Materialize", "Opened", "connector"); + let opened = match verify.not_eof(connector_rx.next().await)? { + materialize::Response { + opened: Some(opened), + .. + } => opened, + other => return Err(verify.fail_msg(other)), + }; + let materialize::response::Opened { + disable_load_optimization, + runtime_checkpoint, + } = opened; + + // Tell Leader and Controller of C:Opened. + _ = leader_tx.send(proto::Materialize { + opened: Some(proto::materialize::Opened { + container: None, + connector_checkpoint: runtime_checkpoint, + }), + ..Default::default() + }); + _ = controller_tx.send(Ok(proto::Materialize { + opened: Some(proto::materialize::Opened { + container, + connector_checkpoint: None, + }), + ..Default::default() + })); + + let shuffle_reader = + shuffle::log::Reader::new(std::path::Path::new(&shuffle_directory), shard_index); + + let accumulator = crate::Accumulator::new(super::task::combine_spec(&bindings)?) + .context("building materialize combiner")?; + + // Densify the leader's sparse `max_keys` map into a per-binding Vec. + let max_keys: Vec<(bytes::Bytes, bytes::Bytes)> = (0..bindings.len() as u32) + .map(|i| { + ( + max_keys.get(&i).cloned().unwrap_or_default(), // Previous max. + bytes::Bytes::new(), // Next max. + ) + }) + .collect(); + + Ok(Startup { + accumulator, + bindings, + binding_state_keys, + connector_rx, + connector_tx, + db, + disable_load_optimization, + leader_rx, + leader_tx, + max_keys, + shuffle_reader, + }) +} diff --git a/crates/runtime-next/src/shard/materialize/task.rs b/crates/runtime-next/src/shard/materialize/task.rs new file mode 100644 index 00000000000..3d2115bf356 --- /dev/null +++ b/crates/runtime-next/src/shard/materialize/task.rs @@ -0,0 +1,202 @@ +use super::Binding; +use anyhow::Context; +use proto_flow::flow; + +/// Build binding structures and shard_ref for a materialization task. +pub fn build_bindings( + spec: &flow::MaterializationSpec, + shard: &ops::proto::ShardLabeling, +) -> anyhow::Result<(Vec, ops::ShardRef)> { + let flow::MaterializationSpec { + bindings, + config_json, + connector_type: _, + name, + network_ports: _, + recovery_log_template: _, + shard_template: _, + inactive_bindings: _, + triggers_json: _, + } = spec; + + let ops::proto::ShardLabeling { + range, + build: version, + .. + } = shard; + + let range = range.context("missing range")?; + + if range.r_clock_begin != 0 || range.r_clock_end != u32::MAX { + anyhow::bail!("materialization cannot split on r-clock: {range:?}"); + } + + // TODO(johnny): Hack to limit serialized value sizes for these common materialization connectors + // that don't handle large strings very well. This should be negotiated via connector protocol. + // See go/runtime/materialize.go:135 + let ser_policy = if let Some(limit) = [ + ("ghcr.io/estuary/materialize-azure-fabric-warehouse", 1000), + ("ghcr.io/estuary/materialize-bigquery", 1500), + ("ghcr.io/estuary/materialize-kafka", 1000), + ("ghcr.io/estuary/materialize-snowflake", 1000), + ("ghcr.io/estuary/materialize-redshift", 1000), + ("ghcr.io/estuary/materialize-sqlite", 1000), + ] + .iter() + .filter_map(|(image, limit)| { + config_json + .windows(image.len()) + .any(|window| window == image.as_bytes()) + .then_some(*limit) + }) + .next() + { + doc::SerPolicy { + str_truncate_after: 1 << 16, // Truncate at 64KB. + nested_obj_truncate_after: limit, + array_truncate_after: limit, + ..doc::SerPolicy::noop() + } + } else { + doc::SerPolicy::noop() + }; + + let bindings = bindings + .into_iter() + .enumerate() + .map(|(index, spec)| build_binding(spec, &ser_policy).context(index)) + .collect::, _>>()?; + + let shard_ref = ops::ShardRef { + kind: ops::TaskType::Materialization as i32, + name: name.clone(), + key_begin: format!("{:08x}", range.key_begin), + r_clock_begin: format!("{:08x}", range.r_clock_begin), + build: version.clone(), + }; + + Ok((bindings, shard_ref)) +} + +// Build the runtime structure for a single binding. +fn build_binding( + spec: &flow::materialization_spec::Binding, + default_ser_policy: &doc::SerPolicy, +) -> anyhow::Result { + let flow::materialization_spec::Binding { + backfill: _, + collection, + delta_updates, + deprecated_shuffle: _, + field_selection, + journal_read_suffix: _, + not_after: _, + not_before: _, + partition_selector: _, + priority: _, + resource_config_json: _, + resource_path: _, + state_key, + ser_policy: binding_ser_policy, + } = spec; + + let flow::FieldSelection { + document: selected_root, + field_config_json_map: _, + keys: selected_key, + values: selected_values, + } = field_selection + .as_ref() + .context("missing field selection")?; + + let flow::CollectionSpec { + ack_template_json: _, + derivation: _, + key: _, + name: collection_name, + partition_fields: _, + partition_template: _, + projections, + read_schema_json, + uuid_ptr: _, + write_schema_json, + } = collection.as_ref().context("missing collection")?; + + // TODO(whb): At some point once all built materialization specs have + // been updated we can get rid of the `default_ser_policy` parameter and + // just default to doc::SerPolicy::noop() with overrides from the + // specific binding serialization policy. + let ser_policy = if let Some(binding_ser_policy) = binding_ser_policy { + let mut base = doc::SerPolicy::noop(); + if binding_ser_policy.str_truncate_after > 0 { + base.str_truncate_after = binding_ser_policy.str_truncate_after as usize; + }; + if binding_ser_policy.nested_obj_truncate_after > 0 { + base.nested_obj_truncate_after = binding_ser_policy.nested_obj_truncate_after as usize; + }; + if binding_ser_policy.array_truncate_after > 0 { + base.array_truncate_after = binding_ser_policy.array_truncate_after as usize; + }; + base + } else { + default_ser_policy.clone() + }; + + let key_extractors = extractors::for_fields(selected_key, projections, &ser_policy)?; + let value_plan = doc::ExtractorPlan::new(&extractors::for_fields( + selected_values, + projections, + &ser_policy, + )?); + + let read_schema_json = if read_schema_json.is_empty() { + write_schema_json + } else { + read_schema_json + } + .clone(); + + Ok(Binding { + collection_name: collection_name.clone(), + delta_updates: *delta_updates, + key_extractors, + read_schema_json, + ser_policy, + state_key: state_key.clone(), + store_document: !selected_root.is_empty(), + value_plan, + }) +} + +pub fn combine_spec(bindings: &[Binding]) -> anyhow::Result { + let mut combiner_specs = Vec::with_capacity(bindings.len()); + + for Binding { + state_key, + read_schema_json, + delta_updates, + key_extractors, + collection_name, + .. + } in bindings + { + let built_schema = doc::validation::build_bundle(read_schema_json) + .context("collection read_schema_json is not a JSON schema")?; + let validator = doc::Validator::new(built_schema).with_context(|| { + format!("could not build a schema validator for binding {state_key}",) + })?; + + combiner_specs.push(( + !delta_updates, + key_extractors.clone(), + format!("materialized collection {collection_name}"), + validator, + )); + } + + // Build combiner Spec with all bindings, plus one extra for state reductions. + Ok(doc::combine::Spec::with_bindings( + combiner_specs, + Vec::new(), + )) +} diff --git a/crates/runtime-next/src/shard/mod.rs b/crates/runtime-next/src/shard/mod.rs new file mode 100644 index 00000000000..3cdb0ec1966 --- /dev/null +++ b/crates/runtime-next/src/shard/mod.rs @@ -0,0 +1,7 @@ +pub mod materialize; +pub(crate) mod recovery; +mod rocksdb; +mod service; + +use rocksdb::RocksDB; +pub use service::Service; diff --git a/crates/runtime-next/src/shard/recovery.rs b/crates/runtime-next/src/shard/recovery.rs new file mode 100644 index 00000000000..fa61a288ffc --- /dev/null +++ b/crates/runtime-next/src/shard/recovery.rs @@ -0,0 +1,961 @@ +//! Codec for the RocksDB keys and values that back runtime tasks. +//! +//! [`encode_persist`] turns a [`proto::Persist`] message into an ordered +//! stream of [`KeyOp`] effects, invoking a caller-supplied closure for each. +//! `shard::rocksdb` stages those effects directly into a real +//! `rocksdb::WriteBatch`. [`decode_recover_key_value`] is called once per +//! `(key, value)` pair from a full `rocksdb::DB` scan on session startup, +//! folding singleton state directly into a `proto::Recover` while collecting +//! frontier entries separately for final sort and proto encoding. +//! +//! `{state_key}` below is the binding-stable `state_key` field of +//! `flow::MaterializationSpec.Binding` — distinct from `journal_read_suffix`, +//! which is `materialize/{materialization}/{state_key}`. Both encode and +//! decode use a caller-supplied `binding_state_keys` mapping to translate +//! between the binding indices used in the leader protocol and the +//! `state_key` strings used in RocksDB keys. +//! +//! | Prefix | Key tail | Value | +//! |--------------|------------------------------------------|----------------------------------| +//! | `FH:` | `{journal}\0{state_key}\0{producer[6]}` | proto `shuffle.ProducerFrontier` | +//! | `FC:` | `{journal}\0{state_key}\0{producer[6]}` | proto `shuffle.ProducerFrontier` | +//! | `AI:` | `{journal}` | raw ACK intent bytes | +//! | `MK-v2:` | `{state_key}` | `tuple::pack` packed key | +//! | (singleton) | `checkpoint` | legacy `consumer.Checkpoint` | +//! | (singleton) | `committed-close` | fixed64 little-endian clock | +//! | (singleton) | `connector-state` | reduced JSON merge-patch | +//! | (singleton) | `hinted-close` | fixed64 little-endian clock | +//! | (singleton) | `last-applied` | proto task spec | +//! | (singleton) | `trigger-params` | JSON `TriggerVariables` | + +use crate::proto; +use bytes::{BufMut, Bytes, BytesMut}; +use prost::Message; +use proto_gazette::{consumer, uuid}; + +/// Key prefix for hinted Frontier entries: +/// `FH:{journal}\0{state_key}\0{producer}`. +pub const PREFIX_HINTED_FRONTIER: &[u8] = b"FH:"; +/// Exclusive upper bound used for `DeleteRange` over `PREFIX_HINTED_FRONTIER`. +pub const PREFIX_HINTED_FRONTIER_END: &[u8] = b"FH;"; +/// Key prefix for committed Frontier entries: +/// `FC:{journal}\0{state_key}\0{producer}`. +pub const PREFIX_COMMITTED_FRONTIER: &[u8] = b"FC:"; +/// Key prefix for per-journal ACK intent entries: `AI:{journal}`. +pub const PREFIX_ACK_INTENT: &[u8] = b"AI:"; +/// Exclusive upper bound used for `DeleteRange` over `PREFIX_ACK_INTENT`. +pub const PREFIX_ACK_INTENT_END: &[u8] = b"AI;"; +/// Key prefix for per-binding max-key entries: `MK-v2:{state_key}`. +pub const PREFIX_MAX_KEY: &[u8] = b"MK-v2:"; +/// Legacy checkpoint. +pub const KEY_LEGACY_CHECKPOINT: &[u8] = b"checkpoint"; +/// Clock at which the last-committed transaction closed. +pub const KEY_COMMITTED_CLOSE: &[u8] = b"committed-close"; +/// Reduced connector state (opaque JSON). +pub const KEY_CONNECTOR_STATE: &[u8] = b"connector-state"; +/// Clock at which the last-hinted transaction closed. +pub const KEY_HINTED_CLOSE: &[u8] = b"hinted-close"; +/// Last-applied task spec (protobuf bytes). +pub const KEY_LAST_APPLIED: &[u8] = b"last-applied"; +/// Trigger parameters (JSON `models::TriggerVariables`). +pub const KEY_TRIGGER_PARAMS: &[u8] = b"trigger-params"; + +/// A single write effect contributed by a `Persist`. Values are carried as +/// [`Bytes`] so shared allocations (e.g. a proto-decoded +/// `connector_patches_json` buffer) can be split without copies. +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum KeyOp { + Put { + key: Bytes, + value: Bytes, + }, + Merge { + key: Bytes, + value: Bytes, + }, + Delete { + key: Bytes, + }, + /// Delete keys in the half-open range `[from, to)`. + DeleteRange { + from: Bytes, + to: Bytes, + }, +} + +/// Errors produced by [`encode_persist`]. +#[derive(Debug, thiserror::Error)] +pub enum EncodeError { + #[error( + "Frontier journal has binding index {binding}, but only {num_bindings} \ + binding state_keys were supplied" + )] + UnknownBinding { binding: u32, num_bindings: usize }, + #[error("connector_patches_json is framed (starts with `[`) but is malformed: {reason}")] + MalformedStatePatches { reason: &'static str }, +} + +impl From for EncodeError { + fn from(err: crate::patches::MalformedStatePatches) -> Self { + Self::MalformedStatePatches { reason: err.reason } + } +} + +/// Errors produced by [`decode_recover_key_value`]. +#[derive(Debug, thiserror::Error)] +pub enum DecodeError { + #[error("duplicate {kind} singleton key")] + DuplicateSingleton { kind: &'static str }, + #[error("FH:/FC: key is malformed: {reason}")] + MalformedFrontierKey { reason: &'static str }, + #[error("FH:/FC: value failed to decode as ProducerFrontier")] + FrontierValueDecode(#[source] prost::DecodeError), + #[error("checkpoint value failed to decode as consumer.Checkpoint")] + CheckpointValueDecode(#[source] prost::DecodeError), + #[error("key component is not valid UTF-8")] + InvalidUtf8(#[source] std::str::Utf8Error), + #[error("{kind} singleton value is {got} bytes, want 8")] + InvalidClockValue { kind: &'static str, got: usize }, + #[error(transparent)] + RocksDB(#[from] rocksdb::Error), +} + +/// Encode a [`proto::Persist`] as an ordered stream of [`KeyOp`] effects, +/// invoking `emit` for each. +/// +/// `binding_state_keys[i]` is the stable `state_key` for binding index `i`. +pub fn encode_persist>( + persist: &proto::Persist, + binding_state_keys: &[S], + mut emit: impl FnMut(KeyOp), +) -> Result<(), EncodeError> { + let mut buf = BytesMut::new(); + + if persist.delete_ack_intents { + emit(KeyOp::DeleteRange { + from: Bytes::from_static(PREFIX_ACK_INTENT), + to: Bytes::from_static(PREFIX_ACK_INTENT_END), + }); + } + for (journal, intent) in &persist.ack_intents { + buf.extend_from_slice(PREFIX_ACK_INTENT); + buf.extend_from_slice(journal.as_bytes()); + emit(KeyOp::Put { + key: buf.split().freeze(), + value: intent.clone(), + }); + } + + if persist.committed_close_clock != 0 { + emit(KeyOp::Put { + key: Bytes::from_static(KEY_COMMITTED_CLOSE), + value: Bytes::copy_from_slice(&persist.committed_close_clock.to_le_bytes()), + }); + } + + if let Some(frontier) = &persist.committed_frontier { + encode_frontier( + PREFIX_COMMITTED_FRONTIER, + frontier, + binding_state_keys, + &mut emit, + &mut buf, + )?; + } + + for patch in crate::patches::split_state_patches(&persist.connector_patches_json)? { + emit(KeyOp::Merge { + key: Bytes::from_static(KEY_CONNECTOR_STATE), + value: patch, + }); + } + + if persist.hinted_close_clock != 0 { + emit(KeyOp::Put { + key: Bytes::from_static(KEY_HINTED_CLOSE), + value: Bytes::copy_from_slice(&persist.hinted_close_clock.to_le_bytes()), + }); + } + + if persist.delete_hinted_frontier { + emit(KeyOp::DeleteRange { + from: Bytes::from_static(PREFIX_HINTED_FRONTIER), + to: Bytes::from_static(PREFIX_HINTED_FRONTIER_END), + }); + } + if let Some(frontier) = &persist.hinted_frontier { + encode_frontier( + PREFIX_HINTED_FRONTIER, + frontier, + binding_state_keys, + &mut emit, + &mut buf, + )?; + } + + if !persist.last_applied.is_empty() { + emit(KeyOp::Put { + key: Bytes::from_static(KEY_LAST_APPLIED), + value: persist.last_applied.clone(), + }); + } + + if let Some(checkpoint) = &persist.legacy_checkpoint { + checkpoint + .encode(&mut buf) + .expect("BytesMut never errors on encode"); + emit(KeyOp::Put { + key: Bytes::from_static(KEY_LEGACY_CHECKPOINT), + value: buf.split().freeze(), + }); + } + + for (binding, packed_key) in &persist.max_keys { + let state_key = binding_state_keys + .get(*binding as usize) + .ok_or(EncodeError::UnknownBinding { + binding: *binding, + num_bindings: binding_state_keys.len(), + })? + .as_ref(); + + buf.extend_from_slice(PREFIX_MAX_KEY); + buf.extend_from_slice(state_key.as_bytes()); + emit(KeyOp::Put { + key: buf.split().freeze(), + value: packed_key.clone(), + }); + } + + if persist.delete_trigger_params { + emit(KeyOp::Delete { + key: Bytes::from_static(KEY_TRIGGER_PARAMS), + }); + } + if !persist.trigger_params_json.is_empty() { + emit(KeyOp::Put { + key: Bytes::from_static(KEY_TRIGGER_PARAMS), + value: persist.trigger_params_json.clone(), + }); + } + + Ok(()) +} + +fn encode_frontier>( + prefix: &'static [u8], + frontier: &shuffle::proto::Frontier, + binding_state_keys: &[S], + emit: &mut impl FnMut(KeyOp), + buf: &mut BytesMut, +) -> Result<(), EncodeError> { + // Frontier.journals is delta-encoded against a running journal name. + let mut journal = String::new(); + + for jf in &frontier.journals { + let new_len = journal + .len() + .saturating_sub(jf.journal_name_truncate_delta.max(0) as usize); + journal.truncate(new_len); + journal.push_str(&jf.journal_name_suffix); + + let state_key = binding_state_keys + .get(jf.binding as usize) + .ok_or(EncodeError::UnknownBinding { + binding: jf.binding, + num_bindings: binding_state_keys.len(), + })? + .as_ref(); + + for producer in &jf.producers { + let producer_id = uuid::Producer::from_i64(producer.producer); + append_frontier_key(buf, prefix, &journal, state_key, producer_id.as_bytes()); + let key = buf.split().freeze(); + + // The producer id is captured in the key tail; clear it from the + // value so the field has exactly one source of truth. + let value = shuffle::proto::ProducerFrontier { + producer: 0, + ..*producer + }; + value.encode(buf).expect("BytesMut never errors on encode"); + emit(KeyOp::Put { + key, + value: buf.split().freeze(), + }); + } + } + + Ok(()) +} + +fn append_frontier_key( + out: &mut BytesMut, + prefix: &[u8], + journal: &str, + state_key: &str, + producer: &[u8; 6], +) { + out.extend_from_slice(prefix); + out.extend_from_slice(journal.as_bytes()); + out.put_u8(0); + out.extend_from_slice(state_key.as_bytes()); + out.put_u8(0); + out.extend_from_slice(producer); +} + +// --------------------------------------------------------------------------- +// Decoder +// --------------------------------------------------------------------------- + +/// Decode one RocksDB `(key, value)` pair into recovery accumulators. +/// +/// `binding_state_keys` is a slice of `(state_key, binding_index)` tuples +/// sorted on `state_key`, used to translate persisted `state_key`s in +/// `FH:`/`FC:`/`MK-v2:` keys into their current binding indices. Entries +/// whose `state_key` does not appear in the slice are silently dropped: they +/// belong to bindings that have been removed or backfilled. +pub fn decode_recover_key_value( + recover: &mut proto::Recover, + committed_frontier: &mut Vec, + hinted_frontier: &mut Vec, + key: &[u8], + value: &[u8], + binding_state_keys: &[(String, u32)], +) -> Result<(), DecodeError> { + if let Some(rest) = key.strip_prefix(PREFIX_HINTED_FRONTIER) { + decode_frontier_entry(hinted_frontier, rest, value, binding_state_keys) + } else if let Some(rest) = key.strip_prefix(PREFIX_COMMITTED_FRONTIER) { + decode_frontier_entry(committed_frontier, rest, value, binding_state_keys) + } else if let Some(rest) = key.strip_prefix(PREFIX_ACK_INTENT) { + let journal = std::str::from_utf8(rest).map_err(DecodeError::InvalidUtf8)?; + recover + .ack_intents + .insert(journal.to_owned(), Bytes::copy_from_slice(value)); + Ok(()) + } else if let Some(rest) = key.strip_prefix(PREFIX_MAX_KEY) { + let state_key = std::str::from_utf8(rest).map_err(DecodeError::InvalidUtf8)?; + if let Some(binding) = lookup_binding(binding_state_keys, state_key) { + recover + .max_keys + .insert(binding, Bytes::copy_from_slice(value)); + } + Ok(()) + } else if key == KEY_COMMITTED_CLOSE { + recover.committed_close_clock = decode_clock(value, "committed-close-clock")?; + Ok(()) + } else if key == KEY_CONNECTOR_STATE { + set_bytes_singleton(&mut recover.connector_state_json, value, "connector-state") + } else if key == KEY_HINTED_CLOSE { + recover.hinted_close_clock = decode_clock(value, "hinted-close-clock")?; + Ok(()) + } else if key == KEY_TRIGGER_PARAMS { + set_bytes_singleton(&mut recover.trigger_params_json, value, "trigger-params") + } else if key == KEY_LAST_APPLIED { + set_bytes_singleton(&mut recover.last_applied, value, "last-applied") + } else if key == KEY_LEGACY_CHECKPOINT { + set_checkpoint(&mut recover.legacy_checkpoint, value) + } else { + Ok(()) + } +} + +fn decode_clock(value: &[u8], kind: &'static str) -> Result { + let bytes: [u8; 8] = value + .try_into() + .map_err(|_| DecodeError::InvalidClockValue { + kind, + got: value.len(), + })?; + Ok(u64::from_le_bytes(bytes)) +} + +fn set_bytes_singleton( + slot: &mut Bytes, + value: &[u8], + kind: &'static str, +) -> Result<(), DecodeError> { + // `slot.is_empty()` is symmetric with `encode_persist`, which skips + // emitting Puts for empty values. + if !slot.is_empty() { + Err(DecodeError::DuplicateSingleton { kind }) + } else { + *slot = Bytes::copy_from_slice(value); + Ok(()) + } +} + +fn set_checkpoint( + slot: &mut Option, + value: &[u8], +) -> Result<(), DecodeError> { + if slot.is_some() { + return Err(DecodeError::DuplicateSingleton { kind: "checkpoint" }); + } + *slot = Some(consumer::Checkpoint::decode(value).map_err(DecodeError::CheckpointValueDecode)?); + Ok(()) +} + +/// Binary-search `binding_state_keys` for `state_key`, returning the +/// associated binding index if present. +fn lookup_binding(binding_state_keys: &[(String, u32)], state_key: &str) -> Option { + binding_state_keys + .binary_search_by(|probe| probe.0.as_str().cmp(state_key)) + .ok() + .map(|i| binding_state_keys[i].1) +} + +fn decode_frontier_entry( + target: &mut Vec, + rest: &[u8], + value: &[u8], + binding_state_keys: &[(String, u32)], +) -> Result<(), DecodeError> { + // Layout: journal \0 state_key \0 producer[6] + if rest.len() < 6 + 2 { + return Err(DecodeError::MalformedFrontierKey { + reason: "too short", + }); + } + let (head, producer_bytes) = rest.split_at(rest.len() - 6); + // `head` should end with the NUL that separates state_key from the + // producer id. Strip it so the remaining bytes are `journal \0 state_key`. + let head = head + .strip_suffix(b"\0") + .ok_or(DecodeError::MalformedFrontierKey { + reason: "missing NUL before producer id", + })?; + + let sep = head + .iter() + .position(|b| *b == 0) + .ok_or(DecodeError::MalformedFrontierKey { + reason: "missing state_key separator", + })?; + let (journal, after) = head.split_at(sep); + let state_key = &after[1..]; + if state_key.contains(&0) { + return Err(DecodeError::MalformedFrontierKey { + reason: "stray NUL in state_key", + }); + } + let journal = std::str::from_utf8(journal).map_err(DecodeError::InvalidUtf8)?; + let state_key = std::str::from_utf8(state_key).map_err(DecodeError::InvalidUtf8)?; + + let Some(binding) = lookup_binding(binding_state_keys, state_key) else { + return Ok(()); // Backfilled or removed binding — discard. + }; + let binding = binding as u16; + + let proto = shuffle::proto::ProducerFrontier::decode(value) + .map_err(DecodeError::FrontierValueDecode)?; + // The encoder stores the producer id only in the key; rehydrate it here. + let key_id: [u8; 6] = producer_bytes.try_into().unwrap(); + + let producer = shuffle::ProducerFrontier { + producer: uuid::Producer::from_bytes(key_id), + last_commit: uuid::Clock::from_u64(proto.last_commit), + hinted_commit: uuid::Clock::from_u64(proto.hinted_commit), + offset: proto.offset, + }; + + // RocksDB iterates in byte-sort order, so consecutive FH/FC entries for + // the same (journal, state_key) — and therefore (journal, binding) — + // arrive together. + if let Some(last) = target.last_mut() + && last.journal.as_ref() == journal + && last.binding == binding + { + last.producers.push(producer); + } else { + target.push(shuffle::JournalFrontier { + journal: journal.into(), + binding, + producers: vec![producer], + bytes_read_delta: 0, + bytes_behind_delta: 0, + }); + } + + Ok(()) +} + +#[cfg(test)] +mod test { + use super::*; + + fn producer_id(tag: u8) -> [u8; 6] { + [0x01, tag, 0, 0, 0, 0] + } + + fn producer_frontier( + tag: u8, + last_commit: u64, + offset: i64, + ) -> shuffle::proto::ProducerFrontier { + shuffle::proto::ProducerFrontier { + producer: uuid::Producer::from_bytes(producer_id(tag)).as_i64(), + last_commit, + hinted_commit: 0, + offset, + } + } + + // Two journals, two bindings, with the second journal name delta-encoded + // against the first. + fn frontier_fixture() -> shuffle::proto::Frontier { + shuffle::proto::Frontier { + journals: vec![ + shuffle::proto::JournalFrontier { + journal_name_suffix: "acme/events/000".into(), + binding: 0, + producers: vec![ + producer_frontier(0xaa, 100, 250), + producer_frontier(0xbb, 90, -300), + ], + ..Default::default() + }, + shuffle::proto::JournalFrontier { + journal_name_truncate_delta: 3, + journal_name_suffix: "001".into(), + binding: 1, + producers: vec![producer_frontier(0xcc, 50, -50)], + ..Default::default() + }, + ], + ..Default::default() + } + } + + fn frontier_key(prefix: &[u8], journal: &[u8], state_key: &[u8], producer: [u8; 6]) -> Bytes { + let mut k = Vec::from(prefix); + k.extend_from_slice(journal); + k.push(0); + k.extend_from_slice(state_key); + k.push(0); + k.extend_from_slice(&producer); + Bytes::from(k) + } + + fn prefixed(prefix: &[u8], tail: &[u8]) -> Bytes { + let mut k = Vec::from(prefix); + k.extend_from_slice(tail); + Bytes::from(k) + } + + fn max_keys_map(entries: &[(u32, &'static [u8])]) -> std::collections::BTreeMap { + entries + .iter() + .map(|(k, v)| (*k, Bytes::from_static(v))) + .collect() + } + + /// Sorted `Vec<(state_key, binding_index)>` mapping for + /// `decode_recover_key_value`. + fn state_key_index(entries: &[(&str, u32)]) -> Vec<(String, u32)> { + let mut v: Vec<(String, u32)> = entries + .iter() + .map(|(sk, idx)| ((*sk).to_string(), *idx)) + .collect(); + v.sort_by(|a, b| a.0.cmp(&b.0)); + v + } + + #[derive(Debug)] + struct DecodedRecover { + recover: proto::Recover, + #[allow(dead_code)] // Read by insta's Debug snapshot. + committed_frontier: Vec, + hinted_frontier: Vec, + } + + /// Drive a (key, value) iterable through `decode_recover_key_value`. + fn decode_pairs( + pairs: I, + binding_state_keys: &[(String, u32)], + ) -> Result + where + I: IntoIterator, + { + let mut recover = proto::Recover::default(); + let mut committed_frontier = Vec::new(); + let mut hinted_frontier = Vec::new(); + for (k, v) in pairs { + decode_recover_key_value( + &mut recover, + &mut committed_frontier, + &mut hinted_frontier, + &k, + &v, + binding_state_keys, + )?; + } + Ok(DecodedRecover { + recover, + committed_frontier, + hinted_frontier, + }) + } + + fn ack_intents_map( + entries: &[(&str, &'static [u8])], + ) -> std::collections::BTreeMap { + entries + .iter() + .map(|(j, v)| (j.to_string(), Bytes::from_static(v))) + .collect() + } + + fn checkpoint_fixture() -> consumer::Checkpoint { + consumer::Checkpoint { + ack_intents: ack_intents_map(&[("acme/events/000", b"legacy-ack")]), + ..Default::default() + } + } + + #[test] + fn encode_persist_snapshots() { + // Fixture strings stand in for `state_key`s resolved by binding index. + let binding_state_keys: &[&str] = &["materialize/mat/t1", "materialize/mat/t2"]; + let cases: Vec<(&str, proto::Persist)> = vec![ + ("empty", proto::Persist::default()), + // Maximal one-shot commit pins op ordering across every populated field. + ( + "one_shot_commit", + proto::Persist { + delete_hinted_frontier: true, + hinted_frontier: Some(frontier_fixture()), + committed_frontier: Some(frontier_fixture()), + connector_patches_json: Bytes::from_static(b"[{\"cursor\":\"abc\"}\n]"), + max_keys: max_keys_map(&[(0, b"packed-1"), (1, b"packed-2")]), + delete_ack_intents: true, + ack_intents: ack_intents_map(&[("acme/events/000", b"ack-bytes-A")]), + trigger_params_json: Bytes::from_static(b"{\"run_id\":\"r1\"}"), + last_applied: Bytes::from_static(b"spec-proto-bytes"), + ..Default::default() + }, + ), + ( + "clocks_and_legacy_checkpoint", + proto::Persist { + committed_close_clock: 123, + hinted_close_clock: 456, + legacy_checkpoint: Some(checkpoint_fixture()), + ..Default::default() + }, + ), + // committed_frontier without the AI: prelude: the new proto + // decouples delete_ack_intents from committed_frontier. + ( + "committed_no_acks", + proto::Persist { + committed_frontier: Some(frontier_fixture()), + ..Default::default() + }, + ), + // Standalone ack clear: DeleteRange alone, no Put follow-up. + ( + "delete_ack_alone", + proto::Persist { + delete_ack_intents: true, + ..Default::default() + }, + ), + ( + "standalone_trigger_delete", + proto::Persist { + connector_patches_json: Bytes::from_static(b"[{\"idle\":true}\n]"), + delete_trigger_params: true, + ..Default::default() + }, + ), + // delete_trigger_params combined with a fresh Put: the codec emits + // Delete then Put so the final state is the new trigger value. + ( + "trigger_delete_then_put", + proto::Persist { + delete_trigger_params: true, + trigger_params_json: Bytes::from_static(b"{\"run_id\":\"r2\"}"), + ..Default::default() + }, + ), + ]; + + let snapshot: Vec<(&str, Vec)> = cases + .into_iter() + .map(|(name, p)| { + let mut ops = Vec::new(); + encode_persist(&p, binding_state_keys, |op| ops.push(op)).unwrap(); + (name, ops) + }) + .collect(); + insta::assert_debug_snapshot!(snapshot); + } + + #[test] + fn encode_persist_hinted_then_committed_roundtrip() { + // Encode a hinted batch followed by a committed batch, replay both + // through an in-memory store, and round-trip the result through the + // decoder. + let persist1 = proto::Persist { + delete_hinted_frontier: true, + hinted_frontier: Some(frontier_fixture()), + max_keys: max_keys_map(&[(0, b"mk-v1")]), + ..Default::default() + }; + let persist2 = proto::Persist { + committed_frontier: Some(frontier_fixture()), + connector_patches_json: Bytes::from_static( + b"[{\"a\":1}\n,{\"b\":null}\n,{\"c\":[1,2]}\n]", + ), + delete_ack_intents: true, + ack_intents: ack_intents_map(&[ + ("acme/events/000", b"ack-000"), + ("acme/events/001", b"ack-001"), + ]), + delete_trigger_params: true, + ..Default::default() + }; + + let binding_state_keys = &["materialize/mat/t1", "materialize/mat/t2"]; + let mut ops1 = Vec::new(); + encode_persist(&persist1, binding_state_keys, |op| ops1.push(op)).unwrap(); + let mut ops2 = Vec::new(); + encode_persist(&persist2, binding_state_keys, |op| ops2.push(op)).unwrap(); + + let mut store: Vec<(Bytes, Bytes)> = Vec::new(); + for op in ops1.into_iter().chain(ops2.into_iter()) { + apply_op(&mut store, op); + } + store.sort_by(|a, b| a.0.cmp(&b.0)); + + let mapping = state_key_index(&[("materialize/mat/t1", 0), ("materialize/mat/t2", 1)]); + insta::assert_debug_snapshot!(decode_pairs(store, &mapping).unwrap()); + } + + #[test] + fn encode_persist_errors() { + let cases: Vec<(&str, proto::Persist, &[&str], &str)> = vec![( + "unknown_binding", + proto::Persist { + committed_frontier: Some(frontier_fixture()), + ..Default::default() + }, + &["only-one-state-key"], + "UnknownBinding", + )]; + + for (label, persist, state_keys, want) in cases { + let err = encode_persist(&persist, state_keys, |_| {}).unwrap_err(); + assert!( + format!("{err:?}").contains(want), + "{label}: expected {want}, got {err:?}" + ); + } + } + + #[test] + fn decode_recover_classifies_ranges() { + let fh_value = producer_frontier(0xaa, 777, 12345).encode_to_vec(); + let fc_value = producer_frontier(0xbb, 999, 4242).encode_to_vec(); + + let pairs = vec![ + ( + frontier_key( + PREFIX_HINTED_FRONTIER, + b"journal/0", + b"derive/d/binding", + producer_id(0xaa), + ), + Bytes::from(fh_value), + ), + ( + frontier_key( + PREFIX_COMMITTED_FRONTIER, + b"journal/0", + b"derive/d/binding", + producer_id(0xbb), + ), + Bytes::from(fc_value), + ), + ( + Bytes::from_static(KEY_CONNECTOR_STATE), + Bytes::from_static(b"{\"reduced\":true}"), + ), + ( + Bytes::from_static(KEY_TRIGGER_PARAMS), + Bytes::from_static(b"{\"run_id\":\"r\"}"), + ), + ( + Bytes::from_static(KEY_LAST_APPLIED), + Bytes::from_static(b"proto-bytes"), + ), + ( + Bytes::from_static(KEY_COMMITTED_CLOSE), + Bytes::copy_from_slice(&123_u64.to_le_bytes()), + ), + ( + Bytes::from_static(KEY_HINTED_CLOSE), + Bytes::copy_from_slice(&456_u64.to_le_bytes()), + ), + ( + Bytes::from_static(KEY_LEGACY_CHECKPOINT), + Bytes::from(checkpoint_fixture().encode_to_vec()), + ), + ( + prefixed(PREFIX_MAX_KEY, b"derive/d/binding"), + Bytes::from_static(b"pk"), + ), + ( + prefixed(PREFIX_ACK_INTENT, b"j1"), + Bytes::from_static(b"ack"), + ), + (Bytes::from_static(b"unknown-key"), Bytes::from_static(b"!")), + ]; + + let mapping = state_key_index(&[("derive/d/binding", 0)]); + insta::assert_debug_snapshot!(decode_pairs(pairs, &mapping).unwrap()); + } + + #[test] + fn decode_recover_drops_unknown_state_keys() { + // FH:/FC: and MK-v2: entries whose state_key is not in the + // current binding mapping are silently discarded — they belong + // to backfilled or removed bindings. + let fh = producer_frontier(0xaa, 1, 0).encode_to_vec(); + let pairs = vec![ + ( + frontier_key( + PREFIX_HINTED_FRONTIER, + b"j", + b"removed-binding", + producer_id(0xaa), + ), + Bytes::from(fh), + ), + ( + prefixed(PREFIX_MAX_KEY, b"removed-binding"), + Bytes::from_static(b"pk"), + ), + ]; + let decoded = decode_pairs(pairs, &state_key_index(&[("kept-binding", 0)])).unwrap(); + assert!(decoded.hinted_frontier.is_empty()); + assert!(decoded.recover.max_keys.is_empty()); + } + + #[test] + fn decode_recover_errors() { + let valid_value = Bytes::from(producer_frontier(0xaa, 1, 0).encode_to_vec()); + + // FH:/FC: layout: rest = journal \0 state_key \0 producer[6]. + #[allow(clippy::type_complexity)] + let cases: Vec<(&str, Vec<(Bytes, Bytes)>, &str)> = vec![ + ( + "fh_too_short", + vec![( + prefixed(PREFIX_HINTED_FRONTIER, b"abc"), + valid_value.clone(), + )], + "too short", + ), + ( + "fh_missing_nul_before_producer", + // 9-byte rest: head = "abc" (no trailing \0), producer = 6 bytes. + vec![( + prefixed(PREFIX_HINTED_FRONTIER, b"abc\xaa\xaa\xaa\xaa\xaa\xaa"), + valid_value.clone(), + )], + "missing NUL before producer id", + ), + ( + "fh_missing_state_key_separator", + // head = "abc\0", strip → "abc", no inner \0. + vec![( + prefixed(PREFIX_HINTED_FRONTIER, b"abc\0\xaa\xaa\xaa\xaa\xaa\xaa"), + valid_value.clone(), + )], + "missing state_key separator", + ), + ( + "fh_stray_nul_in_state_key", + // head = "j\0sk\0extra\0" → strip → "j\0sk\0extra"; state_key contains \0. + vec![( + prefixed( + PREFIX_HINTED_FRONTIER, + b"j\0sk\0extra\0\xaa\xaa\xaa\xaa\xaa\xaa", + ), + valid_value.clone(), + )], + "stray NUL in state_key", + ), + ( + "fh_value_decode_failure", + // Single byte 0x80 is an incomplete varint and fails prost decoding. + vec![( + frontier_key(PREFIX_HINTED_FRONTIER, b"j", b"sk", producer_id(0xaa)), + Bytes::from_static(b"\x80"), + )], + "FrontierValueDecode", + ), + ( + "ai_invalid_utf8", + vec![(prefixed(PREFIX_ACK_INTENT, b"\xff\xfe"), Bytes::new())], + "InvalidUtf8", + ), + ( + "duplicate_connector_state", + vec![ + ( + Bytes::from_static(KEY_CONNECTOR_STATE), + Bytes::from_static(b"{}"), + ), + ( + Bytes::from_static(KEY_CONNECTOR_STATE), + Bytes::from_static(b"{}"), + ), + ], + "connector-state", + ), + ]; + + // Mapping must include the state_keys exercised by the FH/FC fixtures + // (else those errors are masked by silent drop). + let mapping = state_key_index(&[("sk", 0)]); + for (label, pairs, want) in cases { + let err = decode_pairs(pairs, &mapping).unwrap_err(); + assert!( + format!("{err:?}").contains(want), + "{label}: expected {want}, got {err:?}" + ); + } + } + + // Apply a KeyOp to an in-memory sorted store, respecting DeleteRange. + // Merge is treated as append-with-newline so the round-trip snapshot sees + // the framed accumulation; real RocksDB would reduce via the merge operator. + fn apply_op(store: &mut Vec<(Bytes, Bytes)>, op: KeyOp) { + match op { + KeyOp::Put { key, value } => { + store.retain(|(k, _)| k != &key); + store.push((key, value)); + } + KeyOp::Merge { key, value } => { + if let Some(existing) = store.iter_mut().find(|(k, _)| k == &key) { + let mut merged = Vec::with_capacity(existing.1.len() + 1 + value.len()); + merged.extend_from_slice(&existing.1); + merged.push(b'\n'); + merged.extend_from_slice(&value); + existing.1 = Bytes::from(merged); + } else { + store.push((key, value)); + } + } + KeyOp::Delete { key } => { + store.retain(|(k, _)| k != &key); + } + KeyOp::DeleteRange { from, to } => { + store.retain(|(k, _)| !(k.as_ref() >= from.as_ref() && k.as_ref() < to.as_ref())); + } + } + } +} diff --git a/crates/runtime-next/src/shard/rocksdb.rs b/crates/runtime-next/src/shard/rocksdb.rs new file mode 100644 index 00000000000..1a7d166c5f4 --- /dev/null +++ b/crates/runtime-next/src/shard/rocksdb.rs @@ -0,0 +1,859 @@ +use super::recovery; +use crate::proto; +use anyhow::Context; +use proto_flow::runtime::RocksDbDescriptor; +use tokio::runtime; + +/// RocksDB database used for task state. +pub struct RocksDB { + db: rocksdb::DB, + _tmp: Option, +} + +impl RocksDB { + /// Open a RocksDB from an optional descriptor. + pub async fn open(desc: Option) -> anyhow::Result { + let (opts, path, _tmp) = unpack_descriptor(desc)?; + + let db = runtime::Handle::current() + .spawn_blocking(move || Self::open_blocking(opts, path)) + .await + .unwrap()?; + + Ok(Self { db, _tmp }) + } + + fn open_blocking( + mut opts: rocksdb::Options, + path: std::path::PathBuf, + ) -> anyhow::Result { + // RocksDB requires that all column families be explicitly passed in on open + // or it will fail. We don't currently use column families, but have in the + // past and may in the future. Flexibly open the DB by explicitly listing, + // opening, and then ignoring column families we don't care about. + let column_families = match rocksdb::DB::list_cf(&opts, &path) { + Ok(cf) => cf, + // Listing column families will fail if the DB doesn't exist. + // Assume as such, as we'll otherwise fail when we attempt to open. + Err(_) => vec![rocksdb::DEFAULT_COLUMN_FAMILY_NAME.to_string()], + }; + tracing::debug!(column_families=?ops::DebugJson(&column_families), "listed existing rocksdb column families"); + + let mut cf_descriptors = Vec::with_capacity(column_families.len()); + for name in column_families { + let mut cf_opts = rocksdb::Options::default(); + + if name == rocksdb::DEFAULT_COLUMN_FAMILY_NAME { + let state_schema = doc::reduce::merge_patch_schema(); + + set_json_schema_merge_operator( + &mut cf_opts, + &task_state_default_json_schema(&state_schema).to_string(), + )?; + } + cf_descriptors.push(rocksdb::ColumnFamilyDescriptor::new(name, cf_opts)); + } + + opts.create_if_missing(true); + opts.create_missing_column_families(true); + + let db = rocksdb::DB::open_cf_descriptors(&opts, &path, cf_descriptors) + .context("failed to open RocksDB")?; + + Ok(db) + } + + /// Perform an async write_opt using a blocking background thread. + async fn write_opt( + self, + wb: rocksdb::WriteBatch, + wo: rocksdb::WriteOptions, + ) -> Result { + runtime::Handle::current() + .spawn_blocking(move || { + self.db.write_opt(wb, &wo)?; + Ok(self) + }) + .await + .unwrap() + } + + /// Encode a [`proto::Persist`] into a WriteBatch and synchronously write it. + /// `binding_state_keys[i]` is the stable `state_key` for binding index `i`. + pub async fn persist>( + self, + persist: &proto::Persist, + binding_state_keys: &[S], + ) -> anyhow::Result { + let mut wb = rocksdb::WriteBatch::default(); + recovery::encode_persist(persist, binding_state_keys, |op| match op { + recovery::KeyOp::Put { key, value } => wb.put(&key, &value), + recovery::KeyOp::Merge { key, value } => wb.merge(&key, &value), + recovery::KeyOp::Delete { key } => wb.delete(&key), + recovery::KeyOp::DeleteRange { from, to } => wb.delete_range(&from, &to), + }) + .context("encoding Persist into WriteBatch")?; + + let mut wo = rocksdb::WriteOptions::new(); + wo.set_sync(true); + + self.write_opt(wb, wo) + .await + .context("RocksDB Persist write") + } + + /// Scan the entire DB into a [`proto::Recover`] using a blocking + /// background thread. Returns `(self, Recover)`. + /// + /// `binding_state_keys` is a sorted slice of `(state_key, binding_index)` + /// tuples used to map from stable `state_key` to current binding index. + pub async fn scan( + self, + binding_state_keys: Vec<(String, u32)>, + ) -> Result<(Self, proto::Recover), recovery::DecodeError> { + debug_assert!( + binding_state_keys.windows(2).all(|w| w[0].0 < w[1].0), + "binding_state_keys must be sorted by state_key for binary_search" + ); + runtime::Handle::current() + .spawn_blocking(move || { + let mut recover = proto::Recover::default(); + let mut committed_frontier: Vec = Vec::new(); + let mut hinted_frontier: Vec = Vec::new(); + + let mut it = self.db.raw_iterator(); + it.seek_to_first(); + + while let Some((key, value)) = it.item() { + recovery::decode_recover_key_value( + &mut recover, + &mut committed_frontier, + &mut hinted_frontier, + key, + value, + &binding_state_keys, + )?; + it.next(); + } + // Check final status for iteration errors. + () = it.status()?; + std::mem::drop(it); + + for (frontier, slot) in [ + (&mut committed_frontier, &mut recover.committed_frontier), + (&mut hinted_frontier, &mut recover.hinted_frontier), + ] { + // Mapping from state-key to binding index means journal frontiers are unordered. + frontier + .sort_by(|a, b| a.journal.cmp(&b.journal).then(a.binding.cmp(&b.binding))); + + *slot = + (!frontier.is_empty()).then(|| shuffle::JournalFrontier::encode(&frontier)); + } + + Ok((self, recover)) + }) + .await + .unwrap() + } +} + +// Unpack a RocksDbDescriptor into its rocksdb::Options and path. +// If the descriptor does not include an explicit path, a TempDir to use is +// created and returned for the caller to keep alive alongside the DB. +fn unpack_descriptor( + desc: Option, +) -> anyhow::Result<( + rocksdb::Options, + std::path::PathBuf, + Option, +)> { + Ok(match desc { + Some(RocksDbDescriptor { + rocksdb_path, + rocksdb_env_memptr, + }) => { + tracing::debug!( + ?rocksdb_path, + ?rocksdb_env_memptr, + "opening hooked RocksDB database" + ); + let mut opts = rocksdb::Options::default(); + + if rocksdb_env_memptr != 0 { + // Re-hydrate the provided memory address into rocksdb::Env wrapping + // an owned *mut librocksdb_sys::rocksdb_env_t. + let env = unsafe { + rocksdb::Env::from_raw(rocksdb_env_memptr as *mut librocksdb_sys::rocksdb_env_t) + }; + opts.set_env(&env); + } + (opts, std::path::PathBuf::from(rocksdb_path), None) + } + None => { + let dir = tempfile::TempDir::new().context("failed to create RocksDB tempdir")?; + let opts = rocksdb::Options::default(); + + tracing::debug!( + rocksdb_path = ?dir.path(), + "opening temporary RocksDB database" + ); + + (opts, dir.path().to_owned(), Some(dir)) + } + }) +} + +// RocksDB merge operator schema which uses `state_schema` for keys matching "connector-state". +fn task_state_default_json_schema(state_schema: &serde_json::Value) -> serde_json::Value { + // KEY_CONNECTOR_STATE is `&[u8]`; render it as a string so the JSON-schema + // `const` matches the string literal at index 0 of the [key, doc] array. + let key_str = std::str::from_utf8(recovery::KEY_CONNECTOR_STATE) + .expect("KEY_CONNECTOR_STATE is valid UTF-8"); + + serde_json::json!({ + "oneOf": [ + { + "items": [ + {"const": key_str}, + state_schema, + ] + } + ], + "reduce": {"strategy": "merge"} + }) +} + +// Set a reduction merge operator using the given `schema`. +fn set_json_schema_merge_operator(opts: &mut rocksdb::Options, schema: &str) -> anyhow::Result<()> { + // Check that we can build a validator for `schema`. + let bundle = doc::validation::build_bundle(schema.as_bytes())?; + let _validator = doc::Validator::new(bundle)?; + + let schema_1 = schema.to_owned(); + let schema_2 = schema.to_owned(); + + let full_merge_fn = move |key: &[u8], + initial: Option<&[u8]>, + operands: &rocksdb::merge_operator::MergeOperands| + -> Option> { + match do_merge_rocks(true, initial, key, operands, &schema_1) { + Ok(ok) => Some(ok), + Err(err) => { + tracing::error!(%err, "error within RocksDB full-merge operator"); + if cfg!(debug_assertions) { + eprintln!("(debug) full-merge error: {err:?}"); + } + None + } + } + }; + let partial_merge_fn = move |key: &[u8], + initial: Option<&[u8]>, + operands: &rocksdb::merge_operator::MergeOperands| + -> Option> { + match do_merge_rocks(false, initial, key, operands, &schema_2) { + Ok(ok) => Some(ok), + Err(err) => { + tracing::error!(%err, "error within RocksDB partial-merge operator"); + if cfg!(debug_assertions) { + eprintln!("(debug) partial-merge error: {err:?}"); + } + None + } + } + }; + opts.set_merge_operator("json-schema", full_merge_fn, partial_merge_fn); + + Ok(()) +} + +fn do_merge_rocks( + full: bool, + initial: Option<&[u8]>, + key: &[u8], + operands: &rocksdb::merge_operator::MergeOperands, + schema: &str, +) -> anyhow::Result> { + // Collect all input documents (initial + operands, each newline-delimited). + let mut inputs: Vec<&[u8]> = Vec::new(); + if let Some(initial) = initial { + for doc_bytes in initial.split(|c| *c == b'\n') { + inputs.push(doc_bytes); + } + } + for op in operands { + for doc_bytes in op.split(|c| *c == b'\n') { + inputs.push(doc_bytes); + } + } + + do_merge(full, key, inputs, schema, 32 * 1024 * 1024, usize::MAX) +} + +/// Iteratively reduce `inputs` in batches, re-merging batch outputs until a +/// single result remains. `batch_byte_target` is the initial allocator memory per batch, +/// and `batch_op_target` is the initial documents per batch (useful for testing). +fn do_merge( + full: bool, + key: &[u8], + inputs: Vec<&[u8]>, + schema: &str, + mut batch_byte_target: usize, + mut batch_op_target: usize, +) -> anyhow::Result> { + const MAX_BYTE_THRESHOLD: usize = 1 << 30; // 1 GiB + + // Shadow `inputs` to decouple the reference lifetime from the caller, + // allowing reassignment to borrow from local `input_storage` in the loop. + let mut inputs: Vec<&[u8]> = inputs; + let mut input_storage: Vec>; + + let mut iteration = 0usize; + let mut prev_batches = usize::MAX; + + loop { + let mut offset = 0; + let mut outputs: Vec> = Vec::new(); + + // Only the first batch of each iteration uses `full` reduction. + // The initial/base value is always in the first batch. Subsequent batches + // contain only operands and must use associative (non-full) reduction to + // preserve null deletion markers in merge-patch schemas. + let mut is_first_batch = true; + + // Process all inputs in batches constrained to `byte_threshold` and `max_count`. + while offset != inputs.len() { + let batch_full = full && is_first_batch; + is_first_batch = false; + + let (output, consumed) = do_merge_bounded( + batch_full, + key, + &inputs[offset..], + schema, + batch_byte_target, + batch_op_target, + )?; + + outputs.push(output); + offset += consumed; + } + + tracing::debug!( + iteration, + inputs = inputs.len(), + batches = outputs.len(), + prev_batches, + batch_byte_target, + "do_merge iteration complete" + ); + + // Stop when we have a single batch output. + if outputs.len() <= 1 { + return Ok(outputs.into_iter().next().unwrap_or_default()); + } + + // Batch count didn't decrease: double thresholds to make progress. + // Bail if both thresholds are already at their caps — we've exhausted + // our escalation budget and still can't converge. + if outputs.len() >= prev_batches { + if batch_byte_target >= MAX_BYTE_THRESHOLD && batch_op_target >= 1_024 * 1_024 { + anyhow::bail!( + "merge operation failed to converge \ + (batch_byte_target {batch_byte_target}, batch_op_target {batch_op_target}, \ + iteration {iteration}, batches {}, input_docs {})", + outputs.len(), + inputs.len() + ); + } + batch_byte_target = batch_byte_target.saturating_mul(2).min(MAX_BYTE_THRESHOLD); + batch_op_target = batch_op_target.saturating_mul(2); + } + + // Iterate again with batch outputs (which are a newline-separated + // remainder of non-associative merge operands) as new inputs. + // The first output descends from the original base value, so + // `is_first_batch` will correctly be `true` for it on the next iteration. + prev_batches = outputs.len(); + input_storage = outputs; + inputs = input_storage + .iter() + .flat_map(|batch| batch.split(|c| *c == b'\n')) + .collect(); + + iteration += 1; + } +} + +/// Process documents from a slice into a MemTable until `byte_threshold` is +/// reached, `max_count` documents are consumed, or all `inputs` are consumed. +/// Returns newline-separated reduced documents and the count consumed. +fn do_merge_bounded( + full: bool, + key: &[u8], + inputs: &[&[u8]], + schema: &str, + byte_threshold: usize, + max_count: usize, +) -> anyhow::Result<(Vec, usize)> { + let bundle = doc::validation::build_bundle(schema.as_bytes())?; + let validator = doc::Validator::new(bundle)?; + let spec = + doc::combine::Spec::with_one_binding(full, [], "connector state", Vec::new(), validator); + let memtable = doc::combine::MemTable::new(spec); + + let key = String::from_utf8_lossy(key); + let key = doc::BumpStr::from_str(&key, memtable.alloc()); + let mut consumed = 0usize; + + for op_bytes in inputs { + let mut de = serde_json::Deserializer::from_slice(op_bytes); + let op = doc::HeapNode::from_serde(&mut de, memtable.alloc()).with_context(|| { + format!( + "couldn't parse document as JSON: {}", + String::from_utf8_lossy(op_bytes) + ) + })?; + + let doc = doc::HeapNode::new_array( + memtable.alloc(), + [doc::HeapNode::String(key), op].into_iter(), + ); + memtable.add(0, doc, false)?; + consumed += 1; + + let bytes_used = memtable + .alloc() + .allocated_bytes() + .saturating_sub(memtable.alloc().chunk_capacity()); + + if bytes_used > byte_threshold || consumed >= max_count { + break; + } + } + + let mut out = Vec::new(); + for (index, drained) in memtable.try_into_drainer()?.enumerate() { + let doc::combine::DrainedDoc { meta: _, root } = drained?; + let doc::OwnedNode::Heap(root) = root else { + unreachable!() + }; + + if index != 0 { + out.push(b'\n'); + } + + // Extract the document (index 1) from [key, doc] arrays. + // When a full reduction with a null merge-patch operand fires + // the LWW delete flag, the value position is removed from the + // array. In that case the semantic result is JSON null. + let Ok(doc::HeapNode::Array(_, array)) = root.access() else { + unreachable!() + }; + match array.get(1) { + Some(node) => { + serde_json::to_writer(&mut out, &doc::SerPolicy::noop().on(node)) + .expect("serialization cannot fail"); + } + None => out.extend_from_slice(b"null"), + } + } + + Ok((out, consumed)) +} + +#[cfg(test)] +mod test { + use super::*; + + fn test_schema() -> String { + let state_schema = doc::reduce::merge_patch_schema(); + task_state_default_json_schema(&state_schema).to_string() + } + + /// Compute expected result via `json_patch::merge`, then verify `do_merge` + /// matches for both full and partial-then-full merges at each batch size. + fn check_merge(base: &str, ops: &[&str], max_counts: &[usize]) { + let schema = test_schema(); + let key = recovery::KEY_CONNECTOR_STATE; + + let mut expected: serde_json::Value = serde_json::from_str(base).unwrap(); + for op in ops { + json_patch::merge(&mut expected, &serde_json::from_str(op).unwrap()); + } + + let op_bytes: Vec<&[u8]> = ops.iter().map(|o| o.as_bytes()).collect(); + let mut all_inputs: Vec<&[u8]> = vec![base.as_bytes()]; + all_inputs.extend_from_slice(&op_bytes); + + for &mc in max_counts { + // Full merge. + let result = do_merge(true, key, all_inputs.clone(), &schema, usize::MAX, mc).unwrap(); + let actual: serde_json::Value = serde_json::from_slice(&result).unwrap(); + assert_eq!(actual, expected, "full merge at max_count={mc}"); + + // Partial merge of just operands, then full merge with base. + let partial = do_merge(false, key, op_bytes.clone(), &schema, usize::MAX, mc).unwrap(); + let mut final_inputs: Vec<&[u8]> = vec![base.as_bytes()]; + for doc in partial.split(|c| *c == b'\n') { + final_inputs.push(doc); + } + let result = + do_merge(true, key, final_inputs, &schema, usize::MAX, usize::MAX).unwrap(); + let actual: serde_json::Value = serde_json::from_slice(&result).unwrap(); + assert_eq!(actual, expected, "partial+full merge at max_count={mc}"); + } + } + + #[tokio::test] + async fn connector_state_merge() { + let mut wb = rocksdb::WriteBatch::default(); + let db = RocksDB::open(None).await.unwrap(); + + for doc in [ + r#"{"a":"b","n":null}"#, + r#"{"a":"c","nn":null}"#, + r#"{"d":"e","ans":42}"#, + ] { + wb.merge(recovery::KEY_CONNECTOR_STATE, doc); + } + let db = db.write_opt(wb, Default::default()).await.unwrap(); + + let (_db, state) = db.scan(Vec::new()).await.unwrap(); + assert_eq!( + state.connector_state_json, + bytes::Bytes::from_static(br#"{"a":"c","ans":42,"d":"e","n":null}"#) + ); + } + + #[tokio::test] + async fn scan_returns_recover_with_sorted_frontier_chunks() { + let db = RocksDB::open(None).await.unwrap(); + let producer = proto_gazette::uuid::Producer::from_bytes([0x01, 0xaa, 0, 0, 0, 0]); + + let frontier = shuffle::JournalFrontier::encode(&[ + shuffle::JournalFrontier { + journal: "journal/shared".into(), + binding: 1, + producers: vec![shuffle::ProducerFrontier { + producer, + last_commit: proto_gazette::uuid::Clock::from_u64(11), + hinted_commit: proto_gazette::uuid::Clock::default(), + offset: 101, + }], + bytes_read_delta: 0, + bytes_behind_delta: 0, + }, + shuffle::JournalFrontier { + journal: "journal/shared".into(), + binding: 0, + producers: vec![shuffle::ProducerFrontier { + producer, + last_commit: proto_gazette::uuid::Clock::from_u64(22), + hinted_commit: proto_gazette::uuid::Clock::default(), + offset: 202, + }], + bytes_read_delta: 0, + bytes_behind_delta: 0, + }, + ]); + + let db = db + .persist( + &proto::Persist { + committed_frontier: Some(frontier.clone()), + hinted_frontier: Some(frontier), + ..Default::default() + }, + &["z-binding", "a-binding"], + ) + .await + .unwrap(); + let mapping = vec![("a-binding".to_string(), 1), ("z-binding".to_string(), 0)]; + let (_db, recover) = db.scan(mapping).await.unwrap(); + + let committed: Vec<_> = recover + .committed_frontier + .clone() + .into_iter() + .flat_map(shuffle::JournalFrontier::decode) + .collect(); + let hinted: Vec<_> = recover + .hinted_frontier + .clone() + .into_iter() + .flat_map(shuffle::JournalFrontier::decode) + .collect(); + + assert_eq!(committed.len(), 2); + assert_eq!(hinted.len(), 2); + assert_eq!(committed[0].binding, 0); + assert_eq!(committed[1].binding, 1); + assert_eq!(hinted[0].binding, 0); + assert_eq!(hinted[1].binding, 1); + } + + /// Verify merge batching handles many operands that would exceed memory + /// threshold. Connectors may emit many small merge-patch updates, and this + /// can turn into many merge operands (hundreds of thousands). We handle this + /// through iterative merges of batches of inputs, iteratively reducing until + /// convergence. + #[tokio::test] + async fn test_merge_many_operands_batched() { + // Uncomment to initialize tracing and see merge operator logs. + /* + let _ = tracing_subscriber::fmt() + .with_env_filter("runtime=debug") + .with_writer(std::io::stderr) + .try_init(); + */ + + let db = RocksDB::open(None).await.unwrap(); + + // Generate many merge operations that will exceed the batch memory threshold. + // Each document is a merge-patch updating a unique key in a "cursors" object, + // similar to how some connectors track per-partition state. + let num_operands = 50_000; + let mut wb = rocksdb::WriteBatch::default(); + + for i in 0..num_operands { + // Each merge-patch sets a unique cursor key with some payload. + // The payload size ensures we'll exceed the 32MB initial threshold and trigger batching. + let doc = format!( + r#"{{"cursors":{{"partition_{:05}":{{"offset":{},"timestamp":"2024-01-01T00:00:00Z","metadata":"padding_to_increase_document_size_xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx"}}}}}}"#, + i, + i * 1000 + ); + wb.merge(recovery::KEY_CONNECTOR_STATE, &doc); + } + let db = db.write_opt(wb, Default::default()).await.unwrap(); + + // Force a compaction to trigger the merge operator with all operands at once. + // This is what happens during recovery or when RocksDB decides to compact. + db.db.compact_range::<&[u8], &[u8]>(None, None); + + // Load the state - this will also trigger a full merge if not already compacted. + let (_db, state) = db.scan(Vec::new()).await.unwrap(); + + // Verify the merged state contains all cursor entries. + let parsed: serde_json::Value = + serde_json::from_slice(&state.connector_state_json).unwrap(); + let cursors = parsed["cursors"] + .as_object() + .expect("cursors should be an object"); + assert_eq!(cursors.len(), num_operands); + + // Spot-check a few entries. + assert_eq!(cursors["partition_00000"]["offset"], 0); + assert_eq!(cursors["partition_00100"]["offset"], 100_000); + assert_eq!(cursors["partition_49999"]["offset"], 49_999_000); + } + + /// Null deletion markers must survive batching. Historical bug: when a non-first + /// batch used `full=true`, LWW delete stripped null markers within the batch + /// instead of preserving them for re-merge with the base value. + /// Also verifies the partial merge path preserves null markers. + #[test] + fn test_null_deletion_survives_batching() { + // batch1=[base,op1], batch2=[op2,op3_null] at max_count=2. + check_merge( + r#"{"a":1,"x":10}"#, + &[r#"{"b":2}"#, r#"{"a":5}"#, r#"{"a":null}"#], + &[2, 3, usize::MAX], + ); + // Multiple interleaved deletions across batch boundaries. + check_merge( + r#"{"a":1,"b":2,"c":3}"#, + &[ + r#"{"e":5}"#, + r#"{"a":5}"#, + r#"{"a":null}"#, + r#"{"b":5}"#, + r#"{"b":null}"#, + ], + &[2, 3, usize::MAX], + ); + } + + /// Quickcheck for merge-patch-relevant JSON: small key alphabet to force collisions, + /// high null frequency to exercise deletion markers, and controlled nesting depth. + #[derive(Clone, Debug)] + struct MergePatchValue(serde_json::Value); + + impl quickcheck::Arbitrary for MergePatchValue { + fn arbitrary(g: &mut quickcheck::Gen) -> Self { + Self(gen_merge_patch_value(g, 3)) + } + + fn shrink(&self) -> Box> { + match &self.0 { + serde_json::Value::Object(map) => { + let entries: Vec<(String, MergePatchValue)> = map + .iter() + .map(|(k, v)| (k.clone(), MergePatchValue(v.clone()))) + .collect(); + Box::new(entries.shrink().map(|es| { + let map: serde_json::Map = + es.into_iter().map(|(k, v)| (k, v.0)).collect(); + MergePatchValue(serde_json::Value::Object(map)) + })) + } + serde_json::Value::Null => quickcheck::empty_shrinker(), + _ => Box::new(std::iter::once(MergePatchValue(serde_json::Value::Null))), + } + } + } + + fn gen_range(g: &mut quickcheck::Gen, range: std::ops::Range) -> u64 { + ::arbitrary(g) % (range.end - range.start) + range.start + } + + fn gen_merge_patch_value(g: &mut quickcheck::Gen, depth: usize) -> serde_json::Value { + let choices = if depth > 0 { 10 } else { 7 }; + match gen_range(g, 0..choices) { + 0 | 1 | 2 => serde_json::Value::Null, // ~30% null + 3 => serde_json::Value::Bool(::arbitrary(g)), + 4 => serde_json::json!(::arbitrary(g)), + 5 => serde_json::json!(::arbitrary(g)), + 6 => serde_json::json!([::arbitrary(g)]), + _ => { + let keys = ["a", "b", "c", "d", "e"]; + let num_keys = gen_range(g, 1..6) as usize; + let mut map = serde_json::Map::new(); + for _ in 0..num_keys { + let ki = gen_range(g, 0..keys.len() as u64) as usize; + map.insert(keys[ki].to_string(), gen_merge_patch_value(g, depth - 1)); + } + serde_json::Value::Object(map) + } + } + } + + #[derive(Clone, Debug)] + struct MergePatchSequence { + base: MergePatchValue, + operands: Vec, + } + + impl quickcheck::Arbitrary for MergePatchSequence { + fn arbitrary(g: &mut quickcheck::Gen) -> Self { + let n = gen_range(g, 1..21) as usize; + // Connector state base is always a JSON object. + let gen_obj = |g: &mut quickcheck::Gen| loop { + let v = gen_merge_patch_value(g, 3); + if v.is_object() { + return MergePatchValue(v); + } + }; + // Operands can be any merge-patch value, including scalars and + // nulls that trigger NotAssociative reductions during batching. + Self { + base: gen_obj(g), + operands: (0..n).map(|_| MergePatchValue::arbitrary(g)).collect(), + } + } + + fn shrink(&self) -> Box> { + let base = self.base.clone(); + let operands = self.operands.clone(); + Box::new(operands.shrink().map(move |ops| MergePatchSequence { + base: base.clone(), + operands: ops, + })) + } + } + + /// Fuzz: `do_merge` at various batch sizes must match `json_patch::merge`, + /// for both full merges and partial-then-full merges. + #[test] + fn fuzz_batched_merge_matches_reference() { + fn prop(seq: MergePatchSequence) -> bool { + let schema = test_schema(); + let key = recovery::KEY_CONNECTOR_STATE; + + let mut expected = seq.base.0.clone(); + for op in &seq.operands { + json_patch::merge(&mut expected, &op.0); + } + + let base_bytes = serde_json::to_vec(&seq.base.0).unwrap(); + let op_bytes: Vec> = seq + .operands + .iter() + .map(|op| serde_json::to_vec(&op.0).unwrap()) + .collect(); + + let mut all_inputs: Vec<&[u8]> = vec![&base_bytes]; + all_inputs.extend(op_bytes.iter().map(|v| v.as_slice())); + let ops_only: Vec<&[u8]> = op_bytes.iter().map(|v| v.as_slice()).collect(); + + for mc in [2, 3, 5, usize::MAX] { + // Full merge. + let result = + do_merge(true, key, all_inputs.clone(), &schema, usize::MAX, mc).unwrap(); + if serde_json::from_slice::(&result).unwrap() != expected { + return false; + } + + if ops_only.is_empty() { + continue; + } + + // Partial merge of operands, then full merge with base. + let partial = + do_merge(false, key, ops_only.clone(), &schema, usize::MAX, mc).unwrap(); + let mut final_inputs: Vec<&[u8]> = vec![&base_bytes]; + for doc in partial.split(|c| *c == b'\n') { + final_inputs.push(doc); + } + let result = + do_merge(true, key, final_inputs, &schema, usize::MAX, usize::MAX).unwrap(); + if serde_json::from_slice::(&result).unwrap() != expected { + return false; + } + } + true + } + + quickcheck::QuickCheck::new() + .tests(1000) + .max_tests(2000) + .quickcheck(prop as fn(MergePatchSequence) -> bool); + } + + #[tokio::test] + async fn round_trip_persist_stack_to_scan() { + let mut db = RocksDB::open(None).await.unwrap(); + + for persist in [ + crate::proto::Persist { + nonce: 1, + ack_intents: [("j/A".to_string(), bytes::Bytes::from_static(b"INTENT-A"))] + .into_iter() + .collect(), + ..Default::default() + }, + crate::proto::Persist { + nonce: 2, + ack_intents: [("j/B".to_string(), bytes::Bytes::from_static(b"INTENT-B"))] + .into_iter() + .collect(), + ..Default::default() + }, + crate::proto::Persist { + nonce: 99, + last_applied: bytes::Bytes::from_static(b"v9"), + ..Default::default() + }, + ] { + db = db.persist::<&str>(&persist, &[]).await.unwrap(); + } + + let (_db, state) = db.scan(Vec::new()).await.unwrap(); + assert_eq!(state.ack_intents.len(), 2); + assert_eq!(state.ack_intents.get("j/A").unwrap().as_ref(), b"INTENT-A"); + assert_eq!(state.ack_intents.get("j/B").unwrap().as_ref(), b"INTENT-B"); + assert_eq!(state.last_applied.as_ref(), b"v9"); + } +} diff --git a/crates/runtime-next/src/shard/service.rs b/crates/runtime-next/src/shard/service.rs new file mode 100644 index 00000000000..301564caba3 --- /dev/null +++ b/crates/runtime-next/src/shard/service.rs @@ -0,0 +1,116 @@ +//! Top-level Shard service implementation. +//! +//! `Service` directly implements the controller-facing `Shard` trait. +//! "Controller" here is the peer that drives the shard's lifecycle: the +//! Go runtime in production, an in-process driver such as `flowctl +//! preview`, or a unit-test harness. From this crate's perspective the +//! controller is just the peer of the bidi stream that commands the +//! runtime and bounds its lifecycle. + +use crate::proto; +use futures::Stream; +use tokio::sync::mpsc; +use tokio_stream::wrappers; + +/// Service is the implementation of the controller-facing `Shard` gRPC +/// service trait, hosting one shard's transaction loop. +#[derive(Clone)] +pub struct Service { + pub plane: crate::Plane, + pub container_network: String, + pub log_handler: L, + pub set_log_level: Option>, + pub task_name: String, + pub publisher_factory: gazette::journal::ClientFactory, +} + +impl Service { + /// Build a new Shard Service. + /// - `plane`: the type of data plane in which this Service is operating. + /// - `container_network`: the Docker container network used for connector containers. + /// - `log_handler`: handler to which connector logs are dispatched. + /// - `set_log_level`: callback for adjusting the log level implied by runtime requests. + /// - `task_name`: name which is used to label any started connector containers. + /// - `publisher_factory`: client factory for creating and appending to collection partitions. + pub fn new( + plane: crate::Plane, + container_network: String, + log_handler: L, + set_log_level: Option>, + task_name: String, + publisher_factory: gazette::journal::ClientFactory, + ) -> Self { + Self { + plane, + container_network, + log_handler, + set_log_level, + task_name, + publisher_factory, + } + } + + /// Wrap this service in its typed tonic server, for composition + /// with sibling services on a `tonic::transport::Server::builder()`. + pub fn into_tonic_service(self) -> proto_grpc::runtime::shard_server::ShardServer { + proto_grpc::runtime::shard_server::ShardServer::new(self) + .max_decoding_message_size(crate::MAX_MESSAGE_SIZE) + .max_encoding_message_size(usize::MAX) + } + + /// Apply the dynamic log level if a setter was provided. + pub fn set_log_level(&self, level: ops::LogLevel) { + if level == ops::LogLevel::UndefinedLevel { + // No-op + } else if let Some(set_log_level) = &self.set_log_level { + (set_log_level)(level); + } + } + + pub fn spawn_materialize( + &self, + controller_rx: R, + ) -> mpsc::UnboundedReceiver> + where + R: Stream> + Send + Unpin + 'static, + { + let service = self.clone(); + let (controller_tx, response_rx) = + mpsc::unbounded_channel::>(); + let error_tx = controller_tx.clone(); + + tokio::spawn(async move { + if let Err(err) = super::materialize::serve(service, controller_rx, controller_tx).await + { + let _ = error_tx.send(Err(crate::anyhow_to_status(err))); + } + }); + response_rx + } +} + +#[tonic::async_trait] +impl proto_grpc::runtime::shard_server::Shard for Service { + type MaterializeStream = wrappers::UnboundedReceiverStream>; + //type DeriveStream = wrappers::ReceiverStream>; + + async fn materialize( + &self, + request: tonic::Request>, + ) -> tonic::Result> { + Ok(tonic::Response::new( + wrappers::UnboundedReceiverStream::new(self.spawn_materialize(request.into_inner())), + )) + } + + /* + async fn derive( + &self, + _request: tonic::Request>, + ) -> tonic::Result> { + Err(tonic::Status::unimplemented( + "Shard.Derive: not in scope for the materialize phase", + )) + } + */ +} diff --git a/crates/runtime-next/src/shard/snapshots/runtime_next__shard__recovery__test__decode_recover_classifies_ranges.snap b/crates/runtime-next/src/shard/snapshots/runtime_next__shard__recovery__test__decode_recover_classifies_ranges.snap new file mode 100644 index 00000000000..cca32411692 --- /dev/null +++ b/crates/runtime-next/src/shard/snapshots/runtime_next__shard__recovery__test__decode_recover_classifies_ranges.snap @@ -0,0 +1,62 @@ +--- +source: crates/runtime-next/src/shard/recovery.rs +assertion_line: 947 +expression: "decode_pairs(pairs, &mapping).unwrap()" +--- +DecodedRecover { + recover: Recover { + ack_intents: { + "j1": b"ack", + }, + committed_close_clock: 123, + committed_frontier: None, + connector_state_json: b"{\"reduced\":true}", + hinted_close_clock: 456, + hinted_frontier: None, + last_applied: b"proto-bytes", + legacy_checkpoint: Some( + Checkpoint { + sources: {}, + ack_intents: { + "acme/events/000": b"legacy-ack", + }, + }, + ), + max_keys: { + 0: b"pk", + }, + trigger_params_json: b"{\"run_id\":\"r\"}", + }, + committed_frontier: [ + JournalFrontier { + journal: "journal/0", + binding: 0, + producers: [ + ProducerFrontier { + producer: Producer(01:bb:00:00:00:00), + last_commit: Clock(0s 28ns), + hinted_commit: Clock(0s 0ns), + offset: 4242, + }, + ], + bytes_read_delta: 0, + bytes_behind_delta: 0, + }, + ], + hinted_frontier: [ + JournalFrontier { + journal: "journal/0", + binding: 0, + producers: [ + ProducerFrontier { + producer: Producer(01:aa:00:00:00:00), + last_commit: Clock(0s 36ns), + hinted_commit: Clock(0s 0ns), + offset: 12345, + }, + ], + bytes_read_delta: 0, + bytes_behind_delta: 0, + }, + ], +} diff --git a/crates/runtime-next/src/shard/snapshots/runtime_next__shard__recovery__test__encode_persist_hinted_then_committed_roundtrip.snap b/crates/runtime-next/src/shard/snapshots/runtime_next__shard__recovery__test__encode_persist_hinted_then_committed_roundtrip.snap new file mode 100644 index 00000000000..04127c05bd1 --- /dev/null +++ b/crates/runtime-next/src/shard/snapshots/runtime_next__shard__recovery__test__encode_persist_hinted_then_committed_roundtrip.snap @@ -0,0 +1,96 @@ +--- +source: crates/runtime-next/src/shard/recovery.rs +assertion_line: 797 +expression: "decode_pairs(store, &mapping).unwrap()" +--- +DecodedRecover { + recover: Recover { + ack_intents: { + "acme/events/000": b"ack-000", + "acme/events/001": b"ack-001", + }, + committed_close_clock: 0, + committed_frontier: None, + connector_state_json: b"{\"a\":1}\n{\"b\":null}\n{\"c\":[1,2]}", + hinted_close_clock: 0, + hinted_frontier: None, + last_applied: b"", + legacy_checkpoint: None, + max_keys: { + 0: b"mk-v1", + }, + trigger_params_json: b"", + }, + committed_frontier: [ + JournalFrontier { + journal: "acme/events/000", + binding: 0, + producers: [ + ProducerFrontier { + producer: Producer(01:aa:00:00:00:00), + last_commit: Clock(0s 16ns), + hinted_commit: Clock(0s 0ns), + offset: 250, + }, + ProducerFrontier { + producer: Producer(01:bb:00:00:00:00), + last_commit: Clock(0s 40ns), + hinted_commit: Clock(0s 0ns), + offset: -300, + }, + ], + bytes_read_delta: 0, + bytes_behind_delta: 0, + }, + JournalFrontier { + journal: "acme/events/001", + binding: 1, + producers: [ + ProducerFrontier { + producer: Producer(01:cc:00:00:00:00), + last_commit: Clock(0s 8ns), + hinted_commit: Clock(0s 0ns), + offset: -50, + }, + ], + bytes_read_delta: 0, + bytes_behind_delta: 0, + }, + ], + hinted_frontier: [ + JournalFrontier { + journal: "acme/events/000", + binding: 0, + producers: [ + ProducerFrontier { + producer: Producer(01:aa:00:00:00:00), + last_commit: Clock(0s 16ns), + hinted_commit: Clock(0s 0ns), + offset: 250, + }, + ProducerFrontier { + producer: Producer(01:bb:00:00:00:00), + last_commit: Clock(0s 40ns), + hinted_commit: Clock(0s 0ns), + offset: -300, + }, + ], + bytes_read_delta: 0, + bytes_behind_delta: 0, + }, + JournalFrontier { + journal: "acme/events/001", + binding: 1, + producers: [ + ProducerFrontier { + producer: Producer(01:cc:00:00:00:00), + last_commit: Clock(0s 8ns), + hinted_commit: Clock(0s 0ns), + offset: -50, + }, + ], + bytes_read_delta: 0, + bytes_behind_delta: 0, + }, + ], +} diff --git a/crates/runtime-next/src/shard/snapshots/runtime_next__shard__recovery__test__encode_persist_snapshots.snap b/crates/runtime-next/src/shard/snapshots/runtime_next__shard__recovery__test__encode_persist_snapshots.snap new file mode 100644 index 00000000000..15ab86ecc13 --- /dev/null +++ b/crates/runtime-next/src/shard/snapshots/runtime_next__shard__recovery__test__encode_persist_snapshots.snap @@ -0,0 +1,139 @@ +--- +source: crates/runtime-next/src/shard/recovery.rs +assertion_line: 758 +expression: snapshot +--- +[ + ( + "empty", + [], + ), + ( + "one_shot_commit", + [ + DeleteRange { + from: b"AI:", + to: b"AI;", + }, + Put { + key: b"AI:acme/events/000", + value: b"ack-bytes-A", + }, + Put { + key: b"FC:acme/events/000\0materialize/mat/t1\0\x01\xaa\0\0\0\0", + value: b"\x11d\0\0\0\0\0\0\0 \xfa\x01", + }, + Put { + key: b"FC:acme/events/000\0materialize/mat/t1\0\x01\xbb\0\0\0\0", + value: b"\x11Z\0\0\0\0\0\0\0 \xd4\xfd\xff\xff\xff\xff\xff\xff\xff\x01", + }, + Put { + key: b"FC:acme/events/001\0materialize/mat/t2\0\x01\xcc\0\0\0\0", + value: b"\x112\0\0\0\0\0\0\0 \xce\xff\xff\xff\xff\xff\xff\xff\xff\x01", + }, + Merge { + key: b"connector-state", + value: b"{\"cursor\":\"abc\"}", + }, + DeleteRange { + from: b"FH:", + to: b"FH;", + }, + Put { + key: b"FH:acme/events/000\0materialize/mat/t1\0\x01\xaa\0\0\0\0", + value: b"\x11d\0\0\0\0\0\0\0 \xfa\x01", + }, + Put { + key: b"FH:acme/events/000\0materialize/mat/t1\0\x01\xbb\0\0\0\0", + value: b"\x11Z\0\0\0\0\0\0\0 \xd4\xfd\xff\xff\xff\xff\xff\xff\xff\x01", + }, + Put { + key: b"FH:acme/events/001\0materialize/mat/t2\0\x01\xcc\0\0\0\0", + value: b"\x112\0\0\0\0\0\0\0 \xce\xff\xff\xff\xff\xff\xff\xff\xff\x01", + }, + Put { + key: b"last-applied", + value: b"spec-proto-bytes", + }, + Put { + key: b"MK-v2:materialize/mat/t1", + value: b"packed-1", + }, + Put { + key: b"MK-v2:materialize/mat/t2", + value: b"packed-2", + }, + Put { + key: b"trigger-params", + value: b"{\"run_id\":\"r1\"}", + }, + ], + ), + ( + "clocks_and_legacy_checkpoint", + [ + Put { + key: b"committed-close", + value: b"{\0\0\0\0\0\0\0", + }, + Put { + key: b"hinted-close", + value: b"\xc8\x01\0\0\0\0\0\0", + }, + Put { + key: b"checkpoint", + value: b"\x12\x1d\n\x0facme/events/000\x12\nlegacy-ack", + }, + ], + ), + ( + "committed_no_acks", + [ + Put { + key: b"FC:acme/events/000\0materialize/mat/t1\0\x01\xaa\0\0\0\0", + value: b"\x11d\0\0\0\0\0\0\0 \xfa\x01", + }, + Put { + key: b"FC:acme/events/000\0materialize/mat/t1\0\x01\xbb\0\0\0\0", + value: b"\x11Z\0\0\0\0\0\0\0 \xd4\xfd\xff\xff\xff\xff\xff\xff\xff\x01", + }, + Put { + key: b"FC:acme/events/001\0materialize/mat/t2\0\x01\xcc\0\0\0\0", + value: b"\x112\0\0\0\0\0\0\0 \xce\xff\xff\xff\xff\xff\xff\xff\xff\x01", + }, + ], + ), + ( + "delete_ack_alone", + [ + DeleteRange { + from: b"AI:", + to: b"AI;", + }, + ], + ), + ( + "standalone_trigger_delete", + [ + Merge { + key: b"connector-state", + value: b"{\"idle\":true}", + }, + Delete { + key: b"trigger-params", + }, + ], + ), + ( + "trigger_delete_then_put", + [ + Delete { + key: b"trigger-params", + }, + Put { + key: b"trigger-params", + value: b"{\"run_id\":\"r2\"}", + }, + ], + ), +] diff --git a/crates/runtime-next/src/task_service.rs b/crates/runtime-next/src/task_service.rs new file mode 100644 index 00000000000..bdddee39797 --- /dev/null +++ b/crates/runtime-next/src/task_service.rs @@ -0,0 +1,134 @@ +//! CGO entry point: binds a UDS, registers the `Shard` gRPC service, and +//! serves until cancellation. +use crate::{proto, shard}; +use anyhow::Context; +use base64::Engine; +use futures::FutureExt; +use futures::channel::oneshot; + +pub struct TaskService { + cancel_tx: oneshot::Sender<()>, + tokio_context: crate::TokioContext, + server: tokio::task::JoinHandle>, +} + +impl TaskService { + pub fn new(config: proto::TaskServiceConfig, log_file: std::fs::File) -> anyhow::Result { + let proto::TaskServiceConfig { + log_file_fd: _, + task_name, + uds_path, + container_network, + plane, + } = config; + + if !std::path::Path::new(&uds_path).is_absolute() { + anyhow::bail!("uds_path must be an absolute filesystem path"); + } + + // Data-plane configuration variables: + let data_plane_fqdn = + std::env::var("FLOW_DATA_PLANE_FQDN").context("FLOW_DATA_PLANE_FQDN not set")?; + let control_api_endpoint = + std::env::var("FLOW_CONTROL_API").context("FLOW_CONTROL_API not set")?; + let availability_zone = std::env::var("ZONE").unwrap_or_else(|_| "local".to_string()); + let data_plane_signing_key = first_consumer_auth_key()?; + + let log_handler = ::ops::new_encoded_json_write_handler(std::sync::Arc::new( + std::sync::Mutex::new(log_file), + )); + let tokio_context = crate::TokioContext::new( + ops::LogLevel::Warn, + log_handler.clone(), + task_name.clone(), + 1, + ); + + let control_api_endpoint: url::Url = + url::Url::parse(&control_api_endpoint).context("invalid control API endpoint URL")?; + + let publisher_factory = + flow_client_next::workflows::task_collection_auth::new_journal_client_factory( + flow_client_next::rest::Client::new(&control_api_endpoint, "task-service"), + proto_gazette::capability::APPEND | proto_gazette::capability::APPLY, + gazette::Router::new(&availability_zone), + data_plane_fqdn, + tokens::jwt::EncodingKey::from_secret(&data_plane_signing_key), + ); + + std::mem::drop(data_plane_signing_key); + + let shard_svc = shard::Service::new( + crate::proto::Plane::try_from(plane).context("invalid TaskServiceConfig.plane")?, + container_network, + log_handler, + Some(tokio_context.set_log_level_fn()), + task_name, + publisher_factory, + ); + + let uds = tokio_context + .block_on(async move { tokio::net::UnixListener::bind(uds_path) }) + .context("failed to bind task service unix domain socket")?; + let (cancel_tx, cancel_rx) = oneshot::channel::<()>(); + + let uds_stream = futures::stream::try_unfold(uds, move |uds| async move { + let (conn, addr) = uds.accept().await?; + tracing::debug!(?addr, "accepted new unix socket connection"); + Ok::<_, std::io::Error>(Some((conn, uds))) + }); + + let server = tonic::transport::Server::builder() + .add_service(shard_svc.into_tonic_service()) + .serve_with_incoming_shutdown(uds_stream, async move { + _ = cancel_rx.await; + }); + let server = tokio_context.spawn(server); + + Ok(Self { + cancel_tx, + tokio_context, + server, + }) + } + + pub fn graceful_stop(self) { + let Self { + cancel_tx, + tokio_context, + server, + } = self; + + _ = cancel_tx.send(()); + + let log = match tokio_context.block_on(server) { + Err(panic) => async move { + tracing::error!(?panic, "task gRPC service exited with panic"); + } + .boxed(), + Ok(Err(error)) => async move { + tracing::error!(?error, "task gRPC service exited with error"); + } + .boxed(), + Ok(Ok(())) => async move { + tracing::debug!("task gRPC service stopped gracefully"); + } + .boxed(), + }; + let () = tokio_context.block_on(tokio_context.spawn(log)).unwrap(); + } +} + +// Decode the first key from `CONSUMER_AUTH_KEYS`, matching Gazette's +// `auth.NewKeyedAuth` parsing: comma- or whitespace-separated, base64-encoded +// keys; the first key signs. +fn first_consumer_auth_key() -> anyhow::Result> { + let raw = std::env::var("CONSUMER_AUTH_KEYS").context("CONSUMER_AUTH_KEYS not set")?; + let first = raw + .split(|c: char| c == ',' || c.is_whitespace()) + .find(|s| !s.is_empty()) + .context("CONSUMER_AUTH_KEYS is empty")?; + base64::engine::general_purpose::STANDARD + .decode(first) + .context("CONSUMER_AUTH_KEYS first key is not valid base64") +} diff --git a/crates/runtime-next/src/tokio_context.rs b/crates/runtime-next/src/tokio_context.rs new file mode 100644 index 00000000000..f0c81648898 --- /dev/null +++ b/crates/runtime-next/src/tokio_context.rs @@ -0,0 +1,139 @@ +use proto_flow::ops; +use std::ops::{Deref, DerefMut}; +use std::sync::Arc; +use std::sync::atomic::{AtomicI32, Ordering}; +use tracing_subscriber::prelude::*; + +/// TokioContext manages a tokio::Runtime that names its threads under a given thread name, +/// and forwards its tracing events to a provided log handler. +pub struct TokioContext { + runtime: Option, + set_log_level_fn: Arc, +} + +impl TokioContext { + /// Build a new TokioContext and associated tokio::Runtime, + /// having the `thread_name_prefix` and `worker_threads`. + /// Threads of the context are initialized with a tracing Subscriber + /// configured with `initial_log_level`. + pub fn new( + initial_log_level: ops::log::Level, + log_handler: L, + thread_name_prefix: String, + worker_threads: usize, + ) -> Self + where + L: Fn(&ops::Log) + Send + Sync + 'static, + { + // Map the input thread name into unique thread names suffixed with their millisecond start time. + let thread_name_fn = move || { + let millis = std::time::SystemTime::now() + .duration_since(std::time::UNIX_EPOCH) + .unwrap() + .as_millis(); + format!("{thread_name_prefix}-{}", millis) + }; + + // Dynamically configurable ops::log::Level, as a shared atomic. + let log_level = std::sync::Arc::new(AtomicI32::new(initial_log_level as i32)); + + // Function closure which allows for changing the dynamic log level. + let log_level_clone = log_level.clone(); + let set_log_level = Arc::new(move |level: ops::log::Level| { + log_level_clone.store(level as i32, Ordering::Relaxed) + }); + + // Build a tracing_subscriber::Filter which uses our dynamic log level. + let log_filter = tracing_subscriber::filter::DynFilterFn::new(move |metadata, _cx| { + let cur_level = match metadata.level().as_str() { + "TRACE" => ops::log::Level::Trace as i32, + "DEBUG" => ops::log::Level::Debug as i32, + "INFO" => ops::log::Level::Info as i32, + "WARN" => ops::log::Level::Warn as i32, + "ERROR" => ops::log::Level::Error as i32, + _ => ops::log::Level::UndefinedLevel as i32, + }; + + if let Some(path) = metadata.module_path() { + // Hyper / HTTP/2 debug logs are just too noisy and not very useful. + if path.starts_with("h2::") && cur_level >= ops::log::Level::Debug as i32 { + return false; + } + } + + cur_level <= log_level.load(Ordering::Relaxed) + }); + + // Configure a tracing::Dispatch, which is a type-erased form of a tracing::Subscriber, + // that gathers tracing events & spans and logs them to `log_handler`. + let log_dispatch: tracing::Dispatch = tracing_subscriber::registry() + .with( + ::ops::tracing::Layer::new(log_handler, std::time::SystemTime::now) + .with_filter(log_filter), + ) + .into(); + + let runtime = tokio::runtime::Builder::new_multi_thread() + .worker_threads(worker_threads) + .enable_all() + .thread_name_fn(thread_name_fn) + .on_thread_start(move || { + let guard = tracing::dispatcher::set_default(&log_dispatch); + Self::DISPATCH_GUARD.with(|cell| cell.set(Some(guard))); + }) + .on_thread_stop(|| { + Self::DISPATCH_GUARD.with(|cell| cell.take()); + }) + .build() + .unwrap(); + + Self { + runtime: Some(runtime), + set_log_level_fn: set_log_level, + } + } + + /// Return a function closure which dynamically updates the configured log level for tracing events. + pub fn set_log_level_fn(&self) -> Arc { + self.set_log_level_fn.clone() + } + + thread_local!(static DISPATCH_GUARD: std::cell::Cell> = std::cell::Cell::new(None)); +} + +impl Deref for TokioContext { + type Target = tokio::runtime::Runtime; + fn deref(&self) -> &Self::Target { + self.runtime.as_ref().unwrap() + } +} +impl DerefMut for TokioContext { + fn deref_mut(&mut self) -> &mut Self::Target { + self.runtime.as_mut().unwrap() + } +} + +impl Drop for TokioContext { + fn drop(&mut self) { + let rt = self.runtime.take().unwrap(); + let duration = std::time::Duration::from_secs(30); + + // Ask the runtime to shutdown, providing a bounded duration to do so. + // We want to give it a reasonable chance to complete spawned tasks, + // notably because async-process spawns a shutdown task via Drop. + // + // However, we don't want to wait indefinitely (as Runtime::Drop does), + // because there isn't a guarantee that blocking background tasks will + // ever complete (consider a blocking read from stdin: tokio::io maps + // AsyncRead of file descriptors to blocking tasks under the hood). + // + // If we're within another tokio Runtime, we must spawn a blocking task + // to perform the actual shutdown, or we'll block a current async task + // (and in practice, tokio will panic). + if let Ok(parent) = tokio::runtime::Handle::try_current() { + parent.spawn_blocking(move || rt.shutdown_timeout(duration)); + } else { + rt.shutdown_timeout(duration) + } + } +} diff --git a/crates/runtime/src/harness/materialize.rs b/crates/runtime/src/harness/materialize.rs index 3539adc2bc3..4517ad1cd4d 100644 --- a/crates/runtime/src/harness/materialize.rs +++ b/crates/runtime/src/harness/materialize.rs @@ -164,7 +164,9 @@ async fn run_session( // Send initial Acknowledge of the session. request_tx .try_send(Ok(Request { - acknowledge: Some(request::Acknowledge {}), + acknowledge: Some(request::Acknowledge { + connector_state_patches_json: bytes::Bytes::new(), // Not implemented. + }), ..Default::default() })) .expect("sender is empty"); @@ -233,7 +235,9 @@ async fn run_session( // Send Flush. let flush = Request { - flush: Some(request::Flush {}), + flush: Some(request::Flush { + connector_state_patches_json: bytes::Bytes::new(), // Not implemented. + }), ..Default::default() }; () = crate::exchange(Ok(flush), request_tx, response_rx) @@ -253,6 +257,7 @@ async fn run_session( .try_send(Ok(Request { start_commit: Some(request::StartCommit { runtime_checkpoint: Some(checkpoint), + connector_state_patches_json: bytes::Bytes::new(), // Not implemented. }), ..Default::default() })) @@ -269,7 +274,9 @@ async fn run_session( // Send Acknowledge. request_tx .try_send(Ok(Request { - acknowledge: Some(request::Acknowledge {}), + acknowledge: Some(request::Acknowledge { + connector_state_patches_json: bytes::Bytes::new(), // Not implemented. + }), ..Default::default() })) .expect("sender is empty"); diff --git a/crates/runtime/src/materialize/protocol.rs b/crates/runtime/src/materialize/protocol.rs index 34f0da9c6c4..3bfaa8d608f 100644 --- a/crates/runtime/src/materialize/protocol.rs +++ b/crates/runtime/src/materialize/protocol.rs @@ -299,7 +299,7 @@ pub fn recv_client_load_or_flush( } } Some(Request { - flush: Some(request::Flush {}), + flush: Some(request::Flush { .. }), .. }) => { if !*saw_acknowledged { @@ -311,7 +311,7 @@ pub fn recv_client_load_or_flush( _ = std::mem::take(load_keys); Ok(Some(Request { - flush: Some(request::Flush {}), + flush: Some(request::Flush::default()), ..Default::default() })) } @@ -578,6 +578,7 @@ pub fn recv_client_start_commit( start_commit: Some(request::StartCommit { runtime_checkpoint: Some(runtime_checkpoint), + .. }), .. } = &request diff --git a/crates/runtime/src/task_service.rs b/crates/runtime/src/task_service.rs index 65a9aa44656..cd2e3aac417 100644 --- a/crates/runtime/src/task_service.rs +++ b/crates/runtime/src/task_service.rs @@ -18,6 +18,7 @@ impl TaskService { uds_path, container_network, plane, + .. } = config; if !std::path::Path::new(&uds_path).is_absolute() { diff --git a/crates/shuffle/README.md b/crates/shuffle/README.md index c24a55bdbfd..c00c47b23e9 100644 --- a/crates/shuffle/README.md +++ b/crates/shuffle/README.md @@ -137,7 +137,7 @@ resume checkpoint frontier. The Session: 2. Opens a Slice RPC to every shard (shard 0 is in-process; others are remote gRPC calls). 3. Sends `Opened` to the coordinator, then reads the resume checkpoint - (streamed as chunked `FrontierChunk` messages). + `Frontier`. 4. Sends `Start` to all Slices, which triggers journal listing watches. ### 2. Journal Discovery @@ -254,8 +254,7 @@ processes the previous checkpoint. The Session maintains one outstanding `Progress` / `Progressed` cycle per Slice. When the Slice has flushed progress available and a Progress -request pending, it sends the accumulated frontier as chunked -`FrontierChunk` messages. +request pending, it sends the accumulated frontier as a `Frontier`. ### 11. Checkpoint Pipeline @@ -276,7 +275,7 @@ guarantees forward progress. Once all hints resolve, the frontier promotes to `ready`. When the coordinator sends `NextCheckpoint` and `ready` is non-empty, the Session -drains it as chunked `FrontierChunk` messages. +sends it as a single `Frontier` message. At startup, `resume_checkpoint` may contain unresolved hints from the previous session. The `recovery_pending` flag gates promotion until the diff --git a/crates/shuffle/src/binding.rs b/crates/shuffle/src/binding.rs index be6f0774ef3..c25fcd03e49 100644 --- a/crates/shuffle/src/binding.rs +++ b/crates/shuffle/src/binding.rs @@ -80,8 +80,8 @@ impl Binding { /// (e.g. on SliceActor) or discard them (e.g. SessionActor). pub fn from_task( task: &shuffle::Task, - ) -> anyhow::Result<(String, Vec, Vec, u64)> { - let (shard_prefix, pairs, disk_backlog_threshold) = match &task.task { + ) -> anyhow::Result<(String, Vec, Vec)> { + let (shard_prefix, pairs) = match &task.task { Some(shuffle::task::Task::Derivation(collection_spec)) => { let derivation = collection_spec .derivation @@ -104,14 +104,7 @@ impl Binding { .id .as_str(); - // TODO: extract from task spec. - let disk_backlog_threshold = 10 * 1024 * 1024 * 1024u64; - - ( - format!("{shard_template_id}/"), - pairs, - disk_backlog_threshold, - ) + (format!("{shard_template_id}/"), pairs) } Some(shuffle::task::Task::Materialization(materialization)) => { let pairs = materialization @@ -130,20 +123,12 @@ impl Binding { .id .as_str(); - // TODO: extract from task spec. - let disk_backlog_threshold = 10 * 1024 * 1024 * 1024u64; - - ( - format!("{shard_template_id}/"), - pairs, - disk_backlog_threshold, - ) + (format!("{shard_template_id}/"), pairs) } Some(shuffle::task::Task::CollectionPartitions(collection_partitions)) => { let shuffle::CollectionPartitions { collection, partition_selector, - disk_backlog_threshold, } = collection_partitions; let collection_spec = collection @@ -162,7 +147,6 @@ impl Binding { ( String::new(), // No applicable shard prefix. pairs, - *disk_backlog_threshold, ) } None => anyhow::bail!("missing task variant"), @@ -173,7 +157,7 @@ impl Binding { assign_cohorts(&mut bindings); - Ok((shard_prefix, bindings, validators, disk_backlog_threshold)) + Ok((shard_prefix, bindings, validators)) } fn from_derivation_transform( diff --git a/crates/shuffle/src/client.rs b/crates/shuffle/src/client.rs index a002495be97..4fb83db4287 100644 --- a/crates/shuffle/src/client.rs +++ b/crates/shuffle/src/client.rs @@ -48,20 +48,15 @@ impl SessionClient { response => return Err(verify.fail(response)), }; - // Send the resume checkpoint (including the empty-chunk terminator). - let mut drain = crate::frontier::Drain::new(); - drain.start(resume_checkpoint); - - while let Some(chunk) = drain.next_chunk() { - // Sends are best-effort: an error here will just be a RST. - // We'll surface a proper causal error later, on next read. - let _: Result<(), _> = request_tx - .send(shuffle::SessionRequest { - resume_checkpoint_chunk: Some(chunk), - ..Default::default() - }) - .await; - } + // Send the resume checkpoint. + // Best-effort: an error here will just be a RST and we'll surface + // a proper causal error later, on next read. + let _: Result<(), _> = request_tx + .send(shuffle::SessionRequest { + resume_checkpoint: Some(resume_checkpoint.encode()), + ..Default::default() + }) + .await; Ok(Self { request_tx, @@ -69,16 +64,11 @@ impl SessionClient { }) } - /// Send NextCheckpoint and collect the complete frontier response. + /// Send NextCheckpoint and read back the frontier response. pub async fn next_checkpoint(&mut self) -> anyhow::Result { tracing::debug!("requesting NextCheckpoint"); - let verify = crate::verify( - "SessionResponse", - "next_checkpoint_chunk", - "(in-process)", - 0, - ); + let verify = crate::verify("SessionResponse", "next_checkpoint", "(in-process)", 0); let _: Result<(), _> = self .request_tx @@ -88,26 +78,20 @@ impl SessionClient { }) .await; - let mut journals = Vec::new(); - let flushed_lsn; - loop { - let chunk = match verify.not_eof(self.response_rx.recv().await)? { - shuffle::SessionResponse { - next_checkpoint_chunk: Some(chunk), - .. - } => chunk, - response => return Err(verify.fail(response)), - }; - - if chunk.journals.is_empty() { - flushed_lsn = chunk.flushed_lsn; - break; - } - journals.extend(crate::JournalFrontier::decode(chunk)); - } + let proto = match verify.not_eof(self.response_rx.recv().await)? { + shuffle::SessionResponse { + next_checkpoint: Some(proto), + .. + } => proto, + response => return Err(verify.fail(response)), + }; - tracing::debug!(journals = journals.len(), "received NextCheckpoint"); - crate::Frontier::new(journals, flushed_lsn).context("validating checkpoint frontier") + let frontier = crate::Frontier::decode(proto).context("validating checkpoint frontier")?; + tracing::debug!( + journals = frontier.journals.len(), + "received NextCheckpoint" + ); + Ok(frontier) } /// Cleanly close the session by dropping the request sender and reading server EOF. diff --git a/crates/shuffle/src/frontier.rs b/crates/shuffle/src/frontier.rs index 4930900d831..758690f743f 100644 --- a/crates/shuffle/src/frontier.rs +++ b/crates/shuffle/src/frontier.rs @@ -155,16 +155,17 @@ impl JournalFrontier { (advanced, resolved) } - /// Decode a proto `FrontierChunk` into an iterator of `JournalFrontier`. + /// Decode a proto `Frontier`'s journals into an iterator of `JournalFrontier`. /// - /// Each chunk is self-contained: the first entry carries the full journal - /// name (truncate=0, suffix=full name), so decoding requires only - /// chunk-internal state. This is a pure mapping with no validation; - /// use `Frontier::new` to validate ordering invariants. - pub fn decode(chunk: shuffle::FrontierChunk) -> impl Iterator { + /// Journal names within the proto are delta-encoded, with the first entry + /// carrying the full journal name (truncate=0, suffix=full name) and + /// subsequent entries delta-encoded relative to their predecessor. + /// Decoding is a pure mapping with no validation; use `Frontier::decode` + /// or `Frontier::new` to validate ordering invariants. + pub fn decode(proto: shuffle::Frontier) -> impl Iterator { let mut journal_name = String::new(); - chunk.journals.into_iter().map(move |jf| { + proto.journals.into_iter().map(move |jf| { gazette::delta::decode( &mut journal_name, jf.journal_name_truncate_delta, @@ -189,12 +190,14 @@ impl JournalFrontier { }) } - /// Encode a slice of `JournalFrontier` entries as a proto `FrontierChunk`. + /// Encode a slice of `JournalFrontier` entries as a proto `Frontier`. /// - /// The chunk is self-contained: the first entry carries the full journal - /// name (truncate=0, suffix=full name), and subsequent entries are - /// delta-encoded relative to their predecessor within the chunk. - pub fn encode(entries: &[Self]) -> shuffle::FrontierChunk { + /// The first entry carries the full journal name (truncate=0, suffix=full + /// name), and subsequent entries are delta-encoded relative to their + /// predecessor. The returned proto's `flushed_lsn` is empty; callers + /// needing it should populate the field directly, or use + /// `Frontier::encode`. + pub fn encode(entries: &[Self]) -> shuffle::Frontier { let mut prev_journal: &str = ""; let journals = entries @@ -224,7 +227,7 @@ impl JournalFrontier { }) .collect(); - shuffle::FrontierChunk { + shuffle::Frontier { journals, flushed_lsn: vec![], } @@ -502,6 +505,22 @@ impl Frontier { (advanced, resolved) } + /// Encode this Frontier as a proto `shuffle::Frontier`, including + /// `flushed_lsn`. Journal names within the proto are delta-encoded — + /// see `JournalFrontier::encode` for the layout. + pub fn encode(&self) -> shuffle::Frontier { + let mut proto = JournalFrontier::encode(&self.journals); + proto.flushed_lsn = self.flushed_lsn.iter().map(|lsn| lsn.as_u64()).collect(); + proto + } + + /// Decode a proto `shuffle::Frontier` into a validated `Frontier`. + pub fn decode(mut proto: shuffle::Frontier) -> Result { + let flushed_lsn = std::mem::take(&mut proto.flushed_lsn); + let journals: Vec = JournalFrontier::decode(proto).collect(); + Self::new(journals, flushed_lsn) + } + /// Extract producers with unresolved causal hints (`hinted_commit > last_commit`) /// into a new Frontier, filtering out journals that have no such producers. /// Used at startup to project read-through state from `resume_checkpoint`. @@ -549,97 +568,6 @@ fn count_unresolved_hints(journals: &[JournalFrontier]) -> usize { .count() } -/// Drains a `Frontier` as a sequence of chunked `FrontierChunk` messages. -/// -/// Call `start` to begin draining a frontier, `is_empty` to check -/// whether chunks remain, and `next_chunk` to produce the next chunk. -/// The final chunk is an empty terminator (no journals). -/// -/// Callers must verify they can act on a chunk (e.g. channel capacity) -/// *before* calling `next_chunk`, which advances internal state. -pub struct Drain { - /// The frontier being drained. Replaced with `Default` once fully consumed. - frontier: Frontier, - /// Index of the next journal to encode. `usize::MAX` means no drain is in progress. - offset: usize, - /// Maximum number of journals per emitted `FrontierChunk`. - journals_per_chunk: usize, -} - -impl Drain { - /// Default number of journals per chunk in production use. - pub const DEFAULT_JOURNALS_PER_CHUNK: usize = 64; - - pub fn new() -> Self { - Self { - frontier: Default::default(), - offset: usize::MAX, - journals_per_chunk: Self::DEFAULT_JOURNALS_PER_CHUNK, - } - } - - /// Create a Drain with a custom journals-per-chunk size, useful for testing. - pub fn with_journals_per_chunk(journals_per_chunk: usize) -> Self { - assert!(journals_per_chunk > 0, "journals_per_chunk must be > 0"); - Self { - frontier: Default::default(), - offset: usize::MAX, - journals_per_chunk, - } - } - - /// Begin draining the given frontier. - /// Panics if a drain is already in progress. - pub fn start(&mut self, frontier: Frontier) { - assert!(self.is_empty(), "cannot start while a drain is in progress"); - self.frontier = frontier; - self.offset = 0; - } - - /// Whether the drain is complete and has no chunks remaining. - pub fn is_empty(&self) -> bool { - self.offset == usize::MAX - } - - /// Produce the next `FrontierChunk`, advancing the drain offset. - /// Returns `None` when no drain is in progress. - /// An empty chunk (no journals) is the end-of-sequence terminator. - pub fn next_chunk(&mut self) -> Option { - if self.offset == usize::MAX { - return None; - } - - let end = (self.offset + self.journals_per_chunk).min(self.frontier.journals.len()); - let mut chunk = JournalFrontier::encode(&self.frontier.journals[self.offset..end]); - - if chunk.journals.is_empty() { - chunk.flushed_lsn = std::mem::take(&mut self.frontier.flushed_lsn) - .into_iter() - .map(|lsn| lsn.as_u64()) - .collect(); - self.frontier = Default::default(); // Release memory. - self.offset = usize::MAX; - } else { - self.offset += chunk.journals.len(); - } - - Some(chunk) - } -} - -impl std::fmt::Debug for Drain { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - if self.offset == usize::MAX { - f.write_str("empty") - } else { - f.debug_struct("Drain") - .field("offset", &self.offset) - .field("journals", &self.frontier.journals.len()) - .finish() - } - } -} - #[cfg(test)] mod test { use super::*; @@ -822,7 +750,7 @@ mod test { } #[test] - fn test_encode_decode_round_trip() { + fn test_journal_frontier_encode_decode_round_trip() { let frontier = Frontier::new( vec![ jf_with_bytes( @@ -859,10 +787,16 @@ mod test { ) .unwrap(); - // Single chunk round-trips correctly. - let chunk = JournalFrontier::encode(&frontier.journals); - assert_eq!(chunk.journals.len(), 5); - let decoded: Vec<_> = JournalFrontier::decode(chunk).collect(); + let proto = JournalFrontier::encode(&frontier.journals); + assert_eq!(proto.journals.len(), 5); + + // The first entry must have truncate=0 and the full journal name + // as suffix; subsequent entries are delta-encoded. + let first = &proto.journals[0]; + assert_eq!(first.journal_name_truncate_delta, 0); + assert_eq!(first.journal_name_suffix, &*frontier.journals[0].journal); + + let decoded: Vec<_> = JournalFrontier::decode(proto).collect(); assert_eq!(decoded.len(), frontier.journals.len()); for (a, b) in decoded.iter().zip(frontier.journals.iter()) { assert_eq!(&*a.journal, &*b.journal); @@ -870,55 +804,12 @@ mod test { assert_eq!(a.bytes_read_delta, b.bytes_read_delta); assert_eq!(a.bytes_behind_delta, b.bytes_behind_delta); } - - // Multi-chunk: each chunk is independently decodable. - for chunk_size in [1, 2, 3] { - let mut reassembled = Vec::new(); - let mut offset = 0; - while offset < frontier.journals.len() { - let end = (offset + chunk_size).min(frontier.journals.len()); - let chunk = JournalFrontier::encode(&frontier.journals[offset..end]); - - // The first entry of each chunk must have truncate=0 - // and the full journal name as suffix. - let first = &chunk.journals[0]; - assert_eq!( - first.journal_name_truncate_delta, 0, - "chunk at offset {offset}: first entry must have truncate=0" - ); - let expected_name = &*frontier.journals[offset].journal; - assert_eq!( - first.journal_name_suffix, expected_name, - "chunk at offset {offset}: first entry suffix must be the full journal name" - ); - - // Each chunk decodes independently (no external state). - reassembled.extend(JournalFrontier::decode(chunk)); - offset = end; - } - - // Reassembled frontier matches the original. - let reassembled = Frontier::new(reassembled, vec![]).unwrap(); - assert_eq!(reassembled.journals.len(), frontier.journals.len()); - for (a, b) in reassembled.journals.iter().zip(frontier.journals.iter()) { - assert_eq!(&*a.journal, &*b.journal, "chunk_size={chunk_size}"); - assert_eq!(a.binding, b.binding, "chunk_size={chunk_size}"); - assert_eq!( - a.bytes_behind_delta, b.bytes_behind_delta, - "chunk_size={chunk_size}" - ); - assert_eq!( - a.bytes_read_delta, b.bytes_read_delta, - "chunk_size={chunk_size}" - ); - } - } } #[test] fn test_encode_empty() { - let chunk = JournalFrontier::encode(&[]); - assert!(chunk.journals.is_empty()); + let proto = JournalFrontier::encode(&[]); + assert!(proto.journals.is_empty()); } #[test] @@ -1176,79 +1067,8 @@ mod test { ); } - fn drain_all(drain: &mut Drain) -> Vec { - std::iter::from_fn(|| drain.next_chunk()) - .map(|c| c.journals.len()) - .collect() - } - - #[test] - fn test_drain_chunking() { - // (journal_count, journals_per_chunk) => expected per-chunk journal counts. - // Every sequence ends with 0 (the empty terminator). - let cases: &[(usize, usize, &[usize])] = &[ - (0, 2, &[0]), // empty frontier - (1, 1, &[1, 0]), // single journal, chunk size 1 - (2, 2, &[2, 0]), // exact boundary - (3, 2, &[2, 1, 0]), // overflow by one - (3, 100, &[3, 0]), // chunk larger than frontier - (5, 2, &[2, 2, 1, 0]), - ]; - - let all_journals: Vec<_> = (0..5) - .map(|i| jf(&format!("journal/{i}"), 0, vec![pf(0x01, 100, 0, -500)])) - .collect(); - - for &(n, chunk_size, expected) in cases { - let mut drain = Drain::with_journals_per_chunk(chunk_size); - drain.start(Frontier { - journals: all_journals[..n].to_vec(), - flushed_lsn: vec![], - unresolved_hints: 0, - }); - assert_eq!( - drain_all(&mut drain), - expected, - "n={n} chunk_size={chunk_size}" - ); - assert!(drain.is_empty()); - } - } - #[test] - fn test_drain_not_started() { - let mut drain = Drain::new(); - assert!(drain.is_empty()); - assert!(drain.next_chunk().is_none()); - } - - #[test] - fn test_drain_reuse() { - let mut drain = Drain::with_journals_per_chunk(10); - for _ in 0..3 { - drain.start(Frontier { - journals: vec![jf("j", 0, vec![pf(0x01, 1, 0, -1)])], - flushed_lsn: vec![], - unresolved_hints: 0, - }); - assert_eq!(drain_all(&mut drain), [1, 0]); - } - } - - #[test] - #[should_panic(expected = "cannot start while a drain is in progress")] - fn test_drain_double_start_panics() { - let mut drain = Drain::with_journals_per_chunk(1); - drain.start(Frontier { - journals: vec![jf("j", 0, vec![pf(0x01, 1, 0, -1)])], - flushed_lsn: vec![], - unresolved_hints: 0, - }); - drain.start(Frontier::default()); - } - - #[test] - fn test_drain_round_trip() { + fn test_frontier_encode_decode_round_trip() { let original = Frontier::new( vec![ jf("journal/A", 0, vec![pf(0x01, 100, 0, -500)]), @@ -1259,31 +1079,28 @@ mod test { ) .unwrap(); - for chunk_size in [1, 2, 3] { - let mut drain = Drain::with_journals_per_chunk(chunk_size); - drain.start(original.clone()); + let proto = original.encode(); + assert_eq!(proto.journals.len(), 3); + assert_eq!(proto.flushed_lsn, vec![100, 200, 300]); - let mut reassembled_journals = Vec::new(); - let mut terminal_flushed_lsn = Vec::new(); - for chunk in std::iter::from_fn(|| drain.next_chunk()) { - if chunk.journals.is_empty() { - terminal_flushed_lsn = chunk.flushed_lsn; - } else { - reassembled_journals.extend(JournalFrontier::decode(chunk)); - } - } - let reassembled = Frontier::new(reassembled_journals, terminal_flushed_lsn).unwrap(); - - assert_eq!(reassembled.journals.len(), original.journals.len()); - for (a, b) in reassembled.journals.iter().zip(original.journals.iter()) { - assert_eq!(&*a.journal, &*b.journal); - assert_eq!(a.binding, b.binding); - assert_eq!(a.producers.len(), b.producers.len()); - } - assert_eq!( - reassembled.flushed_lsn, original.flushed_lsn, - "flushed_lsn round-trips through drain (chunk_size={chunk_size})" - ); + let reassembled = Frontier::decode(proto).unwrap(); + assert_eq!(reassembled.journals.len(), original.journals.len()); + for (a, b) in reassembled.journals.iter().zip(original.journals.iter()) { + assert_eq!(&*a.journal, &*b.journal); + assert_eq!(a.binding, b.binding); + assert_eq!(a.producers.len(), b.producers.len()); } + assert_eq!(reassembled.flushed_lsn, original.flushed_lsn); + } + + #[test] + fn test_frontier_decode_validates() { + // An out-of-order journals proto should fail to decode (validation). + let proto = JournalFrontier::encode(&[ + jf("journal/B", 0, vec![pf(0x01, 1, 0, -1)]), + jf("journal/A", 0, vec![pf(0x01, 1, 0, -1)]), + ]); + let err = Frontier::decode(proto).unwrap_err(); + assert!(format!("{err}").contains("not ordered")); } } diff --git a/crates/shuffle/src/log/handler.rs b/crates/shuffle/src/log/handler.rs index c0e04279430..c0b853cd4e8 100644 --- a/crates/shuffle/src/log/handler.rs +++ b/crates/shuffle/src/log/handler.rs @@ -24,7 +24,6 @@ where shards, slice_shard_index, log_shard_index, - disk_backlog_threshold, } = open.open.context("first message must be Open")?; let directory = shards @@ -38,7 +37,6 @@ where slice_shard_index, log_shard_index, %directory, - disk_backlog_threshold, "Log received Open" ); let join_key = (directory.to_string(), log_shard_index); @@ -120,7 +118,7 @@ where session_id, shards, log_shard_index, - disk_backlog_threshold, + disk_backlog_threshold: service.disk_backlog_threshold, }, append_heap: super::heap::AppendHeap::new(), slice_prev_journal: vec![String::new(); shard_count], diff --git a/crates/shuffle/src/log/writer/mod.rs b/crates/shuffle/src/log/writer/mod.rs index 13773c4da33..abb6d8576b0 100644 --- a/crates/shuffle/src/log/writer/mod.rs +++ b/crates/shuffle/src/log/writer/mod.rs @@ -153,8 +153,14 @@ impl Writer { impl Drop for Writer { fn drop(&mut self) { let path = super::segment_path(&self.directory, self.shard_index, self.next_lsn.segment()); - if let Err(err) = std::fs::remove_file(&path) { - tracing::warn!(%err, ?path, "failed to unlink in-progress writer segment"); + match std::fs::remove_file(&path) { + Ok(()) => tracing::debug!(?path, "unlinked in-progress writer segment"), + Err(err) if err.kind() == std::io::ErrorKind::NotFound => { + tracing::debug!(?path, "lost race to unlink in-progress writer segment"); + } + Err(err) => { + tracing::warn!(%err, ?path, "failed to unlink in-progress writer segment"); + } } } } diff --git a/crates/shuffle/src/service.rs b/crates/shuffle/src/service.rs index 2968a547d34..778e16ec383 100644 --- a/crates/shuffle/src/service.rs +++ b/crates/shuffle/src/service.rs @@ -14,6 +14,9 @@ pub struct ServiceImpl { pub(crate) peer_endpoint: String, /// Factory for building Gazette journal Clients. pub(crate) journal_client_factory: gazette::journal::ClientFactory, + /// Disk backlog threshold in bytes before LogActor engages back-pressure. + /// Applied uniformly across every task served by this Service. + pub(crate) disk_backlog_threshold: u64, /// Transport channels to dialed peers. pub(crate) channels: std::sync::Mutex>, /// Shared state for coordinating Log RPCs from multiple Slices into a single LogActor. @@ -25,10 +28,12 @@ impl Service { pub fn new( peer_endpoint: String, journal_client_factory: gazette::journal::ClientFactory, + disk_backlog_threshold: u64, ) -> Self { Self(Arc::new(ServiceImpl { peer_endpoint, journal_client_factory, + disk_backlog_threshold, channels: std::sync::Mutex::new(HashMap::new()), log_joins: std::sync::Mutex::new(HashMap::new()), })) diff --git a/crates/shuffle/src/session/actor.rs b/crates/shuffle/src/session/actor.rs index 3f74f255706..cea0e598daa 100644 --- a/crates/shuffle/src/session/actor.rs +++ b/crates/shuffle/src/session/actor.rs @@ -17,8 +17,8 @@ pub struct SessionActor { /// Buffered StartReads to be transmitted to their target Slice channel. /// Each entry is (shard_index, StartRead). Drained in FIFO order. pub start_reads: std::collections::VecDeque<(usize, shuffle::slice_request::StartRead)>, - /// Drain of the checkpoint frontier being transmitted as chunked responses. - pub checkpoint_drain: crate::frontier::Drain, + /// Drain of the checkpoint Frontier to be transmitted. + pub checkpoint_drain: Option, } impl SessionActor { @@ -56,7 +56,7 @@ impl SessionActor { tracing::debug!( loop_count, checkpoint = ?self.checkpoint, - drain_empty = ?self.checkpoint_drain, + drain_pending = self.checkpoint_drain.is_some(), progress_ready = ?self.progress_ready, start_reads = self.start_reads.len(), "SessionActor::serve iteration" @@ -167,25 +167,24 @@ impl SessionActor { // Future which represent an absence of an awake signal. let idle = future::Either::Right(std::future::ready(false)); - if self.checkpoint_drain.is_empty() { + if self.checkpoint_drain.is_none() { if let Some(frontier) = self.checkpoint.take_ready() { tracing::debug!(?frontier, "sending NextCheckpoint to client"); - self.checkpoint_drain.start(frontier); + self.checkpoint_drain = Some(frontier.encode()); } } - // Try to drain NextCheckpoint response chunks. - // Ensure channel capacity *before* next_chunk() to not lose it. - while !self.checkpoint_drain.is_empty() { + // Try to drain a NextCheckpoint response. + // Ensure channel capacity *before* take() to not lose it. + if self.checkpoint_drain.is_some() { let Ok(permit) = self.session_response_tx.try_reserve() else { return Ok(future::Either::Left( self.session_response_tx.clone().reserve_owned().map(ok), )); }; - let chunk = self.checkpoint_drain.next_chunk().unwrap(); permit.send(Ok(shuffle::SessionResponse { - next_checkpoint_chunk: Some(chunk), + next_checkpoint: self.checkpoint_drain.take(), ..Default::default() })); } @@ -244,16 +243,15 @@ impl SessionActor { } shuffle::SliceResponse { - progressed: Some(chunk), + progressed: Some(proto), .. } => { - if self.checkpoint.on_progressed_chunk(shard_index, chunk)? { - tracing::debug!( - shard_index, - "Progressed sequence complete, sending next ProgressRequest" - ); - self.progress_ready[shard_index] = true; - } + self.checkpoint.on_progressed(shard_index, proto)?; + tracing::debug!( + shard_index, + "Progressed received, sending next ProgressRequest" + ); + self.progress_ready[shard_index] = true; Ok(()) } diff --git a/crates/shuffle/src/session/handler.rs b/crates/shuffle/src/session/handler.rs index 1832fcd768f..3445ee3ac59 100644 --- a/crates/shuffle/src/session/handler.rs +++ b/crates/shuffle/src/session/handler.rs @@ -35,8 +35,7 @@ where super::state::validate_shard_ranges(&shards)?; let task = task.context("Open must include task")?; - let (shard_prefix, bindings, _validators, _disk_backlog_threshold) = - crate::Binding::from_task(&task)?; + let (shard_prefix, bindings, _validators) = crate::Binding::from_task(&task)?; tracing::info!( session_id, @@ -77,30 +76,17 @@ where }), )?; - // Read resume-checkpoint frontier chunks. - let mut resume_checkpoint: Vec = Vec::new(); - let verify = crate::verify( - "SessionRequest", - "resume-checkpoint chunk", - "coordinator", - 0, - ); - loop { - match verify.not_eof(request_rx.next().await)? { - shuffle::SessionRequest { - resume_checkpoint_chunk: Some(chunk), - .. - } => { - if chunk.journals.is_empty() { - break; - } - resume_checkpoint.extend(crate::JournalFrontier::decode(chunk)); - } - request => return Err(verify.fail(request)), - }; - } - let resume_checkpoint = crate::Frontier::new(resume_checkpoint, Vec::new()) - .context("validating resume_checkpoint frontier")?; + // Read the resume-checkpoint frontier. + let verify = crate::verify("SessionRequest", "resume_checkpoint", "coordinator", 0); + let proto = match verify.not_eof(request_rx.next().await)? { + shuffle::SessionRequest { + resume_checkpoint: Some(proto), + .. + } => proto, + request => return Err(verify.fail(request)), + }; + let resume_checkpoint = + crate::Frontier::decode(proto).context("validating resume_checkpoint frontier")?; tracing::debug!(session_id, ?resume_checkpoint, "Session resume checkpoint"); @@ -126,11 +112,8 @@ where resume_checkpoint, }; let binding_cohorts: Vec = topology.bindings.iter().map(|b| b.cohort).collect(); - let checkpoint = super::state::CheckpointPipeline::new( - &topology.resume_checkpoint, - shard_count, - binding_cohorts, - ); + let checkpoint = + super::state::CheckpointPipeline::new(&topology.resume_checkpoint, binding_cohorts); super::actor::SessionActor { topology, @@ -139,7 +122,7 @@ where session_response_tx: session_response_tx.clone(), slice_request_tx, start_reads: std::collections::VecDeque::new(), - checkpoint_drain: crate::frontier::Drain::new(), + checkpoint_drain: None, } .serve(request_rx, response_rx) .await diff --git a/crates/shuffle/src/session/state.rs b/crates/shuffle/src/session/state.rs index 930ccd7e038..55dba56c98b 100644 --- a/crates/shuffle/src/session/state.rs +++ b/crates/shuffle/src/session/state.rs @@ -173,7 +173,7 @@ impl Topology { /// Four-stage checkpoint pipeline state machine. /// -/// Progress flows: `on_progressed_chunk()` → `progressed` → `unresolved` → `ready` → `take_ready()`. +/// Progress flows: `on_progressed()` → `progressed` → `unresolved` → `ready` → `take_ready()`. /// /// Causal hints gate the `unresolved` → `ready` promotion: progress stays in /// `unresolved` until all hinted journals confirm the producer committed. @@ -191,9 +191,6 @@ impl Topology { /// Peeks let the client begin processing (e.g. release log /// segments) without waiting for full transactional resolution. pub struct CheckpointPipeline { - /// Accumulated partial Progressed frontier chunks per-shard, - /// awaiting end-of-sequence (empty journals terminator). - partials: Vec>, /// Raw accumulated and reduced Progressed responses. /// Promoted to `unresolved` when a) non-empty and b) `unresolved` is empty, /// or promoted directly to `ready` if it contains no unresolved hints. @@ -236,11 +233,7 @@ pub struct CheckpointPipeline { impl CheckpointPipeline { /// Create a new pipeline using the `recovery` Frontier, /// which may contain unresolved hints. - pub fn new( - resume_checkpoint: &crate::Frontier, - shard_count: usize, - binding_cohorts: Vec, - ) -> Self { + pub fn new(resume_checkpoint: &crate::Frontier, binding_cohorts: Vec) -> Self { // Project read-through state from resume_checkpoint: producers with // hinted_commit > last_commit represent transactions that were prepared // but not yet committed during the previous session. @@ -264,7 +257,6 @@ impl CheckpointPipeline { let recovery_pending = unresolved.unresolved_hints != 0; Self { - partials: vec![Vec::new(); shard_count], progressed: Default::default(), unresolved, unresolved_armed: false, @@ -363,25 +355,15 @@ impl CheckpointPipeline { None } - /// Ingest a Progressed frontier chunk from a shard. Accumulates partial - /// chunks until end-of-sequence (empty journals), then assembles the full - /// frontier, resolves causal hints, and promotes through the pipeline. - /// - /// Returns `true` when a complete sequence was ingested, signaling that - /// the caller should send the next ProgressRequest to this shard. - pub fn on_progressed_chunk( + /// Ingest a Progressed frontier from a shard, resolve causal hints, and + /// promote through the pipeline. + pub fn on_progressed( &mut self, shard_index: usize, - chunk: proto_flow::shuffle::FrontierChunk, - ) -> anyhow::Result { - if !chunk.journals.is_empty() { - self.partials[shard_index].extend(crate::JournalFrontier::decode(chunk)); - return Ok(false); - } - - let journals = std::mem::take(&mut self.partials[shard_index]); - let mut progressed = crate::Frontier::new(journals, chunk.flushed_lsn) - .context("validating Progressed frontier delta")?; + proto: proto_flow::shuffle::Frontier, + ) -> anyhow::Result<()> { + let mut progressed = + crate::Frontier::decode(proto).context("validating Progressed frontier delta")?; tracing::debug!(shard_index, ?progressed, "received Progressed response"); @@ -449,7 +431,7 @@ impl CheckpointPipeline { self.progressed = std::mem::take(&mut self.progressed).reduce(progressed); self.try_promote(); - Ok(true) + Ok(()) } /// Called on each actor tick to detect stalled causal hint resolution. @@ -736,32 +718,21 @@ mod test { } } - /// Build a CheckpointPipeline with no recovery state and one test shard. + /// Build a CheckpointPipeline with no recovery state. /// Binding indices 0 and 1 both map to cohort 0. fn test_pipeline() -> CheckpointPipeline { - CheckpointPipeline::new(&Default::default(), 1, vec![0, 0]) + CheckpointPipeline::new(&Default::default(), vec![0, 0]) } - /// Feed a sequence of JournalFrontiers to the pipeline as a chunked - /// Progressed sequence (data chunk + empty terminator) via shard 0. + /// Feed a Progressed frontier to the pipeline via shard 0. fn ingest_progressed( pipeline: &mut CheckpointPipeline, journals: Vec, flushed_lsn: Vec, ) { - let chunk = crate::JournalFrontier::encode(&journals); - assert!(!pipeline.on_progressed_chunk(0, chunk).unwrap()); - assert!( - pipeline - .on_progressed_chunk( - 0, - proto_flow::shuffle::FrontierChunk { - journals: vec![], - flushed_lsn, - } - ) - .unwrap() - ); + let mut proto = crate::JournalFrontier::encode(&journals); + proto.flushed_lsn = flushed_lsn; + pipeline.on_progressed(0, proto).unwrap(); } // --- Tests --- @@ -1177,7 +1148,6 @@ mod test { flushed_lsn: vec![], unresolved_hints: 1, }, - 1, vec![0], ); assert!(pipeline.recovery_pending); @@ -1299,7 +1269,6 @@ mod test { flushed_lsn: vec![], unresolved_hints: 0, }, - 1, vec![0], ); @@ -1718,7 +1687,7 @@ mod test { #[test] fn test_stale_hint_cross_cohort_isolation() { // Binding 0 in cohort 0, binding 1 in cohort 1. - let mut pipeline = CheckpointPipeline::new(&Default::default(), 1, vec![0, 1]); + let mut pipeline = CheckpointPipeline::new(&Default::default(), vec![0, 1]); // Cohort 0 commits producer 0x01 at clock 500. ingest_progressed( @@ -1756,7 +1725,7 @@ mod test { flushed_lsn: vec![], unresolved_hints: 0, }; - let mut pipeline = CheckpointPipeline::new(&resume, 1, vec![0, 0]); + let mut pipeline = CheckpointPipeline::new(&resume, vec![0, 0]); // The recovery hint must be in unresolved, not silently filtered. assert_eq!(pipeline.unresolved.unresolved_hints, 1); diff --git a/crates/shuffle/src/slice/actor.rs b/crates/shuffle/src/slice/actor.rs index c33fdc0256c..7f4d9c84a60 100644 --- a/crates/shuffle/src/slice/actor.rs +++ b/crates/shuffle/src/slice/actor.rs @@ -45,8 +45,8 @@ pub struct SliceActor { pub parser: simd_doc::SimdParser, /// Ordered heap of reads with ready documents. pub ready_read_heap: ReadyReadHeap, - /// Drain of the Progressed frontier being transmitted as chunked responses. - pub progressed_drain: crate::frontier::Drain, + /// Drain of the next Progressed response to transmit. + pub progressed_drain: Option, } struct Buffers { @@ -845,27 +845,24 @@ impl SliceActor { // Future which represent an absence of an awake signal. let idle = future::Either::Right(std::future::ready(false)); - // If no drain is in progress, check whether we should start one. - if self.progressed_drain.is_empty() { + if self.progressed_drain.is_none() { let Some(frontier) = self.progress.take_progressed() else { return Ok(idle); }; tracing::debug!(?frontier, "sending Progressed to Session"); - self.progressed_drain.start(frontier); + self.progressed_drain = Some(frontier.encode()); } - // Drain chunked Progressed responses. - // Ensure channel capacity *before* next_chunk() to not lose it. - while !self.progressed_drain.is_empty() { + // Drain a Progressed response. + // Ensure channel capacity *before* take() to not lose it. + if self.progressed_drain.is_some() { let Ok(permit) = self.slice_response_tx.try_reserve() else { return Ok(future::Either::Left( self.slice_response_tx.clone().reserve_owned().map(|_| true), )); }; - let chunk = self.progressed_drain.next_chunk().unwrap(); - permit.send(Ok(shuffle::SliceResponse { - progressed: Some(chunk), + progressed: self.progressed_drain.take(), ..Default::default() })); } diff --git a/crates/shuffle/src/slice/handler.rs b/crates/shuffle/src/slice/handler.rs index 68ab31bd198..cd268ff2e10 100644 --- a/crates/shuffle/src/slice/handler.rs +++ b/crates/shuffle/src/slice/handler.rs @@ -33,8 +33,7 @@ where ); } let task = task.context("Open must include task")?; - let (shard_prefix, bindings, validators, disk_backlog_threshold) = - crate::Binding::from_task(&task)?; + let (shard_prefix, bindings, validators) = crate::Binding::from_task(&task)?; tracing::info!( session_id, @@ -52,7 +51,6 @@ where slice_shard_index as u32, &shards, log_shard_index as u32, - disk_backlog_threshold, ) })) .await; @@ -123,7 +121,7 @@ where parser: simd_doc::SimdParser::new(1_000_000), ready_read_heap: ReadyReadHeap::new(), tailing_reads: 0, - progressed_drain: crate::frontier::Drain::new(), + progressed_drain: None, } .serve(slice_request_rx, log_response_rx) .await @@ -137,7 +135,6 @@ async fn open_log_rpc( slice_shard_index: u32, shards: &[shuffle::Shard], log_shard_index: u32, - disk_backlog_threshold: u64, ) -> anyhow::Result<( mpsc::Sender, stream::BoxStream<'static, tonic::Result>, @@ -178,7 +175,6 @@ async fn open_log_rpc( shards: shards.to_vec(), slice_shard_index, log_shard_index, - disk_backlog_threshold, }), append: None, flush: None, diff --git a/crates/shuffle/tests/scenario_fixtures.rs b/crates/shuffle/tests/scenario_fixtures.rs index 552639701b4..72db272d2c9 100644 --- a/crates/shuffle/tests/scenario_fixtures.rs +++ b/crates/shuffle/tests/scenario_fixtures.rs @@ -214,7 +214,7 @@ async fn shuffle_scenarios() { let journal_client = data_plane.journal_client.clone(); move |_authz_sub, _authz_obj| journal_client.clone() }); - let service = shuffle::Service::new(endpoint.clone(), factory); + let service = shuffle::Service::new(endpoint.clone(), factory, 10 * 1024 * 1024 * 1024); let server = service.clone().build_tonic_server(); let server_handle = tokio::spawn(async move { diff --git a/crates/shuffle/tests/scenario_fuzz.rs b/crates/shuffle/tests/scenario_fuzz.rs index 0fb4177e62c..8e4f7ecd833 100644 --- a/crates/shuffle/tests/scenario_fuzz.rs +++ b/crates/shuffle/tests/scenario_fuzz.rs @@ -215,7 +215,7 @@ fn get_harness() -> &'static SharedHarness { let journal_client = data_plane.journal_client.clone(); move |_authz_sub, _authz_obj| journal_client.clone() }); - let service = shuffle::Service::new(endpoint, factory); + let service = shuffle::Service::new(endpoint, factory, 10 * 1024 * 1024 * 1024); let server = service.clone().build_tonic_server(); let server_handle = tokio::spawn(async move { diff --git a/go/protocols/materialize/materialize.pb.go b/go/protocols/materialize/materialize.pb.go index 2c216750a88..5126993a8b3 100644 --- a/go/protocols/materialize/materialize.pb.go +++ b/go/protocols/materialize/materialize.pb.go @@ -497,9 +497,17 @@ var xxx_messageInfo_Request_Load proto.InternalMessageInfo // and the runtime will await the connectors's remaining Loaded // responses followed by one Flushed response. type Request_Flush struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + // Aggregated connector state patches from all shards' + // prior-transaction C:Acknowledged responses, in State Update Wire + // Format. Includes this shard's own patches (the runtime feeds the + // shard's contribution back to it for symmetry with the scaled-out + // case). Connectors participating in cooperative multi-shard + // strategies use this to observe peers' state. The runtime forwards + // it verbatim from the leader's L:Flush. + ConnectorStatePatchesJson encoding_json.RawMessage `protobuf:"bytes,1,opt,name=connector_state_patches_json,json=connectorStatePatches,proto3,casttype=encoding/json.RawMessage" json:"connector_state_patches_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *Request_Flush) Reset() { *m = Request_Flush{} } @@ -597,10 +605,18 @@ var xxx_messageInfo_Request_Store proto.InternalMessageInfo // instruct it to start committing its transaction. type Request_StartCommit struct { // Flow runtime checkpoint to commit with this transaction. - RuntimeCheckpoint *protocol.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:"-"` + RuntimeCheckpoint *protocol.Checkpoint `protobuf:"bytes,1,opt,name=runtime_checkpoint,json=runtimeCheckpoint,proto3" json:"runtime_checkpoint,omitempty"` + // Aggregated connector state patches from all shards' current-transaction + // C:Flushed responses, in State Update Wire Format. Includes this + // shard's own patches (the runtime feeds the shard's contribution back + // to it for symmetry with the scaled-out case). Connectors use this + // to observe peers' Flushed state for cooperative strategies like + // parallel file staging. The runtime forwards it verbatim from the + // leader's L:StartCommit. + ConnectorStatePatchesJson encoding_json.RawMessage `protobuf:"bytes,2,opt,name=connector_state_patches_json,json=connectorStatePatches,proto3,casttype=encoding/json.RawMessage" json:"connector_state_patches_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *Request_StartCommit) Reset() { *m = Request_StartCommit{} } @@ -639,9 +655,16 @@ var xxx_messageInfo_Request_StartCommit proto.InternalMessageInfo // Acknowledge to the connector that the previous transaction // has committed to the Flow runtime's recovery log. type Request_Acknowledge struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + // Aggregated connector state patches from all shards' just-committed + // C:StartedCommit responses, in State Update Wire Format. Includes + // this shard's own patches (the runtime feeds the shard's contribution + // back to it for symmetry with the scaled-out case). Connectors use + // this to observe peers' StartedCommit state. The runtime forwards + // it verbatim from the leader's L:Acknowledge. + ConnectorStatePatchesJson encoding_json.RawMessage `protobuf:"bytes,1,opt,name=connector_state_patches_json,json=connectorStatePatches,proto3,casttype=encoding/json.RawMessage" json:"connector_state_patches_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *Request_Acknowledge) Reset() { *m = Request_Acknowledge{} } @@ -1415,123 +1438,125 @@ func init() { } var fileDescriptor_3e8b62b327f34bc6 = []byte{ - // 1843 bytes of a gzipped FileDescriptorProto + // 1880 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0xcf, 0x6f, 0x1b, 0xc7, - 0x15, 0xf6, 0x52, 0xfc, 0xf9, 0x48, 0x49, 0xd4, 0x98, 0xae, 0xe9, 0x8d, 0x63, 0xcb, 0x4a, 0x82, - 0x08, 0x2e, 0x42, 0x19, 0x72, 0xdb, 0xc4, 0x0e, 0x5c, 0x94, 0xa4, 0x28, 0x80, 0x2e, 0x25, 0x2a, - 0x23, 0xdb, 0x01, 0x72, 0x59, 0x8c, 0x76, 0x47, 0xd4, 0x5a, 0xcb, 0x9d, 0xed, 0xce, 0xd2, 0x0e, - 0x7b, 0x2d, 0x8a, 0xa2, 0x3d, 0x14, 0xed, 0xb1, 0x40, 0x0f, 0x3d, 0xf5, 0x1f, 0x28, 0x0a, 0x14, - 0xbd, 0x16, 0x05, 0x7c, 0xec, 0xa9, 0x47, 0x17, 0x4d, 0xff, 0x8b, 0x9c, 0x8a, 0xf9, 0xb1, 0xcb, - 0x25, 0x2d, 0x52, 0x0c, 0xe0, 0xe4, 0x22, 0xec, 0xbc, 0xf9, 0xbe, 0xc7, 0x37, 0x6f, 0xdf, 0xbc, - 0xf7, 0xad, 0xe0, 0xee, 0x80, 0xed, 0x04, 0x21, 0x8b, 0x98, 0xcd, 0x3c, 0xbe, 0x33, 0x24, 0x11, - 0x0d, 0x5d, 0xe2, 0xb9, 0x3f, 0xa7, 0xe9, 0xe7, 0x86, 0x44, 0xa0, 0x72, 0xca, 0x64, 0x6e, 0xda, - 0xcc, 0xe7, 0xa3, 0x21, 0x0d, 0x13, 0x7a, 0xf2, 0xa0, 0xe0, 0xe6, 0xcd, 0x29, 0xd7, 0xa7, 0x1e, - 0x7b, 0x29, 0xff, 0xe8, 0xdd, 0xda, 0x80, 0x0d, 0x98, 0x7c, 0xdc, 0x11, 0x4f, 0xca, 0xba, 0xf5, - 0xef, 0xab, 0x50, 0xc0, 0xf4, 0x67, 0x23, 0xca, 0x23, 0xf4, 0x11, 0x64, 0x79, 0x40, 0xed, 0xba, - 0xb1, 0x69, 0x6c, 0x97, 0x77, 0x6f, 0x34, 0xd2, 0x01, 0x69, 0x4c, 0xe3, 0x38, 0xa0, 0x36, 0x96, - 0x30, 0xf4, 0x00, 0x8a, 0x2f, 0x88, 0xe7, 0x3a, 0x24, 0xa2, 0xf5, 0x8c, 0xa4, 0xbc, 0x7b, 0x21, - 0xe5, 0x99, 0x06, 0xe1, 0x04, 0x8e, 0xee, 0x41, 0x8e, 0x04, 0x81, 0x37, 0xae, 0xaf, 0x48, 0x9e, - 0x79, 0x21, 0xaf, 0x29, 0x10, 0x58, 0x01, 0x45, 0x6c, 0x2c, 0xa0, 0x7e, 0x3d, 0xbb, 0x20, 0xb6, - 0x7e, 0x40, 0x7d, 0x2c, 0x61, 0x02, 0xee, 0x31, 0xe2, 0xd4, 0x73, 0x0b, 0xe0, 0x3d, 0x46, 0x1c, - 0x2c, 0x61, 0x22, 0x9e, 0x53, 0x6f, 0xc4, 0xcf, 0xea, 0xf9, 0x05, 0xf1, 0xec, 0x0b, 0x04, 0x56, - 0x40, 0xc1, 0xe0, 0x11, 0x0b, 0x69, 0xbd, 0xb0, 0x80, 0x71, 0x2c, 0x10, 0x58, 0x01, 0x51, 0x1b, - 0x2a, 0x3c, 0x22, 0x61, 0x64, 0xd9, 0x6c, 0x38, 0x74, 0xa3, 0x7a, 0x51, 0x12, 0x37, 0xe7, 0x10, - 0x49, 0x18, 0xb5, 0x25, 0x0e, 0x97, 0xf9, 0x64, 0x81, 0x5a, 0x50, 0x26, 0xf6, 0xb9, 0xcf, 0x5e, - 0x7a, 0xd4, 0x19, 0xd0, 0x7a, 0x69, 0x81, 0x8f, 0xe6, 0x04, 0x87, 0xd3, 0x24, 0xf4, 0x0e, 0x14, - 0x5d, 0x3f, 0xa2, 0xa1, 0x4f, 0xbc, 0xba, 0xb3, 0x69, 0x6c, 0x57, 0x70, 0xe9, 0xfd, 0xd8, 0x60, - 0xfe, 0xde, 0x80, 0xac, 0x78, 0xc7, 0xe8, 0x10, 0xd6, 0x6c, 0xe6, 0xfb, 0xd4, 0x8e, 0x58, 0x68, - 0x45, 0xe3, 0x80, 0xca, 0xb2, 0x58, 0xdb, 0xfd, 0xb0, 0x21, 0x6b, 0xea, 0x20, 0xf9, 0x45, 0x12, - 0xb9, 0xcc, 0x17, 0x94, 0x46, 0x3b, 0xc6, 0x3f, 0x19, 0x07, 0x14, 0xaf, 0xda, 0xe9, 0x25, 0x7a, - 0x00, 0x65, 0x9b, 0xf9, 0xa7, 0xee, 0xc0, 0x7a, 0xce, 0x99, 0x2f, 0x0b, 0xa6, 0xd2, 0xba, 0xf9, - 0xf5, 0xeb, 0xdb, 0x75, 0xea, 0xdb, 0xcc, 0x71, 0xfd, 0xc1, 0x8e, 0xd8, 0x68, 0x60, 0xf2, 0xf2, - 0x80, 0x72, 0x4e, 0x06, 0x14, 0xe7, 0x15, 0xc1, 0xfc, 0x5b, 0x1e, 0x8a, 0x71, 0x11, 0xa1, 0xcf, - 0x20, 0xeb, 0x93, 0xa1, 0x8a, 0xa6, 0xd4, 0x7a, 0xf4, 0xf5, 0xeb, 0xdb, 0x0f, 0x06, 0x6e, 0x74, - 0x36, 0x3a, 0x69, 0xd8, 0x6c, 0xb8, 0x43, 0x79, 0x34, 0x22, 0xe1, 0x58, 0x15, 0xff, 0x1b, 0xd7, - 0x61, 0x36, 0x6a, 0x2c, 0x5d, 0x5d, 0x70, 0xd4, 0xcc, 0xdb, 0x3c, 0xea, 0xca, 0xf2, 0x47, 0x45, - 0x4d, 0x28, 0x9e, 0xb8, 0xbe, 0x80, 0xf0, 0x7a, 0x76, 0x73, 0x65, 0xbb, 0xbc, 0xfb, 0xc1, 0xc2, - 0x3b, 0xd5, 0x68, 0x29, 0x34, 0x4e, 0x68, 0xa8, 0x07, 0x35, 0x8f, 0xf0, 0xc8, 0x1a, 0x4e, 0x87, - 0x9d, 0x5c, 0x85, 0x79, 0x67, 0xc2, 0x57, 0x05, 0x6d, 0x66, 0x03, 0xdd, 0x81, 0x8a, 0xf4, 0xf6, - 0x82, 0x86, 0x5c, 0x78, 0x11, 0x17, 0xa4, 0x84, 0xcb, 0xc2, 0xf6, 0x4c, 0x99, 0xcc, 0x3f, 0xac, - 0x40, 0x41, 0x87, 0x81, 0x1e, 0x43, 0x2d, 0xa4, 0x9c, 0x8d, 0x42, 0x9b, 0x5a, 0xe9, 0x1c, 0x18, - 0x4b, 0xe4, 0x60, 0x2d, 0x66, 0xb6, 0x55, 0x2e, 0x1e, 0x02, 0xd8, 0xcc, 0xf3, 0xa8, 0x2d, 0xc3, - 0x57, 0x1d, 0xa6, 0xa6, 0xc2, 0x6f, 0x27, 0x76, 0x11, 0x79, 0x2b, 0xfb, 0xea, 0xf5, 0xed, 0x2b, - 0x38, 0x85, 0x46, 0xbf, 0x32, 0xe0, 0xda, 0xa9, 0x4b, 0x3d, 0x27, 0x1d, 0x85, 0x35, 0x24, 0x41, - 0x7d, 0x45, 0x66, 0xf5, 0xd1, 0x52, 0x59, 0x6d, 0xec, 0x0b, 0x17, 0x2a, 0x9c, 0xc7, 0x9c, 0xf9, - 0x07, 0x24, 0xe8, 0xf8, 0x51, 0x38, 0x6e, 0xdd, 0xfc, 0xcd, 0x7f, 0x16, 0x1c, 0xa4, 0x7c, 0x3a, - 0xa1, 0x21, 0x13, 0x8a, 0x27, 0xc4, 0x3e, 0x3f, 0x75, 0x3d, 0x4f, 0x36, 0xaf, 0x55, 0x9c, 0xac, - 0xd1, 0x0d, 0x28, 0x0e, 0x42, 0x36, 0x0a, 0xac, 0x93, 0x71, 0x3d, 0xb7, 0xb9, 0xb2, 0x5d, 0xc2, - 0x05, 0xb9, 0x6e, 0x8d, 0xcd, 0x0e, 0x5c, 0x9f, 0xf3, 0xe3, 0xa8, 0x0a, 0x2b, 0xe7, 0x74, 0xac, - 0x2e, 0x00, 0x16, 0x8f, 0xa8, 0x06, 0xb9, 0x17, 0xc4, 0x1b, 0xa9, 0xba, 0xad, 0x60, 0xb5, 0x78, - 0x98, 0xf9, 0xc4, 0x30, 0x7f, 0x97, 0x81, 0x9c, 0xec, 0xa3, 0xa8, 0x0d, 0xeb, 0xb3, 0x15, 0x61, - 0x5c, 0x56, 0x11, 0xb3, 0x0c, 0x54, 0x87, 0x42, 0x5c, 0x08, 0x19, 0xf9, 0xf3, 0xf1, 0x72, 0x6e, - 0xd5, 0x65, 0xdf, 0x4a, 0xd5, 0xe5, 0xde, 0xa8, 0x3a, 0xf4, 0x31, 0x00, 0x8f, 0x48, 0x44, 0x55, - 0x7d, 0xe5, 0x97, 0xa8, 0xaf, 0x9c, 0xc4, 0x9b, 0xff, 0x34, 0x20, 0x2b, 0x26, 0xc5, 0xb7, 0x9d, - 0x91, 0x0f, 0x20, 0x17, 0x12, 0x7f, 0x40, 0xf5, 0x8c, 0x5b, 0x57, 0x4e, 0xb1, 0x30, 0x49, 0x57, - 0x6a, 0x77, 0xe6, 0x1c, 0xd9, 0xe5, 0xcf, 0x11, 0x41, 0x56, 0x4c, 0x30, 0x11, 0x81, 0xbe, 0xfb, - 0x32, 0xfc, 0x55, 0x1c, 0x2f, 0xd1, 0x7d, 0x28, 0x9e, 0xd3, 0xf1, 0xf2, 0xfd, 0x56, 0xd6, 0xd2, - 0xbb, 0x00, 0x82, 0x14, 0x10, 0xfb, 0x9c, 0x3a, 0xaa, 0x77, 0xe1, 0xd2, 0x39, 0x1d, 0x1f, 0x49, - 0x83, 0x59, 0x80, 0x9c, 0x9c, 0x83, 0xe6, 0xdf, 0x33, 0x90, 0x93, 0xf3, 0xed, 0xbb, 0x0d, 0x40, - 0x34, 0x57, 0x59, 0xde, 0x7c, 0xf9, 0x84, 0xe5, 0x15, 0x01, 0xbd, 0x07, 0xab, 0x9a, 0xaa, 0x9d, - 0xe7, 0xa4, 0xf3, 0x8a, 0x32, 0x6a, 0xff, 0xf7, 0xa1, 0xe8, 0x30, 0x7b, 0xf9, 0xaa, 0x5a, 0x71, - 0x98, 0x8d, 0xbe, 0x07, 0x79, 0xfa, 0xa5, 0xcb, 0x23, 0x2e, 0xe5, 0x40, 0x11, 0xeb, 0x95, 0xb0, - 0x3b, 0xd4, 0xa3, 0x11, 0x95, 0xd3, 0xbe, 0x88, 0xf5, 0xca, 0xc4, 0x50, 0x4e, 0x8d, 0x78, 0xd4, - 0x06, 0x14, 0x8e, 0xfc, 0xc8, 0x1d, 0x52, 0xcb, 0x3e, 0xa3, 0xf6, 0x79, 0xc0, 0x5c, 0x3f, 0xd2, - 0xc5, 0x58, 0x6b, 0xc4, 0xba, 0xaf, 0xd1, 0x4e, 0xf6, 0xf0, 0x86, 0xc6, 0x4f, 0x4c, 0xe6, 0x2a, - 0x94, 0x53, 0x23, 0x7f, 0xeb, 0x2f, 0x55, 0x28, 0x62, 0xca, 0x03, 0xe6, 0x73, 0x8a, 0x1a, 0x53, - 0xca, 0x6e, 0x56, 0xac, 0x28, 0x50, 0x5a, 0xda, 0x3d, 0x82, 0x52, 0xac, 0xd5, 0x1c, 0xdd, 0x79, - 0x6f, 0x5f, 0x4c, 0x8a, 0x5b, 0xa6, 0x83, 0x27, 0x0c, 0xf4, 0x31, 0x14, 0x84, 0x6a, 0x73, 0xf5, - 0xfb, 0x7b, 0x53, 0x18, 0x6a, 0x72, 0x53, 0x81, 0x70, 0x8c, 0x46, 0x3f, 0x80, 0xbc, 0x90, 0x6f, - 0xd4, 0xd1, 0x7d, 0xe3, 0xe6, 0xc5, 0xbc, 0xbe, 0xc4, 0x60, 0x8d, 0x15, 0x2c, 0xa1, 0xe2, 0x68, - 0x2c, 0xf7, 0xe6, 0xb0, 0x7a, 0x12, 0x83, 0x35, 0x56, 0x04, 0x29, 0xa5, 0x1c, 0x75, 0xb4, 0xea, - 0x9b, 0x13, 0xe4, 0xbe, 0x02, 0xe1, 0x18, 0x8d, 0x1e, 0xc3, 0x9a, 0x94, 0x64, 0xd4, 0x89, 0xa5, - 0x9c, 0xd2, 0x80, 0xef, 0xcd, 0x49, 0xab, 0xc2, 0x6a, 0x35, 0xb7, 0xca, 0xd3, 0x4b, 0xb4, 0x0f, - 0x95, 0x94, 0x34, 0x73, 0xb4, 0x28, 0xdc, 0x9a, 0x93, 0xae, 0x14, 0x12, 0x4f, 0xf1, 0x16, 0x6b, - 0xba, 0x3f, 0x66, 0xb4, 0xa6, 0x33, 0xa1, 0x18, 0x0b, 0x22, 0x7d, 0x55, 0x93, 0x35, 0xda, 0x07, - 0xa4, 0x47, 0x25, 0xb7, 0xcf, 0xe8, 0x90, 0x2c, 0x7f, 0x6b, 0x2b, 0x8a, 0x77, 0x2c, 0x69, 0xe8, - 0x73, 0x78, 0x67, 0x56, 0x01, 0xa4, 0x1d, 0x2e, 0x23, 0x86, 0x6a, 0xd3, 0x42, 0x40, 0x3b, 0xfe, - 0x3e, 0x6c, 0x38, 0xcc, 0x1e, 0x0d, 0xa9, 0x1f, 0xc9, 0xd6, 0x6b, 0x8d, 0x42, 0x35, 0x51, 0x4b, - 0xb8, 0x3a, 0xb5, 0xf1, 0x34, 0xf4, 0xd0, 0xfb, 0x90, 0x67, 0x64, 0x14, 0x9d, 0xed, 0xea, 0x92, - 0xa8, 0xa8, 0xee, 0xdb, 0x6f, 0x0a, 0x1b, 0xd6, 0x7b, 0x8f, 0xb3, 0xc5, 0x7c, 0xb5, 0x60, 0xfe, - 0x36, 0x0f, 0xa5, 0xa4, 0x8c, 0x51, 0x3b, 0xa5, 0xc0, 0x0c, 0xa9, 0x15, 0x3e, 0xbc, 0xa4, 0xf2, - 0xdf, 0xd4, 0x60, 0xe6, 0x5f, 0x33, 0x00, 0x6d, 0xe6, 0xf3, 0x28, 0x24, 0xae, 0x2f, 0xee, 0x77, - 0x36, 0x25, 0x2b, 0x77, 0x2e, 0xf3, 0x37, 0x61, 0x36, 0xa4, 0xbc, 0x94, 0x64, 0xd1, 0x4b, 0x42, - 0x4a, 0xe2, 0x1c, 0x96, 0xb0, 0x5e, 0x89, 0x59, 0x79, 0xca, 0x3c, 0x87, 0x3a, 0x96, 0x94, 0x1d, - 0x3a, 0x25, 0x65, 0x65, 0x93, 0x1a, 0x62, 0xeb, 0xcf, 0x06, 0x64, 0xa5, 0x32, 0x2d, 0x43, 0xa1, - 0x7b, 0xf8, 0xac, 0xd9, 0xeb, 0xee, 0x55, 0xaf, 0x20, 0x04, 0x6b, 0xfb, 0xdd, 0x4e, 0x6f, 0xcf, - 0xc2, 0x9d, 0xcf, 0x9e, 0x76, 0x71, 0x67, 0xaf, 0x6a, 0xa0, 0x6b, 0xb0, 0xd1, 0xeb, 0xb7, 0x9b, - 0x4f, 0xba, 0xfd, 0xc3, 0x89, 0x39, 0x83, 0xea, 0x50, 0x4b, 0x99, 0xdb, 0xfd, 0x83, 0x83, 0xce, - 0xe1, 0x5e, 0x67, 0xaf, 0xba, 0x32, 0x71, 0xd2, 0x3f, 0x12, 0xbb, 0xcd, 0x5e, 0x35, 0x8b, 0xae, - 0xc2, 0xba, 0xb2, 0xed, 0xf7, 0x71, 0xab, 0xbb, 0xb7, 0xd7, 0x39, 0xac, 0xe6, 0x50, 0x15, 0x2a, - 0xdd, 0xc3, 0x76, 0xff, 0xe0, 0xa8, 0xf9, 0xa4, 0xdb, 0xea, 0x75, 0xaa, 0x79, 0xb4, 0x01, 0xab, - 0x4f, 0x0f, 0x8f, 0x9b, 0x4f, 0xba, 0xc7, 0xfb, 0xdd, 0xa6, 0x30, 0x15, 0xcc, 0x5f, 0xa4, 0xa4, - 0xe4, 0x8f, 0xe0, 0xba, 0x4d, 0x38, 0xb5, 0x5c, 0x9f, 0x53, 0x9f, 0xbb, 0x91, 0xfb, 0x82, 0xaa, - 0x13, 0x72, 0xf9, 0x4e, 0x8b, 0xf8, 0x9a, 0xd8, 0xee, 0x4e, 0x76, 0xe5, 0x59, 0x39, 0xfa, 0x42, - 0xaa, 0x6f, 0x9d, 0xc0, 0xf8, 0x1d, 0x7e, 0xb2, 0xe4, 0x3b, 0x4c, 0xe5, 0x9e, 0x4b, 0xb5, 0x85, - 0xd3, 0xce, 0xc4, 0x04, 0x49, 0x8a, 0x3b, 0x20, 0xd1, 0x59, 0x3d, 0x23, 0x55, 0x5b, 0x25, 0x36, - 0x1e, 0x91, 0xe8, 0x4c, 0x80, 0x1c, 0xea, 0x45, 0xc4, 0x1a, 0x05, 0xc2, 0x37, 0x97, 0xef, 0xab, - 0x88, 0x2b, 0xd2, 0xf8, 0x54, 0xd9, 0x50, 0x03, 0x80, 0xd3, 0xd0, 0x0a, 0x98, 0xe7, 0xda, 0x63, - 0xdd, 0xed, 0xb4, 0x44, 0x38, 0xa6, 0xe1, 0x91, 0x34, 0xe3, 0x12, 0x8f, 0x1f, 0xcd, 0xe7, 0x50, - 0x9d, 0x0d, 0xed, 0x02, 0x21, 0xf8, 0x93, 0xb4, 0x10, 0x2c, 0xef, 0xde, 0x5d, 0xbe, 0xd2, 0xd2, - 0xa2, 0xd1, 0x81, 0x82, 0xee, 0xcc, 0xe8, 0x23, 0x40, 0x44, 0x2a, 0x6a, 0xcb, 0xa1, 0xdc, 0x0e, - 0xdd, 0x20, 0x91, 0x49, 0x25, 0xbc, 0xa1, 0x76, 0xf6, 0x26, 0x1b, 0xe8, 0x2e, 0x28, 0x71, 0x32, - 0xab, 0xd6, 0xf5, 0xd7, 0xd1, 0xb1, 0xd8, 0x8b, 0xf5, 0xcb, 0xaf, 0x0d, 0xc8, 0xab, 0x46, 0xfe, - 0x56, 0xe6, 0x1f, 0x7a, 0x08, 0x37, 0x1c, 0x97, 0x93, 0x13, 0x8f, 0x5a, 0xa2, 0xc3, 0x5b, 0x2c, - 0x88, 0xdc, 0x61, 0x2c, 0xec, 0x32, 0xf2, 0x15, 0x5c, 0xd7, 0x00, 0x31, 0x09, 0xfa, 0xa9, 0x6d, - 0xf3, 0x73, 0xc8, 0xab, 0xe9, 0xb0, 0x58, 0xcc, 0x24, 0xc2, 0x20, 0xb3, 0xa4, 0x30, 0x30, 0x7f, - 0x08, 0x05, 0x3d, 0x3f, 0x26, 0xb9, 0x31, 0x2e, 0xcf, 0xcd, 0xa7, 0xb0, 0x3a, 0x35, 0x36, 0xbe, - 0x11, 0xf9, 0x21, 0x54, 0xd2, 0x93, 0xe2, 0x9b, 0x70, 0xb7, 0x7e, 0x99, 0x85, 0x5c, 0xe7, 0xcb, - 0x28, 0x24, 0xe6, 0x3f, 0x0c, 0xb8, 0x13, 0x17, 0x4a, 0x47, 0xa8, 0x19, 0xd7, 0x1f, 0x1c, 0x85, - 0xec, 0xb9, 0xfa, 0xc0, 0x8a, 0xff, 0x65, 0xd4, 0x83, 0x2a, 0xd5, 0x9b, 0x56, 0x3a, 0x6f, 0xe5, - 0xdd, 0x3b, 0xf3, 0x3f, 0x9e, 0xe3, 0x7e, 0xb9, 0x1e, 0x53, 0xe3, 0x2b, 0x7f, 0x04, 0xd5, 0x20, - 0x64, 0x01, 0xe3, 0xd4, 0x49, 0xbc, 0xa9, 0x4a, 0x5a, 0xf2, 0x2b, 0x78, 0x3d, 0xa6, 0x6b, 0x83, - 0x68, 0xc4, 0xc9, 0x29, 0xb4, 0xad, 0x39, 0x20, 0xae, 0xcf, 0xa3, 0xd4, 0x6d, 0x42, 0x9f, 0x4e, - 0xbf, 0xf4, 0xa5, 0x82, 0x4f, 0xea, 0x62, 0x30, 0xdd, 0x6f, 0x32, 0xb2, 0xdf, 0x74, 0xa6, 0xe2, - 0x95, 0x19, 0x6d, 0x5c, 0x1a, 0xc7, 0xe2, 0xe6, 0xf3, 0x5d, 0xb6, 0x80, 0xdd, 0x9f, 0x42, 0x29, - 0x29, 0x10, 0xf4, 0x63, 0x28, 0x4f, 0x32, 0x41, 0x51, 0xed, 0xa2, 0x77, 0x61, 0x5e, 0xbb, 0xf0, - 0x87, 0xb6, 0x8d, 0x7b, 0x46, 0xab, 0xf5, 0xea, 0xbf, 0xb7, 0xae, 0xbc, 0xfa, 0xea, 0x96, 0xf1, - 0xaf, 0xaf, 0x6e, 0x19, 0x7f, 0xfa, 0xdf, 0x2d, 0xe3, 0x8b, 0x7b, 0x4b, 0xfd, 0xcb, 0x26, 0xe5, - 0xf0, 0x24, 0x2f, 0xcd, 0xf7, 0xff, 0x1f, 0x00, 0x00, 0xff, 0xff, 0x51, 0x0d, 0x8e, 0x1e, 0x3f, - 0x15, 0x00, 0x00, + 0xf5, 0xf7, 0x52, 0xfc, 0xf9, 0x48, 0x49, 0xd4, 0x44, 0xfa, 0x9a, 0xde, 0x28, 0xb6, 0xac, 0x24, + 0x88, 0xe0, 0x2f, 0x42, 0x19, 0x72, 0xdb, 0xc4, 0x0e, 0x5c, 0x94, 0xa4, 0x28, 0x80, 0xae, 0x24, + 0x2a, 0x23, 0xdb, 0x41, 0x73, 0x59, 0x8c, 0x76, 0x47, 0xe4, 0x5a, 0xcb, 0x9d, 0xed, 0xce, 0xd2, + 0x0e, 0x7b, 0x2d, 0x8a, 0xa2, 0x3d, 0x14, 0xed, 0xb1, 0x40, 0x0f, 0x3d, 0xf5, 0xd6, 0x53, 0x51, + 0xa0, 0xe8, 0xb5, 0x28, 0xe0, 0x63, 0xff, 0x02, 0x17, 0x4d, 0x2f, 0xfd, 0x1b, 0x72, 0x2a, 0xe6, + 0xc7, 0x2e, 0x97, 0xb4, 0x48, 0x31, 0x85, 0x93, 0x0b, 0xb1, 0xf3, 0xe6, 0xf3, 0x79, 0x7c, 0x33, + 0xf3, 0xe6, 0xbd, 0xcf, 0x2e, 0xdc, 0xe9, 0xb1, 0xdd, 0x20, 0x64, 0x11, 0xb3, 0x99, 0xc7, 0x77, + 0x07, 0x24, 0xa2, 0xa1, 0x4b, 0x3c, 0xf7, 0x27, 0x34, 0xfd, 0x5c, 0x97, 0x08, 0x54, 0x4e, 0x99, + 0xcc, 0x2d, 0x9b, 0xf9, 0x7c, 0x38, 0xa0, 0x61, 0x42, 0x4f, 0x1e, 0x14, 0xdc, 0xdc, 0x9c, 0x70, + 0x7d, 0xee, 0xb1, 0x17, 0xf2, 0x47, 0xcf, 0xae, 0xf7, 0x58, 0x8f, 0xc9, 0xc7, 0x5d, 0xf1, 0xa4, + 0xac, 0xdb, 0xff, 0xd9, 0x80, 0x02, 0xa6, 0x3f, 0x1e, 0x52, 0x1e, 0xa1, 0x0f, 0x21, 0xcb, 0x03, + 0x6a, 0xd7, 0x8c, 0x2d, 0x63, 0xa7, 0xbc, 0x77, 0xa3, 0x9e, 0x0e, 0x48, 0x63, 0xea, 0xa7, 0x01, + 0xb5, 0xb1, 0x84, 0xa1, 0xfb, 0x50, 0x7c, 0x4e, 0x3c, 0xd7, 0x21, 0x11, 0xad, 0x65, 0x24, 0xe5, + 0x9d, 0x4b, 0x29, 0x4f, 0x35, 0x08, 0x27, 0x70, 0x74, 0x17, 0x72, 0x24, 0x08, 0xbc, 0x51, 0x6d, + 0x49, 0xf2, 0xcc, 0x4b, 0x79, 0x0d, 0x81, 0xc0, 0x0a, 0x28, 0x62, 0x63, 0x01, 0xf5, 0x6b, 0xd9, + 0x39, 0xb1, 0x75, 0x03, 0xea, 0x63, 0x09, 0x13, 0x70, 0x8f, 0x11, 0xa7, 0x96, 0x9b, 0x03, 0x3f, + 0x64, 0xc4, 0xc1, 0x12, 0x26, 0xe2, 0x39, 0xf7, 0x86, 0xbc, 0x5f, 0xcb, 0xcf, 0x89, 0xe7, 0x40, + 0x20, 0xb0, 0x02, 0x0a, 0x06, 0x8f, 0x58, 0x48, 0x6b, 0x85, 0x39, 0x8c, 0x53, 0x81, 0xc0, 0x0a, + 0x88, 0x5a, 0x50, 0xe1, 0x11, 0x09, 0x23, 0xcb, 0x66, 0x83, 0x81, 0x1b, 0xd5, 0x8a, 0x92, 0xb8, + 0x35, 0x83, 0x48, 0xc2, 0xa8, 0x25, 0x71, 0xb8, 0xcc, 0xc7, 0x03, 0xd4, 0x84, 0x32, 0xb1, 0x2f, + 0x7c, 0xf6, 0xc2, 0xa3, 0x4e, 0x8f, 0xd6, 0x4a, 0x73, 0x7c, 0x34, 0xc6, 0x38, 0x9c, 0x26, 0xa1, + 0xb7, 0xa1, 0xe8, 0xfa, 0x11, 0x0d, 0x7d, 0xe2, 0xd5, 0x9c, 0x2d, 0x63, 0xa7, 0x82, 0x4b, 0xef, + 0xc5, 0x06, 0xf3, 0x37, 0x06, 0x64, 0xc5, 0x19, 0xa3, 0x63, 0x58, 0xb1, 0x99, 0xef, 0x53, 0x3b, + 0x62, 0xa1, 0x15, 0x8d, 0x02, 0x2a, 0xd3, 0x62, 0x65, 0xef, 0x83, 0xba, 0xcc, 0xa9, 0xa3, 0xe4, + 0x1f, 0x49, 0xe4, 0x32, 0x5f, 0x50, 0xea, 0xad, 0x18, 0xff, 0x78, 0x14, 0x50, 0xbc, 0x6c, 0xa7, + 0x87, 0xe8, 0x3e, 0x94, 0x6d, 0xe6, 0x9f, 0xbb, 0x3d, 0xeb, 0x19, 0x67, 0xbe, 0x4c, 0x98, 0x4a, + 0x73, 0xf3, 0xab, 0x57, 0xb7, 0x6a, 0xd4, 0xb7, 0x99, 0xe3, 0xfa, 0xbd, 0x5d, 0x31, 0x51, 0xc7, + 0xe4, 0xc5, 0x11, 0xe5, 0x9c, 0xf4, 0x28, 0xce, 0x2b, 0x82, 0xf9, 0x97, 0x3c, 0x14, 0xe3, 0x24, + 0x42, 0x9f, 0x42, 0xd6, 0x27, 0x03, 0x15, 0x4d, 0xa9, 0xf9, 0xf0, 0xab, 0x57, 0xb7, 0xee, 0xf7, + 0xdc, 0xa8, 0x3f, 0x3c, 0xab, 0xdb, 0x6c, 0xb0, 0x4b, 0x79, 0x34, 0x24, 0xe1, 0x48, 0x25, 0xff, + 0x6b, 0xd7, 0x61, 0x3a, 0x6a, 0x2c, 0x5d, 0x5d, 0xb2, 0xd4, 0xcc, 0x9b, 0x5c, 0xea, 0xd2, 0xe2, + 0x4b, 0x45, 0x0d, 0x28, 0x9e, 0xb9, 0xbe, 0x80, 0xf0, 0x5a, 0x76, 0x6b, 0x69, 0xa7, 0xbc, 0xf7, + 0xfe, 0xdc, 0x3b, 0x55, 0x6f, 0x2a, 0x34, 0x4e, 0x68, 0xe8, 0x10, 0xd6, 0x3d, 0xc2, 0x23, 0x6b, + 0x30, 0x19, 0x76, 0x72, 0x15, 0x66, 0xad, 0x09, 0xbf, 0x25, 0x68, 0x53, 0x13, 0xe8, 0x36, 0x54, + 0xa4, 0xb7, 0xe7, 0x34, 0xe4, 0xc2, 0x8b, 0xb8, 0x20, 0x25, 0x5c, 0x16, 0xb6, 0xa7, 0xca, 0x64, + 0xfe, 0x76, 0x09, 0x0a, 0x3a, 0x0c, 0xf4, 0x08, 0xd6, 0x43, 0xca, 0xd9, 0x30, 0xb4, 0xa9, 0x95, + 0xde, 0x03, 0x63, 0x81, 0x3d, 0x58, 0x89, 0x99, 0x2d, 0xb5, 0x17, 0x0f, 0x00, 0x6c, 0xe6, 0x79, + 0xd4, 0x96, 0xe1, 0xab, 0x0a, 0xb3, 0xae, 0xc2, 0x6f, 0x25, 0x76, 0x11, 0x79, 0x33, 0xfb, 0xf2, + 0xd5, 0xad, 0x6b, 0x38, 0x85, 0x46, 0x3f, 0x37, 0x60, 0xe3, 0xdc, 0xa5, 0x9e, 0x93, 0x8e, 0xc2, + 0x1a, 0x90, 0xa0, 0xb6, 0x24, 0x77, 0xf5, 0xe1, 0x42, 0xbb, 0x5a, 0x3f, 0x10, 0x2e, 0x54, 0x38, + 0x8f, 0x38, 0xf3, 0x8f, 0x48, 0xd0, 0xf6, 0xa3, 0x70, 0xd4, 0xdc, 0xfc, 0xe5, 0x3f, 0xe7, 0x2c, + 0xa4, 0x7c, 0x3e, 0xa6, 0x21, 0x13, 0x8a, 0x67, 0xc4, 0xbe, 0x38, 0x77, 0x3d, 0x4f, 0x16, 0xaf, + 0x65, 0x9c, 0x8c, 0xd1, 0x0d, 0x28, 0xf6, 0x42, 0x36, 0x0c, 0xac, 0xb3, 0x51, 0x2d, 0xb7, 0xb5, + 0xb4, 0x53, 0xc2, 0x05, 0x39, 0x6e, 0x8e, 0xcc, 0x36, 0x5c, 0x9f, 0xf1, 0xe7, 0xa8, 0x0a, 0x4b, + 0x17, 0x74, 0xa4, 0x2e, 0x00, 0x16, 0x8f, 0x68, 0x1d, 0x72, 0xcf, 0x89, 0x37, 0x54, 0x79, 0x5b, + 0xc1, 0x6a, 0xf0, 0x20, 0xf3, 0xb1, 0x61, 0xfe, 0x3a, 0x03, 0x39, 0x59, 0x47, 0x51, 0x0b, 0x56, + 0xa7, 0x33, 0xc2, 0xb8, 0x2a, 0x23, 0xa6, 0x19, 0xa8, 0x06, 0x85, 0x38, 0x11, 0x32, 0xf2, 0xef, + 0xe3, 0xe1, 0xcc, 0xac, 0xcb, 0xbe, 0x91, 0xac, 0xcb, 0xbd, 0x96, 0x75, 0xe8, 0x23, 0x00, 0x1e, + 0x91, 0x88, 0xaa, 0xfc, 0xca, 0x2f, 0x90, 0x5f, 0x39, 0x89, 0x37, 0xff, 0x6e, 0x40, 0x56, 0x74, + 0x8a, 0x6f, 0x7a, 0x47, 0xde, 0x87, 0x5c, 0x48, 0xfc, 0x1e, 0xd5, 0x3d, 0x6e, 0x55, 0x39, 0xc5, + 0xc2, 0x24, 0x5d, 0xa9, 0xd9, 0xa9, 0x75, 0x64, 0x17, 0x5f, 0x47, 0x04, 0x59, 0xd1, 0xc1, 0x44, + 0x04, 0xfa, 0xee, 0xcb, 0xf0, 0x97, 0x71, 0x3c, 0x44, 0xf7, 0xa0, 0x78, 0x41, 0x47, 0x8b, 0xd7, + 0x5b, 0x99, 0x4b, 0xef, 0x00, 0x08, 0x52, 0x40, 0xec, 0x0b, 0xea, 0xa8, 0xda, 0x85, 0x4b, 0x17, + 0x74, 0x74, 0x22, 0x0d, 0xe6, 0x19, 0xe4, 0x64, 0x1f, 0x44, 0x3f, 0x82, 0xcd, 0x71, 0xd1, 0x54, + 0x2b, 0x08, 0x48, 0x64, 0xf7, 0x29, 0x5f, 0xfc, 0xc6, 0x6f, 0x24, 0x1e, 0x4e, 0x85, 0x83, 0x13, + 0xc5, 0x37, 0xff, 0x9a, 0x81, 0x9c, 0x6c, 0x9d, 0xdf, 0xee, 0xda, 0x44, 0xdd, 0x96, 0x37, 0x87, + 0x2f, 0x7e, 0x16, 0x79, 0x45, 0x40, 0xef, 0xc2, 0xb2, 0xa6, 0x6a, 0xe7, 0x39, 0xe9, 0xbc, 0xa2, + 0x8c, 0xda, 0xff, 0x3d, 0x28, 0x3a, 0xcc, 0x5e, 0x3c, 0x61, 0x97, 0x1c, 0x66, 0xa3, 0xff, 0x83, + 0x3c, 0xfd, 0xc2, 0xe5, 0x11, 0x97, 0x4a, 0xa3, 0x88, 0xf5, 0x48, 0xd8, 0x1d, 0xea, 0xd1, 0x88, + 0x4a, 0x21, 0x51, 0xc4, 0x7a, 0x64, 0xfe, 0xd1, 0x80, 0x72, 0x4a, 0x3e, 0xa0, 0x16, 0xa0, 0x70, + 0xe8, 0x47, 0xee, 0x80, 0x5a, 0x76, 0x9f, 0xda, 0x17, 0x01, 0x73, 0xfd, 0x48, 0x27, 0xfa, 0x7a, + 0x3d, 0xd6, 0x94, 0xf5, 0x56, 0x32, 0x87, 0xd7, 0x34, 0x7e, 0x6c, 0xba, 0xf2, 0xb0, 0x33, 0xff, + 0xfb, 0x61, 0xf7, 0xa1, 0x9c, 0x52, 0x2a, 0xdf, 0x60, 0x5a, 0x6d, 0xff, 0xa9, 0x0a, 0x45, 0x4c, + 0x79, 0xc0, 0x7c, 0x4e, 0x51, 0x7d, 0x42, 0xeb, 0x4e, 0xcb, 0x37, 0x05, 0x4a, 0x8b, 0xdd, 0x87, + 0x50, 0x8a, 0xd5, 0xab, 0xa3, 0x7b, 0xd1, 0xad, 0xcb, 0x49, 0x71, 0x13, 0x71, 0xf0, 0x98, 0x81, + 0x3e, 0x82, 0x82, 0xd0, 0xb1, 0xae, 0x4e, 0xbb, 0xd7, 0xa5, 0xb2, 0x26, 0x37, 0x14, 0x08, 0xc7, + 0x68, 0xf4, 0x1d, 0xc8, 0x0b, 0x41, 0x4b, 0x1d, 0x5d, 0x49, 0x37, 0x2f, 0xe7, 0x75, 0x25, 0x06, + 0x6b, 0xac, 0x60, 0x09, 0x5d, 0x4b, 0x63, 0x01, 0x3c, 0x83, 0x75, 0x28, 0x31, 0x58, 0x63, 0x45, + 0x90, 0x52, 0xdc, 0x52, 0x47, 0xeb, 0xe0, 0x19, 0x41, 0x1e, 0x28, 0x10, 0x8e, 0xd1, 0xe8, 0x11, + 0xac, 0x48, 0x91, 0x4a, 0x9d, 0x58, 0xdc, 0x2a, 0x55, 0xfc, 0xee, 0x8c, 0x6d, 0x55, 0x58, 0xad, + 0x6f, 0x97, 0x79, 0x7a, 0x88, 0x0e, 0xa0, 0x92, 0x12, 0xab, 0x8e, 0x96, 0xc9, 0xdb, 0x33, 0xb6, + 0x2b, 0x85, 0xc4, 0x13, 0xbc, 0xf9, 0x2a, 0xf7, 0x77, 0x19, 0xad, 0x72, 0x4d, 0x28, 0xc6, 0x12, + 0x51, 0x57, 0x98, 0x64, 0x8c, 0x0e, 0x00, 0x69, 0xf1, 0xc0, 0xed, 0x3e, 0x1d, 0x90, 0xc5, 0x53, + 0xbd, 0xa2, 0x78, 0xa7, 0x92, 0x86, 0x3e, 0x83, 0xb7, 0xa7, 0x35, 0x51, 0xda, 0xe1, 0x22, 0xf2, + 0x70, 0x7d, 0x52, 0x1a, 0x69, 0xc7, 0xff, 0x0f, 0x6b, 0x0e, 0xb3, 0x87, 0x03, 0xea, 0x47, 0xb2, + 0x19, 0x59, 0xc3, 0x50, 0x69, 0x8c, 0x12, 0xae, 0x4e, 0x4c, 0x3c, 0x09, 0x3d, 0xf4, 0x1e, 0xe4, + 0x19, 0x19, 0x46, 0xfd, 0x3d, 0x9d, 0x12, 0x15, 0xd5, 0x8f, 0xba, 0x0d, 0x61, 0xc3, 0x7a, 0xee, + 0x51, 0xb6, 0x98, 0xaf, 0x16, 0xcc, 0x5f, 0xe5, 0xa1, 0x94, 0xa4, 0x31, 0x6a, 0xa5, 0x34, 0xa9, + 0x21, 0xd5, 0xd3, 0x07, 0x57, 0x64, 0xfe, 0xeb, 0xaa, 0xd4, 0xfc, 0x73, 0x06, 0xa0, 0xc5, 0x7c, + 0x1e, 0x85, 0x44, 0x14, 0x94, 0x16, 0x64, 0x53, 0x42, 0x7b, 0xf7, 0x2a, 0x7f, 0x63, 0x66, 0x5d, + 0x0a, 0x6e, 0x49, 0x16, 0x25, 0x30, 0xa4, 0x24, 0xde, 0xc3, 0x12, 0xd6, 0x23, 0xa1, 0x1e, 0xce, + 0x99, 0xe7, 0x50, 0xc7, 0x92, 0x42, 0x4c, 0x6f, 0x49, 0x59, 0xd9, 0xa4, 0xaa, 0xda, 0xfe, 0x83, + 0x01, 0x59, 0xa9, 0xd5, 0xcb, 0x50, 0xe8, 0x1c, 0x3f, 0x6d, 0x1c, 0x76, 0xf6, 0xab, 0xd7, 0x10, + 0x82, 0x95, 0x83, 0x4e, 0xfb, 0x70, 0xdf, 0xc2, 0xed, 0x4f, 0x9f, 0x74, 0x70, 0x7b, 0xbf, 0x6a, + 0xa0, 0x0d, 0x58, 0x3b, 0xec, 0xb6, 0x1a, 0x8f, 0x3b, 0xdd, 0xe3, 0xb1, 0x39, 0x83, 0x6a, 0xb0, + 0x9e, 0x32, 0xb7, 0xba, 0x47, 0x47, 0xed, 0xe3, 0xfd, 0xf6, 0x7e, 0x75, 0x69, 0xec, 0xa4, 0x7b, + 0x22, 0x66, 0x1b, 0x87, 0xd5, 0x2c, 0x7a, 0x0b, 0x56, 0x95, 0xed, 0xa0, 0x8b, 0x9b, 0x9d, 0xfd, + 0xfd, 0xf6, 0x71, 0x35, 0x87, 0xaa, 0x50, 0xe9, 0x1c, 0xb7, 0xba, 0x47, 0x27, 0x8d, 0xc7, 0x9d, + 0xe6, 0x61, 0xbb, 0x9a, 0x47, 0x6b, 0xb0, 0xfc, 0xe4, 0xf8, 0xb4, 0xf1, 0xb8, 0x73, 0x7a, 0xd0, + 0x69, 0x08, 0x53, 0xc1, 0xfc, 0x69, 0x4a, 0x5c, 0x7f, 0x0f, 0xae, 0xdb, 0x84, 0x53, 0xcb, 0xf5, + 0x39, 0xf5, 0xb9, 0x1b, 0xb9, 0xcf, 0xa9, 0x5a, 0x21, 0x97, 0x67, 0x5a, 0xc4, 0x1b, 0x62, 0xba, + 0x33, 0x9e, 0x95, 0x6b, 0xe5, 0xe8, 0x73, 0xf9, 0x3e, 0xa2, 0x37, 0x30, 0x3e, 0xc3, 0x8f, 0x17, + 0x3c, 0xc3, 0xd4, 0xde, 0x73, 0xa9, 0x3f, 0x71, 0xda, 0x99, 0x68, 0x7c, 0x49, 0x72, 0x07, 0x24, + 0xea, 0xd7, 0x32, 0x52, 0xc7, 0x56, 0x62, 0xe3, 0x09, 0x89, 0xfa, 0x02, 0xe4, 0x50, 0x2f, 0x22, + 0xd6, 0x30, 0x10, 0xbe, 0xb9, 0x3c, 0xaf, 0x22, 0xae, 0x48, 0xe3, 0x13, 0x65, 0x43, 0x75, 0x00, + 0x4e, 0x43, 0x2b, 0x60, 0x9e, 0x6b, 0x8f, 0x74, 0xb5, 0xd3, 0xa2, 0xe9, 0x94, 0x86, 0x27, 0xd2, + 0x8c, 0x4b, 0x3c, 0x7e, 0x34, 0x9f, 0x41, 0x75, 0x3a, 0xb4, 0x4b, 0xa4, 0xf1, 0x0f, 0xd2, 0xd2, + 0xb8, 0xbc, 0x77, 0x67, 0xf1, 0x4c, 0x4b, 0xcb, 0x68, 0x07, 0x0a, 0xba, 0x32, 0xa3, 0x0f, 0x01, + 0x11, 0xf9, 0x8e, 0x61, 0x39, 0x94, 0xdb, 0xa1, 0x1b, 0x24, 0xc2, 0xb1, 0x84, 0xd7, 0xd4, 0xcc, + 0xfe, 0x78, 0x02, 0xdd, 0x01, 0x25, 0xd7, 0xa6, 0xdf, 0x5f, 0xd2, 0x0d, 0x2a, 0x56, 0x74, 0xbf, + 0x30, 0x20, 0xaf, 0x0a, 0xf9, 0x9b, 0xe9, 0xda, 0x0f, 0xe0, 0x86, 0xe3, 0x72, 0x72, 0xe6, 0x51, + 0x4b, 0x54, 0x78, 0x8b, 0x05, 0x91, 0x3b, 0x88, 0xa5, 0x6e, 0x46, 0x1e, 0xc1, 0x75, 0x0d, 0x10, + 0x9d, 0xa0, 0x9b, 0x9a, 0x36, 0x3f, 0x83, 0xbc, 0xea, 0x0e, 0xf3, 0x35, 0x58, 0xa2, 0x67, 0x32, + 0x0b, 0xea, 0x19, 0xf3, 0xbb, 0x50, 0xd0, 0xfd, 0x63, 0xbc, 0x37, 0xc6, 0xd5, 0x7b, 0xf3, 0x09, + 0x2c, 0x4f, 0xb4, 0x8d, 0xaf, 0x45, 0x7e, 0x00, 0x95, 0x74, 0xa7, 0xf8, 0x3a, 0xdc, 0xed, 0x9f, + 0x65, 0x21, 0xd7, 0xfe, 0x22, 0x0a, 0x89, 0xf9, 0x37, 0x03, 0x6e, 0xc7, 0x89, 0xd2, 0x16, 0x22, + 0xcc, 0xf5, 0x7b, 0x27, 0x21, 0x7b, 0xa6, 0x5e, 0x39, 0xe3, 0x8f, 0x68, 0x87, 0x50, 0xa5, 0x7a, + 0xd2, 0x4a, 0xef, 0x5b, 0x79, 0xef, 0xf6, 0xec, 0xcf, 0x09, 0x71, 0xbd, 0x5c, 0x8d, 0xa9, 0xf1, + 0x95, 0x3f, 0x81, 0x6a, 0x10, 0xb2, 0x80, 0x71, 0xea, 0x24, 0xde, 0x54, 0x26, 0x2d, 0xf8, 0x5d, + 0x60, 0x35, 0xa6, 0x6b, 0x83, 0x28, 0xc4, 0xc9, 0x2a, 0xb4, 0xad, 0xd1, 0x23, 0xae, 0xcf, 0xa3, + 0xd4, 0x6d, 0x42, 0x9f, 0x4c, 0x1e, 0xfa, 0x42, 0xc1, 0x27, 0x79, 0xd1, 0x9b, 0xac, 0x37, 0x19, + 0x59, 0x6f, 0xda, 0x13, 0xf1, 0xca, 0x1d, 0xad, 0x5f, 0x19, 0xc7, 0xfc, 0xe2, 0xf3, 0x6d, 0x96, + 0x80, 0xbd, 0x1f, 0x42, 0x29, 0x49, 0x10, 0xf4, 0x7d, 0x28, 0x8f, 0x77, 0x82, 0xa2, 0xf5, 0xcb, + 0xce, 0xc2, 0xdc, 0xb8, 0xf4, 0x8f, 0x76, 0x8c, 0xbb, 0x46, 0xb3, 0xf9, 0xf2, 0x5f, 0x37, 0xaf, + 0xbd, 0xfc, 0xf2, 0xa6, 0xf1, 0x8f, 0x2f, 0x6f, 0x1a, 0xbf, 0xff, 0xf7, 0x4d, 0xe3, 0xf3, 0xbb, + 0x0b, 0x7d, 0xc4, 0x4a, 0x39, 0x3c, 0xcb, 0x4b, 0xf3, 0xbd, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, + 0x30, 0x9a, 0x31, 0x91, 0x51, 0x16, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -2194,6 +2219,13 @@ func (m *Request_Flush) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.ConnectorStatePatchesJson) > 0 { + i -= len(m.ConnectorStatePatchesJson) + copy(dAtA[i:], m.ConnectorStatePatchesJson) + i = encodeVarintMaterialize(dAtA, i, uint64(len(m.ConnectorStatePatchesJson))) + i-- + dAtA[i] = 0xa + } return len(dAtA) - i, nil } @@ -2308,6 +2340,13 @@ func (m *Request_StartCommit) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.ConnectorStatePatchesJson) > 0 { + i -= len(m.ConnectorStatePatchesJson) + copy(dAtA[i:], m.ConnectorStatePatchesJson) + i = encodeVarintMaterialize(dAtA, i, uint64(len(m.ConnectorStatePatchesJson))) + i-- + dAtA[i] = 0x12 + } if m.RuntimeCheckpoint != nil { { size, err := m.RuntimeCheckpoint.MarshalToSizedBuffer(dAtA[:i]) @@ -2347,6 +2386,13 @@ func (m *Request_Acknowledge) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.ConnectorStatePatchesJson) > 0 { + i -= len(m.ConnectorStatePatchesJson) + copy(dAtA[i:], m.ConnectorStatePatchesJson) + i = encodeVarintMaterialize(dAtA, i, uint64(len(m.ConnectorStatePatchesJson))) + i-- + dAtA[i] = 0xa + } return len(dAtA) - i, nil } @@ -3369,6 +3415,10 @@ func (m *Request_Flush) ProtoSize() (n int) { } var l int _ = l + l = len(m.ConnectorStatePatchesJson) + if l > 0 { + n += 1 + l + sovMaterialize(uint64(l)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -3426,6 +3476,10 @@ func (m *Request_StartCommit) ProtoSize() (n int) { l = m.RuntimeCheckpoint.ProtoSize() n += 1 + l + sovMaterialize(uint64(l)) } + l = len(m.ConnectorStatePatchesJson) + if l > 0 { + n += 1 + l + sovMaterialize(uint64(l)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -3438,6 +3492,10 @@ func (m *Request_Acknowledge) ProtoSize() (n int) { } var l int _ = l + l = len(m.ConnectorStatePatchesJson) + if l > 0 { + n += 1 + l + sovMaterialize(uint64(l)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -5403,6 +5461,40 @@ func (m *Request_Flush) Unmarshal(dAtA []byte) error { return fmt.Errorf("proto: Flush: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorStatePatchesJson", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaterialize + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthMaterialize + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthMaterialize + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ConnectorStatePatchesJson = append(m.ConnectorStatePatchesJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorStatePatchesJson == nil { + m.ConnectorStatePatchesJson = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipMaterialize(dAtA[iNdEx:]) @@ -5770,6 +5862,40 @@ func (m *Request_StartCommit) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorStatePatchesJson", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaterialize + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthMaterialize + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthMaterialize + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ConnectorStatePatchesJson = append(m.ConnectorStatePatchesJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorStatePatchesJson == nil { + m.ConnectorStatePatchesJson = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipMaterialize(dAtA[iNdEx:]) @@ -5821,6 +5947,40 @@ func (m *Request_Acknowledge) Unmarshal(dAtA []byte) error { return fmt.Errorf("proto: Acknowledge: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorStatePatchesJson", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaterialize + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthMaterialize + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthMaterialize + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ConnectorStatePatchesJson = append(m.ConnectorStatePatchesJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorStatePatchesJson == nil { + m.ConnectorStatePatchesJson = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipMaterialize(dAtA[iNdEx:]) diff --git a/go/protocols/materialize/materialize.proto b/go/protocols/materialize/materialize.proto index 7765ea414fc..949b475a1a7 100644 --- a/go/protocols/materialize/materialize.proto +++ b/go/protocols/materialize/materialize.proto @@ -161,7 +161,19 @@ message Request { // Flush loads. No further Loads will be sent in this transaction, // and the runtime will await the connectors's remaining Loaded // responses followed by one Flushed response. - message Flush {} + message Flush { + // Aggregated connector state patches from all shards' + // prior-transaction C:Acknowledged responses, in State Update Wire + // Format. Includes this shard's own patches (the runtime feeds the + // shard's contribution back to it for symmetry with the scaled-out + // case). Connectors participating in cooperative multi-shard + // strategies use this to observe peers' state. The runtime forwards + // it verbatim from the leader's L:Flush. + bytes connector_state_patches_json = 1 [ + (gogoproto.casttype) = "encoding/json.RawMessage", + json_name = "connectorStatePatches" + ]; + } Flush flush = 6; // Store documents updated by the current transaction. @@ -201,12 +213,34 @@ message Request { message StartCommit { // Flow runtime checkpoint to commit with this transaction. consumer.Checkpoint runtime_checkpoint = 1; + // Aggregated connector state patches from all shards' current-transaction + // C:Flushed responses, in State Update Wire Format. Includes this + // shard's own patches (the runtime feeds the shard's contribution back + // to it for symmetry with the scaled-out case). Connectors use this + // to observe peers' Flushed state for cooperative strategies like + // parallel file staging. The runtime forwards it verbatim from the + // leader's L:StartCommit. + bytes connector_state_patches_json = 2 [ + (gogoproto.casttype) = "encoding/json.RawMessage", + json_name = "connectorStatePatches" + ]; } StartCommit start_commit = 8; // Acknowledge to the connector that the previous transaction // has committed to the Flow runtime's recovery log. - message Acknowledge {} + message Acknowledge { + // Aggregated connector state patches from all shards' just-committed + // C:StartedCommit responses, in State Update Wire Format. Includes + // this shard's own patches (the runtime feeds the shard's contribution + // back to it for symmetry with the scaled-out case). Connectors use + // this to observe peers' StartedCommit state. The runtime forwards + // it verbatim from the leader's L:Acknowledge. + bytes connector_state_patches_json = 1 [ + (gogoproto.casttype) = "encoding/json.RawMessage", + json_name = "connectorStatePatches" + ]; + } Acknowledge acknowledge = 9; // Reserved for internal use. diff --git a/go/protocols/runtime/runtime.pb.go b/go/protocols/runtime/runtime.pb.go index 117aa85549e..4db9a9027ea 100644 --- a/go/protocols/runtime/runtime.pb.go +++ b/go/protocols/runtime/runtime.pb.go @@ -8,9 +8,12 @@ import ( encoding_binary "encoding/binary" encoding_json "encoding/json" fmt "fmt" + _ "github.com/estuary/flow/go/protocols/derive" flow "github.com/estuary/flow/go/protocols/flow" github_com_estuary_flow_go_protocols_flow "github.com/estuary/flow/go/protocols/flow" + materialize "github.com/estuary/flow/go/protocols/materialize" ops "github.com/estuary/flow/go/protocols/ops" + shuffle "github.com/estuary/flow/go/protocols/shuffle" _ "github.com/gogo/protobuf/gogoproto" proto "github.com/gogo/protobuf/proto" go_gazette_dev_core_broker_protocol "go.gazette.dev/core/broker/protocol" @@ -1348,1814 +1351,2343 @@ func (m *ConnectorProxyResponse) XXX_DiscardUnknown() { var xxx_messageInfo_ConnectorProxyResponse proto.InternalMessageInfo -func init() { - proto.RegisterEnum("runtime.Plane", Plane_name, Plane_value) - 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") - proto.RegisterType((*RocksDBDescriptor)(nil), "runtime.RocksDBDescriptor") - proto.RegisterType((*Container)(nil), "runtime.Container") - proto.RegisterMapType((map[uint32]string)(nil), "runtime.Container.MappedHostPortsEntry") - proto.RegisterType((*CaptureRequestExt)(nil), "runtime.CaptureRequestExt") - proto.RegisterType((*CaptureRequestExt_StartCommit)(nil), "runtime.CaptureRequestExt.StartCommit") - proto.RegisterType((*CaptureResponseExt)(nil), "runtime.CaptureResponseExt") - proto.RegisterType((*CaptureResponseExt_Opened)(nil), "runtime.CaptureResponseExt.Opened") - proto.RegisterType((*CaptureResponseExt_Captured)(nil), "runtime.CaptureResponseExt.Captured") - proto.RegisterType((*CaptureResponseExt_Checkpoint)(nil), "runtime.CaptureResponseExt.Checkpoint") - proto.RegisterType((*DeriveRequestExt)(nil), "runtime.DeriveRequestExt") - proto.RegisterType((*DeriveRequestExt_Open)(nil), "runtime.DeriveRequestExt.Open") - proto.RegisterType((*DeriveResponseExt)(nil), "runtime.DeriveResponseExt") - proto.RegisterType((*DeriveResponseExt_Opened)(nil), "runtime.DeriveResponseExt.Opened") - proto.RegisterType((*DeriveResponseExt_Published)(nil), "runtime.DeriveResponseExt.Published") - proto.RegisterType((*DeriveResponseExt_Flushed)(nil), "runtime.DeriveResponseExt.Flushed") - proto.RegisterType((*MaterializeRequestExt)(nil), "runtime.MaterializeRequestExt") - proto.RegisterType((*MaterializeResponseExt)(nil), "runtime.MaterializeResponseExt") - proto.RegisterType((*MaterializeResponseExt_Flushed)(nil), "runtime.MaterializeResponseExt.Flushed") - proto.RegisterType((*CombineRequest)(nil), "runtime.CombineRequest") - proto.RegisterType((*CombineRequest_Open)(nil), "runtime.CombineRequest.Open") - proto.RegisterType((*CombineRequest_Open_Binding)(nil), "runtime.CombineRequest.Open.Binding") - proto.RegisterType((*CombineRequest_Add)(nil), "runtime.CombineRequest.Add") - proto.RegisterType((*CombineResponse)(nil), "runtime.CombineResponse") - proto.RegisterType((*ConnectorProxyRequest)(nil), "runtime.ConnectorProxyRequest") - proto.RegisterType((*ConnectorProxyResponse)(nil), "runtime.ConnectorProxyResponse") +// Join is sent by each shard to the leader at session start. The leader +// gathers all expected Joins, verifies `shards` are exactly equal, +// and responds with Joined on consensus or disagreement. +// +// Index 0 in `shards` is "shard zero": its reactor is co-located +// with the Shuffle Leader sidecar that owns this session, and it hosts +// the task's recovery log. +type Join struct { + // Etcd mod revision at which `shards` was observed. + // On disagreement all participants await the highest revision, then re-try. + EtcdModRevision int64 `protobuf:"varint,1,opt,name=etcd_mod_revision,json=etcdModRevision,proto3" json:"etcd_mod_revision,omitempty"` + // Current shard topology as observed by the sender. + Shards []*Join_Shard `protobuf:"bytes,2,rep,name=shards,proto3" json:"shards,omitempty"` + // This shard's index within `shards`. + ShardIndex uint32 `protobuf:"varint,3,opt,name=shard_index,json=shardIndex,proto3" json:"shard_index,omitempty"` + // Client-owned local directory where this shuffle Log segments should be written. + ShuffleDirectory string `protobuf:"bytes,4,opt,name=shuffle_directory,json=shuffleDirectory,proto3" json:"shuffle_directory,omitempty"` + // Dial-able gRPC endpoint for reaching this shard's shuffle service. + ShuffleEndpoint string `protobuf:"bytes,5,opt,name=shuffle_endpoint,json=shuffleEndpoint,proto3" json:"shuffle_endpoint,omitempty"` + // Dial-able gRPC endpoint for reaching the shuffle Leader's shuffle service. + LeaderEndpoint string `protobuf:"bytes,6,opt,name=leader_endpoint,json=leaderEndpoint,proto3" json:"leader_endpoint,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func init() { - proto.RegisterFile("go/protocols/runtime/runtime.proto", fileDescriptor_73af6e0737ce390c) +func (m *Join) Reset() { *m = Join{} } +func (m *Join) String() string { return proto.CompactTextString(m) } +func (*Join) ProtoMessage() {} +func (*Join) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{15} } - -var fileDescriptor_73af6e0737ce390c = []byte{ - // 2062 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x38, 0xbd, 0x73, 0x1b, 0xc7, - 0xf5, 0x3a, 0x7c, 0x10, 0xb8, 0x07, 0x90, 0x02, 0x77, 0x64, 0x09, 0x86, 0xf5, 0x23, 0x69, 0x58, - 0xfe, 0x85, 0x63, 0xc9, 0x20, 0x43, 0x67, 0x12, 0xc7, 0x93, 0x78, 0x44, 0x80, 0x64, 0x04, 0x85, - 0x14, 0x91, 0x25, 0xa5, 0x99, 0xa4, 0xb9, 0x39, 0xde, 0x2e, 0x80, 0x13, 0x0e, 0xb7, 0xe7, 0xdd, - 0x3b, 0x4a, 0xf4, 0xbf, 0x90, 0x22, 0x4d, 0xba, 0x34, 0xa9, 0xf3, 0x27, 0xb8, 0x48, 0xad, 0x32, - 0x93, 0x22, 0x93, 0x8a, 0x33, 0x71, 0xda, 0x94, 0x69, 0xc2, 0x49, 0x91, 0xd9, 0x8f, 0x3b, 0x80, - 0x80, 0xa8, 0x28, 0xb4, 0x0b, 0x17, 0x24, 0xf6, 0x7d, 0xee, 0x7b, 0x6f, 0xdf, 0x7b, 0xfb, 0xf6, - 0xa0, 0x39, 0x60, 0x1b, 0x11, 0x67, 0x31, 0xf3, 0x58, 0x20, 0x36, 0x78, 0x12, 0xc6, 0xfe, 0x98, - 0xa6, 0xbf, 0x2d, 0x45, 0x41, 0x25, 0x03, 0x36, 0x56, 0x4e, 0x38, 0x1b, 0x51, 0x9e, 0x09, 0x64, - 0x0b, 0xcd, 0xd8, 0x58, 0xf3, 0x58, 0x28, 0x92, 0xf1, 0x1b, 0x38, 0xee, 0x5e, 0xda, 0xae, 0x1f, - 0xb0, 0x17, 0xea, 0x9f, 0xa1, 0x36, 0x2e, 0x51, 0x59, 0xa4, 0xfe, 0x0c, 0xed, 0xd6, 0x80, 0x0d, - 0x98, 0x5a, 0x6e, 0xc8, 0x95, 0xc6, 0x36, 0xff, 0x68, 0xc1, 0xf2, 0xb1, 0x2b, 0x46, 0x47, 0x94, - 0x9f, 0xfa, 0x1e, 0xed, 0xb0, 0xb0, 0xef, 0x0f, 0xd0, 0x0a, 0x54, 0x02, 0x36, 0x70, 0xfa, 0x7e, - 0x40, 0x9d, 0x3e, 0xa9, 0x5b, 0x6b, 0xd6, 0x7a, 0x11, 0xdb, 0x01, 0x1b, 0xec, 0xf9, 0x01, 0xdd, - 0x23, 0xe8, 0x3d, 0xb0, 0x63, 0x57, 0x8c, 0x9c, 0xd0, 0x1d, 0xd3, 0x7a, 0x6e, 0xcd, 0x5a, 0xb7, - 0x71, 0x59, 0x22, 0x9e, 0xb8, 0x63, 0x8a, 0xde, 0x85, 0x72, 0x42, 0x84, 0x13, 0xb9, 0xf1, 0xb0, - 0x9e, 0x57, 0xb4, 0x52, 0x42, 0x44, 0xcf, 0x8d, 0x87, 0xe8, 0x3e, 0x2c, 0x7b, 0x2c, 0x8c, 0x5d, - 0x3f, 0xa4, 0xdc, 0x09, 0x69, 0xfc, 0x82, 0xf1, 0x51, 0xbd, 0xa0, 0x78, 0x6a, 0x19, 0xe1, 0x89, - 0xc6, 0xa3, 0x7b, 0x50, 0x8c, 0x02, 0x37, 0xa4, 0xf5, 0x85, 0x35, 0x6b, 0x7d, 0x69, 0x6b, 0xa9, - 0x95, 0x06, 0xb5, 0x27, 0xb1, 0x58, 0x13, 0x9b, 0xff, 0x2a, 0xc0, 0xd2, 0xd1, 0x30, 0xe9, 0xf7, - 0x03, 0x8a, 0xe9, 0x17, 0x09, 0x15, 0x31, 0xea, 0x42, 0xe9, 0x39, 0x4b, 0x78, 0xe8, 0x06, 0xca, - 0x72, 0xbb, 0xbd, 0x71, 0x71, 0xbe, 0x7a, 0x7f, 0xc0, 0x5a, 0x03, 0xf7, 0x4b, 0x1a, 0xc7, 0xb4, - 0x45, 0xe8, 0xe9, 0x86, 0xc7, 0x38, 0xdd, 0x98, 0x39, 0x8e, 0xd6, 0x63, 0x2d, 0x86, 0x53, 0x79, - 0x74, 0x1b, 0x16, 0x38, 0x8d, 0x02, 0xf7, 0x4c, 0x79, 0x59, 0xc6, 0x06, 0x92, 0x3e, 0x9e, 0x24, - 0x7e, 0x40, 0x1c, 0x9f, 0xa4, 0x3e, 0x2a, 0xb8, 0x4b, 0xd0, 0x1e, 0x2c, 0xb0, 0x7e, 0x5f, 0xd0, - 0x58, 0x39, 0x96, 0x6f, 0xb7, 0x2e, 0xce, 0x57, 0x3f, 0x7a, 0x9b, 0xcd, 0x0f, 0x95, 0x14, 0x36, - 0xd2, 0xe8, 0x00, 0x80, 0x86, 0xc4, 0x31, 0xba, 0x8a, 0xd7, 0xd2, 0x65, 0xd3, 0x90, 0xe8, 0x25, - 0xba, 0x0f, 0x45, 0xee, 0x86, 0x03, 0x1d, 0xcd, 0xca, 0xd6, 0xcd, 0x96, 0x4a, 0x1b, 0x2c, 0x51, - 0x47, 0x11, 0xf5, 0xda, 0x85, 0x57, 0xe7, 0xab, 0x37, 0xb0, 0xe6, 0x41, 0x47, 0x50, 0xf1, 0x18, - 0xe3, 0xc4, 0x0f, 0xdd, 0x98, 0xf1, 0x7a, 0x49, 0x45, 0xf1, 0xfb, 0x17, 0xe7, 0xab, 0x1f, 0xbf, - 0x6e, 0xf3, 0xb9, 0xa4, 0x6d, 0x1d, 0x0d, 0x5d, 0x4e, 0xba, 0x3b, 0x78, 0x5a, 0x0b, 0xda, 0x04, - 0xe0, 0x54, 0xb0, 0x20, 0x89, 0x7d, 0x16, 0xd6, 0xcb, 0xca, 0x8c, 0x5a, 0x2b, 0x93, 0x79, 0x44, - 0x5d, 0x42, 0x39, 0x9e, 0xe2, 0x41, 0x1f, 0xc0, 0xa2, 0xd0, 0x47, 0xeb, 0xf8, 0x21, 0xa1, 0x2f, - 0xeb, 0xf6, 0x9a, 0xb5, 0xbe, 0x88, 0xab, 0x06, 0xd9, 0x95, 0x38, 0xf4, 0x03, 0x00, 0x42, 0xb9, - 0x7f, 0xea, 0x2a, 0xb5, 0xa0, 0xd4, 0xde, 0xd2, 0xde, 0x75, 0x58, 0x10, 0x50, 0x4f, 0xe2, 0xa5, - 0x8b, 0x78, 0x8a, 0x0f, 0x75, 0xe0, 0xe6, 0xd8, 0x8d, 0x29, 0xf7, 0xdd, 0xc0, 0xff, 0x52, 0x8b, - 0x56, 0x94, 0xe8, 0xbb, 0x5a, 0xf4, 0xe0, 0x32, 0x51, 0xc9, 0xcf, 0x4a, 0x34, 0xff, 0x5c, 0x80, - 0x9b, 0x59, 0xee, 0x89, 0x88, 0x85, 0x82, 0xa2, 0x75, 0x58, 0x10, 0xb1, 0x1b, 0x27, 0x42, 0xe5, - 0xde, 0xd2, 0x56, 0xad, 0x95, 0x86, 0xa7, 0x75, 0xa4, 0xf0, 0xd8, 0xd0, 0x25, 0xe7, 0x50, 0xf9, - 0xac, 0x72, 0xeb, 0x75, 0xb1, 0x30, 0x74, 0xf4, 0x21, 0x2c, 0xc5, 0x94, 0x8f, 0xfd, 0xd0, 0x0d, - 0x1c, 0xca, 0x39, 0xe3, 0x26, 0xe7, 0x16, 0x53, 0xec, 0xae, 0x44, 0xa2, 0x5f, 0x40, 0x95, 0x53, - 0x97, 0x38, 0xf1, 0x90, 0xb3, 0x64, 0x30, 0xbc, 0x66, 0xfe, 0x55, 0xa4, 0x8e, 0x63, 0xad, 0x42, - 0x26, 0xe1, 0x0b, 0xee, 0xc7, 0xd4, 0x91, 0x96, 0x5c, 0x37, 0x09, 0x95, 0x06, 0xe9, 0x12, 0xea, - 0x42, 0xd1, 0xe5, 0x34, 0x74, 0x55, 0x12, 0x56, 0xdb, 0x9f, 0x5c, 0x9c, 0xaf, 0x6e, 0x0c, 0xfc, - 0x78, 0x98, 0x9c, 0xb4, 0x3c, 0x36, 0xde, 0xa0, 0x22, 0x4e, 0x5c, 0x7e, 0xa6, 0xdb, 0xda, 0x5c, - 0xa3, 0x6b, 0x6d, 0x4b, 0x51, 0xac, 0x35, 0xa0, 0x0f, 0xa1, 0x40, 0x98, 0x27, 0xea, 0xa5, 0xb5, - 0xfc, 0x7a, 0x65, 0xab, 0xa2, 0x4f, 0xed, 0x28, 0xf0, 0x3d, 0x6a, 0x52, 0x59, 0x91, 0xd1, 0x23, - 0x28, 0xe9, 0x0a, 0x12, 0xf5, 0xf2, 0x5a, 0xfe, 0x1a, 0xd6, 0xa7, 0xe2, 0x32, 0xcf, 0x92, 0xc4, - 0x27, 0x4e, 0xe4, 0xf2, 0x58, 0xd4, 0x6d, 0xb5, 0xad, 0xa9, 0xa2, 0xa7, 0x4f, 0xbb, 0x3b, 0x3d, - 0x89, 0x36, 0x5b, 0xdb, 0x92, 0x51, 0x21, 0x64, 0xd2, 0x47, 0xae, 0x37, 0xa2, 0xc4, 0x19, 0xd1, - 0xb3, 0x3a, 0x5c, 0x65, 0xac, 0xad, 0x99, 0x7e, 0x4e, 0xcf, 0x9a, 0x04, 0x96, 0x31, 0xf3, 0x46, - 0x62, 0xa7, 0xbd, 0x43, 0x85, 0xc7, 0xfd, 0x48, 0xd6, 0xce, 0x03, 0x40, 0x5c, 0x22, 0xc9, 0x89, - 0x43, 0xc3, 0x53, 0x67, 0x4c, 0xc7, 0x51, 0xcc, 0x55, 0x86, 0x2d, 0xe0, 0x9a, 0xa1, 0xec, 0x86, - 0xa7, 0x07, 0x0a, 0x8f, 0xde, 0x87, 0x6a, 0xca, 0xad, 0xba, 0xb0, 0xee, 0xd0, 0x15, 0x83, 0x93, - 0x9d, 0xb8, 0xf9, 0xdb, 0x1c, 0xd8, 0x9d, 0xb4, 0xe3, 0xa2, 0x3b, 0x50, 0xf2, 0x23, 0xc7, 0x25, - 0x44, 0xeb, 0xb4, 0xf1, 0x82, 0x1f, 0x6d, 0x13, 0xc2, 0xd1, 0x0f, 0x61, 0xd1, 0xb4, 0x69, 0x27, - 0x62, 0xd2, 0xef, 0x9c, 0xf2, 0x60, 0x59, 0x7b, 0x60, 0x3a, 0x75, 0x8f, 0xf1, 0x18, 0x57, 0xc3, - 0x09, 0x20, 0xd0, 0x11, 0x2c, 0x8f, 0xdd, 0x28, 0xa2, 0xc4, 0x19, 0x32, 0x11, 0x1b, 0xd9, 0xbc, - 0x92, 0xfd, 0x5e, 0xd6, 0xc7, 0xb3, 0xfd, 0x5b, 0x07, 0x8a, 0xf7, 0x11, 0x13, 0xb1, 0x12, 0xdf, - 0x0d, 0x63, 0x7e, 0x26, 0xcb, 0xed, 0x12, 0x16, 0xfd, 0x1f, 0x40, 0x22, 0xdc, 0x01, 0x75, 0xb8, - 0x1b, 0x53, 0x95, 0xdd, 0x39, 0x6c, 0x2b, 0x0c, 0x76, 0x63, 0xda, 0x68, 0xc3, 0xad, 0xd7, 0xe9, - 0x41, 0x35, 0xc8, 0xcb, 0xd8, 0x5b, 0xaa, 0x77, 0xc8, 0x25, 0xba, 0x05, 0xc5, 0x53, 0x37, 0x48, - 0xd2, 0xab, 0x4b, 0x03, 0x9f, 0xe5, 0x3e, 0xb5, 0x9a, 0x7f, 0xc8, 0xc1, 0x72, 0xc7, 0x8d, 0xe2, - 0x84, 0xa7, 0xb7, 0xc9, 0xee, 0x4b, 0xd9, 0x3b, 0xe5, 0xdd, 0xe7, 0x04, 0xf4, 0x94, 0x06, 0xa6, - 0xac, 0x97, 0x5a, 0xf2, 0x66, 0xdd, 0x67, 0x83, 0xd6, 0xbe, 0xc4, 0xe2, 0x72, 0xc0, 0x06, 0x6a, - 0x85, 0xba, 0x93, 0xa3, 0x22, 0xd9, 0x01, 0x9a, 0x12, 0x6f, 0x64, 0xbe, 0xcf, 0x1d, 0x31, 0x5e, - 0x36, 0x52, 0x53, 0xa7, 0xde, 0x85, 0xaa, 0x88, 0x5d, 0x1e, 0x3b, 0x1e, 0x1b, 0x8f, 0xfd, 0x58, - 0x55, 0x7d, 0x65, 0xeb, 0xff, 0x27, 0x01, 0x9c, 0xb5, 0x54, 0xb6, 0x18, 0x1e, 0x77, 0x14, 0x37, - 0xae, 0x88, 0x09, 0xd0, 0xc0, 0x50, 0x99, 0xa2, 0xa1, 0x0e, 0x20, 0xa3, 0xc4, 0xf1, 0x86, 0xd4, - 0x1b, 0x45, 0xcc, 0x0f, 0x63, 0xe5, 0x9a, 0x6c, 0x9e, 0x59, 0xc7, 0xea, 0x64, 0x34, 0xbc, 0x6c, - 0xf8, 0x27, 0xa8, 0xe6, 0xbf, 0x0b, 0x80, 0x32, 0x13, 0x74, 0xfb, 0x93, 0xd1, 0xda, 0x04, 0x3b, - 0xbb, 0xcb, 0x8d, 0x4a, 0x34, 0x7f, 0xe6, 0x78, 0xc2, 0x84, 0x3e, 0x83, 0x05, 0x16, 0xd1, 0x90, - 0x12, 0x13, 0xa6, 0xe6, 0xbc, 0x87, 0x99, 0xfa, 0xd6, 0xa1, 0xe2, 0xc4, 0x46, 0x02, 0x3d, 0x84, - 0xb2, 0xa7, 0x99, 0x88, 0x89, 0xcf, 0xbd, 0x37, 0x49, 0x1b, 0x14, 0xc1, 0x99, 0x14, 0xda, 0x03, - 0x98, 0x8a, 0x41, 0xe1, 0xaa, 0x18, 0x4f, 0xe9, 0x98, 0x44, 0x65, 0x4a, 0xb2, 0x71, 0x00, 0x0b, - 0xda, 0xb6, 0x6f, 0x25, 0xba, 0x8d, 0x67, 0x50, 0x4e, 0x8d, 0x95, 0x99, 0x3f, 0xa2, 0x67, 0x8e, - 0x6e, 0x12, 0x4a, 0x51, 0x15, 0xdb, 0x23, 0x7a, 0xd6, 0x53, 0x08, 0x39, 0x56, 0xc9, 0xae, 0xe4, - 0xcb, 0x4b, 0x49, 0xa4, 0x5c, 0x39, 0xc5, 0x55, 0x9b, 0x10, 0x34, 0x73, 0xe3, 0x05, 0xc0, 0x64, - 0x17, 0xb4, 0x06, 0x45, 0x79, 0x1d, 0x09, 0x63, 0x1d, 0xa8, 0xb4, 0x96, 0x17, 0x95, 0xc0, 0x9a, - 0x80, 0x7e, 0x06, 0x95, 0x88, 0x05, 0x81, 0xc3, 0xa9, 0x48, 0x82, 0x58, 0xa9, 0x5d, 0x7a, 0x73, - 0x7c, 0x7a, 0x2c, 0x08, 0xb0, 0xe2, 0xc6, 0x10, 0x65, 0xeb, 0xe6, 0x13, 0x80, 0x09, 0x05, 0x55, - 0xa0, 0xd4, 0x7d, 0xf2, 0x6c, 0x7b, 0xbf, 0xbb, 0x53, 0xbb, 0x81, 0x6c, 0x28, 0xe2, 0xdd, 0xed, - 0x9d, 0x5f, 0xd6, 0x2c, 0xb4, 0x08, 0xf6, 0x93, 0xc3, 0x63, 0x47, 0x83, 0x39, 0x54, 0x85, 0x72, - 0xe7, 0xf0, 0x70, 0xdf, 0x39, 0xdc, 0xdb, 0xab, 0xe5, 0xa5, 0x10, 0xde, 0x3d, 0x3a, 0xde, 0xc6, - 0xc7, 0xb5, 0x42, 0xf3, 0x1f, 0x16, 0xd4, 0x76, 0xe4, 0x8d, 0xfe, 0x5d, 0x28, 0xd5, 0x2d, 0x28, - 0xc8, 0x84, 0x34, 0x29, 0xb8, 0x92, 0x09, 0xcf, 0x1a, 0xa8, 0xd2, 0x17, 0x2b, 0xde, 0xc6, 0x03, - 0x28, 0x48, 0x08, 0xdd, 0x83, 0x25, 0xf1, 0x45, 0x20, 0x6f, 0xd9, 0xd3, 0xbe, 0x70, 0x12, 0xee, - 0x9b, 0x26, 0x5c, 0xd5, 0xd8, 0x67, 0x7d, 0xf1, 0x94, 0xfb, 0xcd, 0x7f, 0xe6, 0x61, 0x39, 0xd5, - 0xf6, 0x4d, 0x8a, 0xed, 0xc7, 0x33, 0xc5, 0xf6, 0xfe, 0x9c, 0xad, 0x57, 0xd6, 0x5a, 0x1b, 0xec, - 0x28, 0x39, 0x09, 0x7c, 0x31, 0x7c, 0x4d, 0xb1, 0xcd, 0x4b, 0xf7, 0x52, 0x5e, 0x3c, 0x11, 0x43, - 0x3f, 0x81, 0x52, 0x3f, 0x48, 0x94, 0x86, 0xc2, 0x4c, 0xb1, 0xcf, 0x6b, 0xd8, 0xd3, 0x9c, 0x38, - 0x15, 0xf9, 0xb6, 0x6b, 0x2c, 0x06, 0x3b, 0x33, 0x52, 0x3e, 0x6a, 0xc6, 0xee, 0x4b, 0xc7, 0x0b, - 0x98, 0x37, 0x32, 0x57, 0x6b, 0x79, 0xec, 0xbe, 0xec, 0x48, 0x78, 0xa6, 0x02, 0x73, 0x6f, 0x55, - 0x81, 0xf9, 0x2b, 0x2a, 0xf0, 0x3e, 0x94, 0x8c, 0x63, 0xff, 0xbd, 0xfc, 0x9a, 0xbf, 0xb1, 0xe0, - 0x9d, 0xc9, 0x30, 0xfa, 0x1d, 0x48, 0xf5, 0xe6, 0x57, 0x16, 0xdc, 0xbe, 0x64, 0xd1, 0x37, 0xc9, - 0xc6, 0xed, 0x49, 0x3a, 0x68, 0x63, 0x26, 0xe3, 0xc1, 0xeb, 0xf7, 0x98, 0xcf, 0x89, 0xff, 0x29, - 0x9c, 0x5f, 0x15, 0x60, 0xa9, 0xc3, 0xc6, 0x27, 0x7e, 0x98, 0x3d, 0x17, 0x37, 0x4d, 0xe9, 0x6a, - 0x99, 0xbb, 0x53, 0xf6, 0x4e, 0xb3, 0x4d, 0x15, 0x2e, 0xfa, 0x18, 0xf2, 0x2e, 0x49, 0x0d, 0x7e, - 0xef, 0x2a, 0x81, 0x6d, 0x42, 0xb0, 0xe4, 0x6b, 0xfc, 0x25, 0x67, 0x0a, 0xfd, 0x21, 0x94, 0x4f, - 0xfc, 0x90, 0xf8, 0xe1, 0x40, 0x5a, 0x98, 0xbf, 0x7c, 0x57, 0xcd, 0xef, 0xd6, 0x6a, 0x6b, 0x66, - 0x9c, 0x49, 0x35, 0x7e, 0x9d, 0x83, 0x92, 0xc1, 0x22, 0x04, 0x85, 0x7e, 0x12, 0xe8, 0xa3, 0x2f, - 0x63, 0xb5, 0x4e, 0x67, 0x1d, 0x39, 0xa5, 0xd9, 0x7a, 0xd6, 0xf9, 0x14, 0x2a, 0x11, 0x67, 0xcf, - 0xf5, 0x33, 0x28, 0x9d, 0xc1, 0x6a, 0x7a, 0x7e, 0xeb, 0x65, 0x04, 0x33, 0x86, 0x4e, 0xb3, 0xa2, - 0x9f, 0x42, 0x45, 0x78, 0x43, 0x3a, 0x76, 0x9d, 0xe7, 0x82, 0x85, 0xaa, 0x5a, 0xab, 0xed, 0xbb, - 0x17, 0xe7, 0xab, 0x75, 0x1a, 0x7a, 0x4c, 0x9a, 0xb0, 0x21, 0x09, 0x2d, 0xec, 0xbe, 0x38, 0xa0, - 0x42, 0x8d, 0x61, 0xa0, 0x05, 0x1e, 0x0b, 0x16, 0xa2, 0x16, 0x80, 0xa0, 0xdc, 0x89, 0x58, 0xe0, - 0x7b, 0x67, 0xea, 0xe9, 0x90, 0xcd, 0xcb, 0x47, 0x94, 0xf7, 0x14, 0x1a, 0xdb, 0x22, 0x5d, 0xaa, - 0xcf, 0x06, 0x6a, 0xbe, 0x8e, 0xb9, 0x7a, 0x1e, 0xd8, 0xb8, 0xa4, 0xc6, 0xe8, 0x98, 0xcb, 0x57, - 0xb8, 0x1a, 0xd1, 0xf4, 0xb4, 0x6f, 0x63, 0x03, 0x35, 0x42, 0xc8, 0x6f, 0x13, 0x82, 0xea, 0x50, - 0x32, 0x01, 0x32, 0x43, 0x5e, 0x0a, 0xa2, 0x1f, 0x41, 0x99, 0x30, 0x4f, 0xdb, 0x9f, 0x7b, 0x0b, - 0xfb, 0x4b, 0x84, 0x79, 0xca, 0xf8, 0x5b, 0x50, 0xec, 0x73, 0x16, 0xea, 0x91, 0xab, 0x8c, 0x35, - 0xd0, 0xfc, 0xab, 0x05, 0x37, 0xb3, 0x73, 0x32, 0xef, 0xbd, 0xab, 0x37, 0xaf, 0x43, 0x89, 0xd0, - 0x80, 0xc6, 0x26, 0xb5, 0xcb, 0x38, 0x05, 0x2f, 0x99, 0x95, 0xbf, 0x96, 0x59, 0x85, 0x29, 0xb3, - 0x66, 0x7a, 0x53, 0x71, 0xb6, 0x37, 0x7d, 0x00, 0x8b, 0x3a, 0x5e, 0x29, 0x87, 0x7a, 0x7c, 0xe1, - 0xaa, 0x46, 0x6a, 0xa6, 0xe6, 0x1d, 0x78, 0xa7, 0xc3, 0xc2, 0x90, 0x7a, 0x31, 0xe3, 0x3d, 0xce, - 0x5e, 0x9e, 0x99, 0x44, 0x6c, 0xfe, 0xce, 0x82, 0xdb, 0xb3, 0x14, 0xe3, 0xfa, 0x63, 0x28, 0xc9, - 0x27, 0x03, 0x15, 0xc2, 0x7c, 0x67, 0xd9, 0xbc, 0x38, 0x5f, 0x7d, 0xf0, 0x36, 0x6f, 0xab, 0xdd, - 0x90, 0xe8, 0x9e, 0x9c, 0x2a, 0x90, 0xa7, 0x1f, 0x49, 0xe5, 0x8e, 0x4f, 0xcc, 0x54, 0x5e, 0x52, - 0x70, 0x97, 0xa0, 0x06, 0xe4, 0x03, 0x36, 0x30, 0xf7, 0x4d, 0x39, 0xed, 0x70, 0x58, 0x22, 0x3f, - 0xba, 0x0f, 0x45, 0xf5, 0x35, 0x08, 0x01, 0x2c, 0xf4, 0x9e, 0xb6, 0xf7, 0xbb, 0x9d, 0xda, 0x0d, - 0x39, 0x25, 0xf4, 0x70, 0xf7, 0xd9, 0xf6, 0xf1, 0x6e, 0xcd, 0x92, 0xa3, 0xc5, 0xfe, 0x61, 0x67, - 0x7b, 0xbf, 0x96, 0xdb, 0x7a, 0x0c, 0x65, 0xf3, 0x5a, 0xe7, 0xe8, 0x73, 0x28, 0x99, 0x35, 0xba, - 0x93, 0xd5, 0xe0, 0xe5, 0xef, 0x48, 0x8d, 0xfa, 0x3c, 0x41, 0x7b, 0xbe, 0x69, 0x6d, 0xed, 0x43, - 0xd9, 0x64, 0x02, 0x47, 0x0f, 0xa1, 0x64, 0xd6, 0x53, 0xba, 0x2e, 0xd7, 0xf3, 0x94, 0xae, 0x99, - 0x04, 0x5a, 0xb7, 0x36, 0xad, 0xad, 0xa1, 0x6c, 0x4a, 0xd3, 0x31, 0x46, 0xcf, 0xe0, 0xa6, 0x5a, - 0x64, 0x68, 0x81, 0x56, 0xa6, 0x3b, 0xe9, 0xfc, 0x49, 0x35, 0x56, 0xaf, 0xa4, 0x4f, 0x76, 0x6a, - 0x7f, 0xfe, 0xea, 0x6f, 0x2b, 0x37, 0x5e, 0x7d, 0xbd, 0x62, 0xfd, 0xe9, 0xeb, 0x15, 0xeb, 0xf7, - 0x7f, 0x5f, 0xb1, 0x7e, 0xf5, 0xe0, 0xad, 0x1e, 0xe0, 0x46, 0xf5, 0xc9, 0x82, 0x42, 0x7d, 0xf2, - 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x92, 0xbf, 0x64, 0xd2, 0xf7, 0x14, 0x00, 0x00, +func (m *Join) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Join) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Join.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 *Join) XXX_Merge(src proto.Message) { + xxx_messageInfo_Join.Merge(m, src) +} +func (m *Join) XXX_Size() int { + return m.ProtoSize() +} +func (m *Join) XXX_DiscardUnknown() { + xxx_messageInfo_Join.DiscardUnknown(m) } -// Reference imports to suppress errors if they are not otherwise used. -var _ context.Context -var _ grpc.ClientConn - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the grpc package it is being compiled against. -const _ = grpc.SupportPackageIsVersion4 +var xxx_messageInfo_Join proto.InternalMessageInfo -// ShufflerClient is the client API for Shuffler service. +// Shard describes one shard and the reactor it is assigned to. // -// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. -type ShufflerClient interface { - Shuffle(ctx context.Context, in *ShuffleRequest, opts ...grpc.CallOption) (Shuffler_ShuffleClient, error) +// Shards are ordered on ascending `shard.range.key_begin` and, +// within a key range, on ascending `shard.range.r_clock_begin`. The +// tiling must be complete and non-overlapping in 2D over +// [key_begin, key_end] and [r_clock_begin, r_clock_end]. +type Join_Shard struct { + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // Labeling that captures all shard metadata. + Labeling *ops.ShardLabeling `protobuf:"bytes,2,opt,name=labeling,proto3" json:"labeling,omitempty"` + // Hosting reactor ID. + Reactor *protocol.ProcessSpec_ID `protobuf:"bytes,3,opt,name=reactor,proto3" json:"reactor,omitempty"` + // Creation revision of the Etcd key representing the shard's + // reactor assignment. Represents assignment identity: + // each re-assignment has a unique, monotonic revision. + EtcdCreateRevision int64 `protobuf:"varint,4,opt,name=etcd_create_revision,json=etcdCreateRevision,proto3" json:"etcd_create_revision,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -type shufflerClient struct { - cc *grpc.ClientConn +func (m *Join_Shard) Reset() { *m = Join_Shard{} } +func (m *Join_Shard) String() string { return proto.CompactTextString(m) } +func (*Join_Shard) ProtoMessage() {} +func (*Join_Shard) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{15, 0} } - -func NewShufflerClient(cc *grpc.ClientConn) ShufflerClient { - return &shufflerClient{cc} +func (m *Join_Shard) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) } - -func (c *shufflerClient) Shuffle(ctx context.Context, in *ShuffleRequest, opts ...grpc.CallOption) (Shuffler_ShuffleClient, error) { - stream, err := c.cc.NewStream(ctx, &_Shuffler_serviceDesc.Streams[0], "/runtime.Shuffler/Shuffle", opts...) - if err != nil { - return nil, err - } - x := &shufflerShuffleClient{stream} - if err := x.ClientStream.SendMsg(in); err != nil { - return nil, err - } - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err +func (m *Join_Shard) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Join_Shard.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return x, nil } - -type Shuffler_ShuffleClient interface { - Recv() (*ShuffleResponse, error) - grpc.ClientStream +func (m *Join_Shard) XXX_Merge(src proto.Message) { + xxx_messageInfo_Join_Shard.Merge(m, src) } - -type shufflerShuffleClient struct { - grpc.ClientStream +func (m *Join_Shard) XXX_Size() int { + return m.ProtoSize() } - -func (x *shufflerShuffleClient) Recv() (*ShuffleResponse, error) { - m := new(ShuffleResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil +func (m *Join_Shard) XXX_DiscardUnknown() { + xxx_messageInfo_Join_Shard.DiscardUnknown(m) } -// ShufflerServer is the server API for Shuffler service. -type ShufflerServer interface { - Shuffle(*ShuffleRequest, Shuffler_ShuffleServer) error -} +var xxx_messageInfo_Join_Shard proto.InternalMessageInfo -// UnimplementedShufflerServer can be embedded to have forward compatible implementations. -type UnimplementedShufflerServer struct { +// Joined is the leader's response to Join, fanned out to every joining shard. +// Each shard then forwards Joined to its controller. +type Joined struct { + // Zero: consensus reached (success); the session proceeds. + // Non-zero: topology disagreement. Await this revision, then retry. + MaxEtcdRevision int64 `protobuf:"varint,1,opt,name=max_etcd_revision,json=maxEtcdRevision,proto3" json:"max_etcd_revision,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (*UnimplementedShufflerServer) Shuffle(req *ShuffleRequest, srv Shuffler_ShuffleServer) error { - return status.Errorf(codes.Unimplemented, "method Shuffle not implemented") +func (m *Joined) Reset() { *m = Joined{} } +func (m *Joined) String() string { return proto.CompactTextString(m) } +func (*Joined) ProtoMessage() {} +func (*Joined) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{16} } - -func RegisterShufflerServer(s *grpc.Server, srv ShufflerServer) { - s.RegisterService(&_Shuffler_serviceDesc, srv) +func (m *Joined) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) } - -func _Shuffler_Shuffle_Handler(srv interface{}, stream grpc.ServerStream) error { - m := new(ShuffleRequest) - if err := stream.RecvMsg(m); err != nil { - return err +func (m *Joined) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Joined.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return srv.(ShufflerServer).Shuffle(m, &shufflerShuffleServer{stream}) } - -type Shuffler_ShuffleServer interface { - Send(*ShuffleResponse) error - grpc.ServerStream +func (m *Joined) XXX_Merge(src proto.Message) { + xxx_messageInfo_Joined.Merge(m, src) } - -type shufflerShuffleServer struct { - grpc.ServerStream +func (m *Joined) XXX_Size() int { + return m.ProtoSize() } - -func (x *shufflerShuffleServer) Send(m *ShuffleResponse) error { - return x.ServerStream.SendMsg(m) +func (m *Joined) XXX_DiscardUnknown() { + xxx_messageInfo_Joined.DiscardUnknown(m) +} + +var xxx_messageInfo_Joined proto.InternalMessageInfo + +// Task which is being processed by the runtime. +// Sent from Controller to Shard, and from Shard zero (only) to Leader +// after Joined. Other shards do not forward Task. +type Task struct { + // Task specification (protobuf-encoded bytes). + Spec []byte `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + // Collection journal partition to which task states are written. + OpsStatsJournal string `protobuf:"bytes,2,opt,name=ops_stats_journal,json=opsStatsJournal,proto3" json:"ops_stats_journal,omitempty"` + // Collection to which task stats are written. + OpsStatsSpec *flow.CollectionSpec `protobuf:"bytes,3,opt,name=ops_stats_spec,json=opsStatsSpec,proto3" json:"ops_stats_spec,omitempty"` + // When true, documents and stats are written to output and not directed to collections. + Preview bool `protobuf:"varint,4,opt,name=preview,proto3" json:"preview,omitempty"` + // Preview / harness control. Zero means unlimited. + MaxTransactions uint32 `protobuf:"varint,5,opt,name=max_transactions,json=maxTransactions,proto3" json:"max_transactions,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -var _Shuffler_serviceDesc = grpc.ServiceDesc{ - ServiceName: "runtime.Shuffler", - HandlerType: (*ShufflerServer)(nil), - Methods: []grpc.MethodDesc{}, - Streams: []grpc.StreamDesc{ - { - StreamName: "Shuffle", - Handler: _Shuffler_Shuffle_Handler, - ServerStreams: true, - }, - }, - Metadata: "go/protocols/runtime/runtime.proto", +func (m *Task) Reset() { *m = Task{} } +func (m *Task) String() string { return proto.CompactTextString(m) } +func (*Task) ProtoMessage() {} +func (*Task) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{17} } - -// CombinerClient is the client API for Combiner service. -// -// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. -type CombinerClient interface { - Combine(ctx context.Context, opts ...grpc.CallOption) (Combiner_CombineClient, error) +func (m *Task) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) } - -type combinerClient struct { - cc *grpc.ClientConn +func (m *Task) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Task.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 NewCombinerClient(cc *grpc.ClientConn) CombinerClient { - return &combinerClient{cc} +func (m *Task) XXX_Merge(src proto.Message) { + xxx_messageInfo_Task.Merge(m, src) } - -func (c *combinerClient) Combine(ctx context.Context, opts ...grpc.CallOption) (Combiner_CombineClient, error) { - stream, err := c.cc.NewStream(ctx, &_Combiner_serviceDesc.Streams[0], "/runtime.Combiner/Combine", opts...) - if err != nil { - return nil, err - } - x := &combinerCombineClient{stream} - return x, nil +func (m *Task) XXX_Size() int { + return m.ProtoSize() } - -type Combiner_CombineClient interface { - Send(*CombineRequest) error - Recv() (*CombineResponse, error) - grpc.ClientStream +func (m *Task) XXX_DiscardUnknown() { + xxx_messageInfo_Task.DiscardUnknown(m) } -type combinerCombineClient struct { - grpc.ClientStream -} +var xxx_messageInfo_Task proto.InternalMessageInfo -func (x *combinerCombineClient) Send(m *CombineRequest) error { - return x.ClientStream.SendMsg(m) +// Recover is sent by each shard to the leader after Joined, and carries +// state recovered from the shard's RocksDB. +// +// Shard zero populates Recover from its actual RocksDB scan. Non-zero +// shards have no recovery log: they MUST send `Recover{}` (proto default) +// or the leader fails the session. This is the migration guard: a non-zero +// shard reporting state is evidence of stale per-shard data left over +// from before consolidation, and we'd rather fail loudly than silently +// merge it. +// +// Hinted vs committed. `committed_*` is durable in the recovery log. +// `hinted_*` was *intended* to commit; it is durable on its last write +// but the corresponding transaction may or may not have completed. +// Remote-authoritative connectors may commit a transaction in the +// *endpoint* (during StartCommit/StartedCommit) ahead of the recovery +// log; the leader detects this at startup by comparing the close-clock +// embedded in `C:Opened`'s checkpoint against `hinted_close_clock`, +// and promotes the hinted Frontier to committed when they agree. +type Recover struct { + // Last-persisted ACK intents. + AckIntents map[string][]byte `protobuf:"bytes,1,rep,name=ack_intents,json=ackIntents,proto3" json:"ack_intents,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Clock at which the last-committed transaction closed, + // or zero if never committed. + CommittedCloseClock uint64 `protobuf:"fixed64,2,opt,name=committed_close_clock,json=committedCloseClock,proto3" json:"committed_close_clock,omitempty"` + // Committed Frontier entries. Not a delta. + CommittedFrontier *shuffle.Frontier `protobuf:"bytes,3,opt,name=committed_frontier,json=committedFrontier,proto3" json:"committed_frontier,omitempty"` + // Persisted connector state. + ConnectorStateJson []byte `protobuf:"bytes,4,opt,name=connector_state_json,json=connectorState,proto3" json:"connector_state_json,omitempty"` + // Clock at which the last hinted transaction closed, or zero if never + // hinted. If equal to `committed_close_clock` the hinted transaction + // committed normally; if ahead of `committed_close_clock` the hinted + // transaction was started but did not commit in the recovery log + // (and may or may not have committed in the connector's endpoint). + HintedCloseClock uint64 `protobuf:"fixed64,5,opt,name=hinted_close_clock,json=hintedCloseClock,proto3" json:"hinted_close_clock,omitempty"` + // Persisted hinted Frontier entries (FH: range). + HintedFrontier *shuffle.Frontier `protobuf:"bytes,6,opt,name=hinted_frontier,json=hintedFrontier,proto3" json:"hinted_frontier,omitempty"` + // Last-applied task specification (protobuf-encoded bytes), or empty. + LastApplied []byte `protobuf:"bytes,7,opt,name=last_applied,json=lastApplied,proto3" json:"last_applied,omitempty"` + // Legacy Checkpoint persisted in RocksDB for roll-forward / roll-back + // capability with the V1 runtime. This will be phased out as we fully + // cut over to the `committed_frontier` representation. + // + // Distinct from the checkpoint returned by C:Opened (see the + // `Recover` message comment above). The C:Opened checkpoint stays; + // only this `legacy_checkpoint` is on the path to removal. + LegacyCheckpoint *protocol1.Checkpoint `protobuf:"bytes,8,opt,name=legacy_checkpoint,json=legacyCheckpoint,proto3" json:"legacy_checkpoint,omitempty"` + // Per-binding max-key entries. + // Key: binding index; Value: packed composite key tuple. + MaxKeys map[uint32][]byte `protobuf:"bytes,9,rep,name=max_keys,json=maxKeys,proto3" json:"max_keys,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Persisted trigger parameters (materialize only), or empty. + TriggerParamsJson []byte `protobuf:"bytes,10,opt,name=trigger_params_json,json=triggerParams,proto3" json:"trigger_params_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (x *combinerCombineClient) Recv() (*CombineResponse, error) { - m := new(CombineResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err +func (m *Recover) Reset() { *m = Recover{} } +func (m *Recover) String() string { return proto.CompactTextString(m) } +func (*Recover) ProtoMessage() {} +func (*Recover) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{18} +} +func (m *Recover) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Recover) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Recover.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return m, nil } - -// CombinerServer is the server API for Combiner service. -type CombinerServer interface { - Combine(Combiner_CombineServer) error +func (m *Recover) XXX_Merge(src proto.Message) { + xxx_messageInfo_Recover.Merge(m, src) } - -// UnimplementedCombinerServer can be embedded to have forward compatible implementations. -type UnimplementedCombinerServer struct { +func (m *Recover) XXX_Size() int { + return m.ProtoSize() } - -func (*UnimplementedCombinerServer) Combine(srv Combiner_CombineServer) error { - return status.Errorf(codes.Unimplemented, "method Combine not implemented") +func (m *Recover) XXX_DiscardUnknown() { + xxx_messageInfo_Recover.DiscardUnknown(m) } -func RegisterCombinerServer(s *grpc.Server, srv CombinerServer) { - s.RegisterService(&_Combiner_serviceDesc, srv) -} +var xxx_messageInfo_Recover proto.InternalMessageInfo -func _Combiner_Combine_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(CombinerServer).Combine(&combinerCombineServer{stream}) +// Persist is sent by the leader to shard zero when state must be durably +// written. Each field maps to a contractual WriteBatch effect on shard +// zero's RocksDB. Absent fields are inert. +// +// All fields of a Persist land together in a single WriteBatch. +// `nonce` is echoed back by the shard's Persisted response, allowing +// the leader to match a Persisted response to its originating request. +type Persist struct { + // Request ID picked by the leader and echoed verbatim in the shard's + // `Persisted` response. The leader chooses any value and the shard + // does not interpret it. + Nonce uint64 `protobuf:"varint,1,opt,name=nonce,proto3" json:"nonce,omitempty"` + // Delete previously-persisted ACK intents. Applies ahead of `ack_intents`. + // Effect: DeleteRange("AI:") + DeleteAckIntents bool `protobuf:"varint,2,opt,name=delete_ack_intents,json=deleteAckIntents,proto3" json:"delete_ack_intents,omitempty"` + // ACK intent entries. + // Key: journal name; Value: raw journal content to write. + // Effect: Put under "AI:{journal}". + AckIntents map[string][]byte `protobuf:"bytes,3,rep,name=ack_intents,json=ackIntents,proto3" json:"ack_intents,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Clock at which the last-committed transaction closed. + // Effect: Put under "committed-close-clock". + CommittedCloseClock uint64 `protobuf:"fixed64,4,opt,name=committed_close_clock,json=committedCloseClock,proto3" json:"committed_close_clock,omitempty"` + // Committed Frontier entries. + // Effect: Put under "FC:..." keys. + CommittedFrontier *shuffle.Frontier `protobuf:"bytes,5,opt,name=committed_frontier,json=committedFrontier,proto3" json:"committed_frontier,omitempty"` + // Connector state patches. State Update Wire Format. + // Effect: Merge each patch under "connector-state". + ConnectorPatchesJson []byte `protobuf:"bytes,6,opt,name=connector_patches_json,json=connectorPatches,proto3" json:"connector_patches_json,omitempty"` + // Clock at which the hinted transaction closed. + // Effect: Put under "hinted-close-clock". + HintedCloseClock uint64 `protobuf:"fixed64,7,opt,name=hinted_close_clock,json=hintedCloseClock,proto3" json:"hinted_close_clock,omitempty"` + // Delete a previously-persisted hinted frontier. Applies ahead of `hinted_frontier`. + // Effect: DeleteRange("FH:") + DeleteHintedFrontier bool `protobuf:"varint,8,opt,name=delete_hinted_frontier,json=deleteHintedFrontier,proto3" json:"delete_hinted_frontier,omitempty"` + // Hinted Frontier entries. + // Effect: Put under "FH:" keys. + HintedFrontier *shuffle.Frontier `protobuf:"bytes,9,opt,name=hinted_frontier,json=hintedFrontier,proto3" json:"hinted_frontier,omitempty"` + // Last-applied task specification (protobuf-encoded bytes), or empty. + // Effect: Put under "last-applied" key. + LastApplied []byte `protobuf:"bytes,10,opt,name=last_applied,json=lastApplied,proto3" json:"last_applied,omitempty"` + // Legacy checkpoint, required for rollback to legacy runtime. + // Effect: Put under "checkpoint" key. + LegacyCheckpoint *protocol1.Checkpoint `protobuf:"bytes,11,opt,name=legacy_checkpoint,json=legacyCheckpoint,proto3" json:"legacy_checkpoint,omitempty"` + // Per-binding max-key updates, reduced to per-binding maximum across shards. + // Key: binding index; Value: packed composite key tuple. + // Effect: Put value under "MK-v2:{state_key}" (state_key resolved by the encoder). + MaxKeys map[uint32][]byte `protobuf:"bytes,12,rep,name=max_keys,json=maxKeys,proto3" json:"max_keys,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // Delete previously-persisted trigger parameters. Applies ahead of `trigger_params_json`. + // Effect: Delete the "trigger-params" key. + DeleteTriggerParams bool `protobuf:"varint,13,opt,name=delete_trigger_params,json=deleteTriggerParams,proto3" json:"delete_trigger_params,omitempty"` + // Materialization trigger parameters. + // Effect: Put under "trigger-params" key. + TriggerParamsJson []byte `protobuf:"bytes,14,opt,name=trigger_params_json,json=triggerParams,proto3" json:"trigger_params_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -type Combiner_CombineServer interface { - Send(*CombineResponse) error - Recv() (*CombineRequest, error) - grpc.ServerStream +func (m *Persist) Reset() { *m = Persist{} } +func (m *Persist) String() string { return proto.CompactTextString(m) } +func (*Persist) ProtoMessage() {} +func (*Persist) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{19} } - -type combinerCombineServer struct { - grpc.ServerStream +func (m *Persist) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Persist) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Persist.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 *Persist) XXX_Merge(src proto.Message) { + xxx_messageInfo_Persist.Merge(m, src) +} +func (m *Persist) XXX_Size() int { + return m.ProtoSize() +} +func (m *Persist) XXX_DiscardUnknown() { + xxx_messageInfo_Persist.DiscardUnknown(m) } -func (x *combinerCombineServer) Send(m *CombineResponse) error { - return x.ServerStream.SendMsg(m) +var xxx_messageInfo_Persist proto.InternalMessageInfo + +// Persisted is sent by shard zero to the leader after the state is durable +// in the recovery log. +type Persisted struct { + // Echoed back from the originating `Persist.nonce` request. + Nonce uint64 `protobuf:"varint,1,opt,name=nonce,proto3" json:"nonce,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (x *combinerCombineServer) Recv() (*CombineRequest, error) { - m := new(CombineRequest) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err +func (m *Persisted) Reset() { *m = Persisted{} } +func (m *Persisted) String() string { return proto.CompactTextString(m) } +func (*Persisted) ProtoMessage() {} +func (*Persisted) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{20} +} +func (m *Persisted) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Persisted) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Persisted.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return m, nil } - -var _Combiner_serviceDesc = grpc.ServiceDesc{ - ServiceName: "runtime.Combiner", - HandlerType: (*CombinerServer)(nil), - Methods: []grpc.MethodDesc{}, - Streams: []grpc.StreamDesc{ - { - StreamName: "Combine", - Handler: _Combiner_Combine_Handler, - ServerStreams: true, - ClientStreams: true, - }, - }, - Metadata: "go/protocols/runtime/runtime.proto", +func (m *Persisted) XXX_Merge(src proto.Message) { + xxx_messageInfo_Persisted.Merge(m, src) +} +func (m *Persisted) XXX_Size() int { + return m.ProtoSize() +} +func (m *Persisted) XXX_DiscardUnknown() { + xxx_messageInfo_Persisted.DiscardUnknown(m) } -// ConnectorProxyClient is the client API for ConnectorProxy service. +var xxx_messageInfo_Persisted proto.InternalMessageInfo + +// Apply asks shard zero to invoke its connector's Apply action, both for +// the initial application of a new spec and for re-application after a +// connector returned state patches that the leader has now persisted. // -// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. -type ConnectorProxyClient interface { - ProxyConnectors(ctx context.Context, opts ...grpc.CallOption) (ConnectorProxy_ProxyConnectorsClient, error) +// The leader's apply loop iterates Apply / Applied / (Persist / +// Persisted) until the connector returns no further patches; on the +// terminal iteration it Persists `last_applied = spec` and returns to +// the rest of session startup. +type Apply struct { + // Task specification to be applied (protobuf-encoded bytes). + Spec []byte `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + // Version of the specification being applied. + Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"` + // Last specification which was successfully applied, or empty. + LastSpec []byte `protobuf:"bytes,4,opt,name=last_spec,json=lastSpec,proto3" json:"last_spec,omitempty"` + // Version of the last applied specification, or empty. + LastVersion string `protobuf:"bytes,5,opt,name=last_version,json=lastVersion,proto3" json:"last_version,omitempty"` + // Reduced connector state observed by this Apply iteration. + ConnectorStateJson []byte `protobuf:"bytes,6,opt,name=connector_state_json,json=connectorStateJson,proto3" json:"connector_state_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -type connectorProxyClient struct { - cc *grpc.ClientConn +func (m *Apply) Reset() { *m = Apply{} } +func (m *Apply) String() string { return proto.CompactTextString(m) } +func (*Apply) ProtoMessage() {} +func (*Apply) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{21} } - -func NewConnectorProxyClient(cc *grpc.ClientConn) ConnectorProxyClient { - return &connectorProxyClient{cc} +func (m *Apply) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) } - -func (c *connectorProxyClient) ProxyConnectors(ctx context.Context, opts ...grpc.CallOption) (ConnectorProxy_ProxyConnectorsClient, error) { - stream, err := c.cc.NewStream(ctx, &_ConnectorProxy_serviceDesc.Streams[0], "/runtime.ConnectorProxy/ProxyConnectors", opts...) - if err != nil { - return nil, err +func (m *Apply) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Apply.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - x := &connectorProxyProxyConnectorsClient{stream} - return x, nil } - -type ConnectorProxy_ProxyConnectorsClient interface { - Send(*ConnectorProxyRequest) error - Recv() (*ConnectorProxyResponse, error) - grpc.ClientStream +func (m *Apply) XXX_Merge(src proto.Message) { + xxx_messageInfo_Apply.Merge(m, src) } - -type connectorProxyProxyConnectorsClient struct { - grpc.ClientStream +func (m *Apply) XXX_Size() int { + return m.ProtoSize() +} +func (m *Apply) XXX_DiscardUnknown() { + xxx_messageInfo_Apply.DiscardUnknown(m) } -func (x *connectorProxyProxyConnectorsClient) Send(m *ConnectorProxyRequest) error { - return x.ClientStream.SendMsg(m) +var xxx_messageInfo_Apply proto.InternalMessageInfo + +type Applied struct { + // Human-readable description of the action that the connector took. + // If empty, this Apply is to be considered a "no-op". + ActionDescription string `protobuf:"bytes,1,opt,name=action_description,json=actionDescription,proto3" json:"action_description,omitempty"` + // Applied connector state patches. State Update Wire Format. + ConnectorPatchesJson []byte `protobuf:"bytes,2,opt,name=connector_patches_json,json=connectorPatches,proto3" json:"connector_patches_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (x *connectorProxyProxyConnectorsClient) Recv() (*ConnectorProxyResponse, error) { - m := new(ConnectorProxyResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err +func (m *Applied) Reset() { *m = Applied{} } +func (m *Applied) String() string { return proto.CompactTextString(m) } +func (*Applied) ProtoMessage() {} +func (*Applied) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{22} +} +func (m *Applied) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Applied) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Applied.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return m, nil } - -// ConnectorProxyServer is the server API for ConnectorProxy service. -type ConnectorProxyServer interface { - ProxyConnectors(ConnectorProxy_ProxyConnectorsServer) error +func (m *Applied) XXX_Merge(src proto.Message) { + xxx_messageInfo_Applied.Merge(m, src) } - -// UnimplementedConnectorProxyServer can be embedded to have forward compatible implementations. -type UnimplementedConnectorProxyServer struct { +func (m *Applied) XXX_Size() int { + return m.ProtoSize() } - -func (*UnimplementedConnectorProxyServer) ProxyConnectors(srv ConnectorProxy_ProxyConnectorsServer) error { - return status.Errorf(codes.Unimplemented, "method ProxyConnectors not implemented") +func (m *Applied) XXX_DiscardUnknown() { + xxx_messageInfo_Applied.DiscardUnknown(m) } -func RegisterConnectorProxyServer(s *grpc.Server, srv ConnectorProxyServer) { - s.RegisterService(&_ConnectorProxy_serviceDesc, srv) +var xxx_messageInfo_Applied proto.InternalMessageInfo + +type Open struct { + // Task specification to be opened (protobuf-encoded bytes). + Spec []byte `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + // Version of the specification being opened. + Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"` + // Range of documents to be processed by this session. + Range *flow.RangeSpec `protobuf:"bytes,3,opt,name=range,proto3" json:"range,omitempty"` + // Last-persisted connector state, post-Apply, from `Recover`. + ConnectorStateJson []byte `protobuf:"bytes,4,opt,name=connector_state_json,json=state,proto3" json:"connector_state_json,omitempty"` + // Materializations only: per-binding maximum keys. + // Key: binding index; Value: packed composite key tuple. + MaxKeys map[uint32][]byte `protobuf:"bytes,5,rep,name=max_keys,json=maxKeys,proto3" json:"max_keys,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func _ConnectorProxy_ProxyConnectors_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(ConnectorProxyServer).ProxyConnectors(&connectorProxyProxyConnectorsServer{stream}) +func (m *Open) Reset() { *m = Open{} } +func (m *Open) String() string { return proto.CompactTextString(m) } +func (*Open) ProtoMessage() {} +func (*Open) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{23} +} +func (m *Open) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Open) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Open.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 *Open) XXX_Merge(src proto.Message) { + xxx_messageInfo_Open.Merge(m, src) +} +func (m *Open) XXX_Size() int { + return m.ProtoSize() +} +func (m *Open) XXX_DiscardUnknown() { + xxx_messageInfo_Open.DiscardUnknown(m) } -type ConnectorProxy_ProxyConnectorsServer interface { - Send(*ConnectorProxyResponse) error - Recv() (*ConnectorProxyRequest, error) - grpc.ServerStream +var xxx_messageInfo_Open proto.InternalMessageInfo + +// CloseNow is sent Controller → Shard → Leader, as a request to +// immediately close a transaction being held open by policy. +type CloseNow struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -type connectorProxyProxyConnectorsServer struct { - grpc.ServerStream +func (m *CloseNow) Reset() { *m = CloseNow{} } +func (m *CloseNow) String() string { return proto.CompactTextString(m) } +func (*CloseNow) ProtoMessage() {} +func (*CloseNow) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{24} +} +func (m *CloseNow) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CloseNow) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_CloseNow.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 *CloseNow) XXX_Merge(src proto.Message) { + xxx_messageInfo_CloseNow.Merge(m, src) +} +func (m *CloseNow) XXX_Size() int { + return m.ProtoSize() +} +func (m *CloseNow) XXX_DiscardUnknown() { + xxx_messageInfo_CloseNow.DiscardUnknown(m) } -func (x *connectorProxyProxyConnectorsServer) Send(m *ConnectorProxyResponse) error { - return x.ServerStream.SendMsg(m) +var xxx_messageInfo_CloseNow proto.InternalMessageInfo + +// Stop is sent Controller → Shard → Leader to request graceful shutdown. +type Stop struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (x *connectorProxyProxyConnectorsServer) Recv() (*ConnectorProxyRequest, error) { - m := new(ConnectorProxyRequest) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err +func (m *Stop) Reset() { *m = Stop{} } +func (m *Stop) String() string { return proto.CompactTextString(m) } +func (*Stop) ProtoMessage() {} +func (*Stop) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{25} +} +func (m *Stop) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Stop) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Stop.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return m, nil +} +func (m *Stop) XXX_Merge(src proto.Message) { + xxx_messageInfo_Stop.Merge(m, src) +} +func (m *Stop) XXX_Size() int { + return m.ProtoSize() +} +func (m *Stop) XXX_DiscardUnknown() { + xxx_messageInfo_Stop.DiscardUnknown(m) } -var _ConnectorProxy_serviceDesc = grpc.ServiceDesc{ - ServiceName: "runtime.ConnectorProxy", - HandlerType: (*ConnectorProxyServer)(nil), - Methods: []grpc.MethodDesc{}, - Streams: []grpc.StreamDesc{ - { - StreamName: "ProxyConnectors", - Handler: _ConnectorProxy_ProxyConnectors_Handler, - ServerStreams: true, - ClientStreams: true, - }, - }, - Metadata: "go/protocols/runtime/runtime.proto", +var xxx_messageInfo_Stop proto.InternalMessageInfo + +// Stopped confirms the session has shut down. The leader sends Stopped +// to each shard followed by EOF; each shard then forwards Stopped to its +// controller and EOFs. +type Stopped struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TaskServiceConfig) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err +func (m *Stopped) Reset() { *m = Stopped{} } +func (m *Stopped) String() string { return proto.CompactTextString(m) } +func (*Stopped) ProtoMessage() {} +func (*Stopped) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{26} +} +func (m *Stopped) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Stopped) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Stopped.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return dAtA[:n], nil +} +func (m *Stopped) XXX_Merge(src proto.Message) { + xxx_messageInfo_Stopped.Merge(m, src) +} +func (m *Stopped) XXX_Size() int { + return m.ProtoSize() +} +func (m *Stopped) XXX_DiscardUnknown() { + xxx_messageInfo_Stopped.DiscardUnknown(m) } -func (m *TaskServiceConfig) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +var xxx_messageInfo_Stopped proto.InternalMessageInfo + +// SessionLoop is sent as the first message of a session-loop stream on +// the Shard service. It carries process-level configuration that +// outlives the cycle of leader sessions on this stream. The Leader +// service never sees SessionLoop. +type SessionLoop struct { + // RocksDB context opened for this Shard stream and reused across all + // of its leader sessions. Absent for non-zero materialize/derive + // shards, which don't host a recovery log. + RocksdbDescriptor *RocksDBDescriptor `protobuf:"bytes,1,opt,name=rocksdb_descriptor,json=rocksdbDescriptor,proto3" json:"rocksdb_descriptor,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TaskServiceConfig) 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.Plane != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.Plane)) - i-- - dAtA[i] = 0x30 - } - if len(m.ContainerNetwork) > 0 { - i -= len(m.ContainerNetwork) - copy(dAtA[i:], m.ContainerNetwork) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.ContainerNetwork))) - i-- - dAtA[i] = 0x22 - } - if len(m.UdsPath) > 0 { - i -= len(m.UdsPath) - copy(dAtA[i:], m.UdsPath) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.UdsPath))) - i-- - dAtA[i] = 0x1a - } - if len(m.TaskName) > 0 { - i -= len(m.TaskName) - copy(dAtA[i:], m.TaskName) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.TaskName))) - i-- - dAtA[i] = 0x12 - } - if m.LogFileFd != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.LogFileFd)) - i-- - dAtA[i] = 0x8 +func (m *SessionLoop) Reset() { *m = SessionLoop{} } +func (m *SessionLoop) String() string { return proto.CompactTextString(m) } +func (*SessionLoop) ProtoMessage() {} +func (*SessionLoop) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{27} +} +func (m *SessionLoop) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SessionLoop) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SessionLoop.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return len(dAtA) - i, nil +} +func (m *SessionLoop) XXX_Merge(src proto.Message) { + xxx_messageInfo_SessionLoop.Merge(m, src) +} +func (m *SessionLoop) XXX_Size() int { + return m.ProtoSize() +} +func (m *SessionLoop) XXX_DiscardUnknown() { + xxx_messageInfo_SessionLoop.DiscardUnknown(m) +} + +var xxx_messageInfo_SessionLoop proto.InternalMessageInfo + +// Materialize is the bidirectional message type for materialization +// sessions. Exactly one field is set per message. +type Materialize struct { + // Controller → Shard. Unary request outside of a SessionLoop. + Spec *materialize.Request_Spec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + // Shard → Controller. Connector's reply to `spec`. + SpecResponse *materialize.Response_Spec `protobuf:"bytes,2,opt,name=spec_response,json=specResponse,proto3" json:"spec_response,omitempty"` + // Controller → Shard. Unary request outside of a SessionLoop. + Validate *materialize.Request_Validate `protobuf:"bytes,3,opt,name=validate,proto3" json:"validate,omitempty"` + // Shard → Controller. Connector's reply to `validate`. + Validated *materialize.Response_Validated `protobuf:"bytes,4,opt,name=validated,proto3" json:"validated,omitempty"` + // Controller → Shard. First message of a session-loop stream; + // never sent to the Leader. + SessionLoop *SessionLoop `protobuf:"bytes,20,opt,name=session_loop,json=sessionLoop,proto3" json:"session_loop,omitempty"` + // Controller → Shard → Leader. Session initiation with topology. + Join *Join `protobuf:"bytes,21,opt,name=join,proto3" json:"join,omitempty"` + // Leader → Shards → Controllers. Consensus or retry directive + // (`max_etcd_revision == 0` means consensus; non-zero means retry). + Joined *Joined `protobuf:"bytes,22,opt,name=joined,proto3" json:"joined,omitempty"` + // Controller → Shard, and Shard zero (only) → Leader. + // Defines the task being processed by the now-joined topology. + Task *Task `protobuf:"bytes,23,opt,name=task,proto3" json:"task,omitempty"` + // Each Shard → Leader. State recovered from RocksDB on startup. + // Non-zero shards MUST send `Recover{}` (default) — see Recover. + Recover *Recover `protobuf:"bytes,24,opt,name=recover,proto3" json:"recover,omitempty"` + // Leader → Shard zero. Re-applied until `Applied` returns no patches. + Apply *Apply `protobuf:"bytes,25,opt,name=apply,proto3" json:"apply,omitempty"` + // Shard zero → Leader. C:Apply complete. If `Applied` emits state + // patches the leader runs Persist/Persisted and calls Apply again, + // until it emits no further patches. + Applied *Applied `protobuf:"bytes,26,opt,name=applied,proto3" json:"applied,omitempty"` + // Leader → Shards. Open connector and prepare for transactions. + Open *Open `protobuf:"bytes,27,opt,name=open,proto3" json:"open,omitempty"` + Opened *Materialize_Opened `protobuf:"bytes,28,opt,name=opened,proto3" json:"opened,omitempty"` + Load *Materialize_Load `protobuf:"bytes,40,opt,name=load,proto3" json:"load,omitempty"` + Loaded *Materialize_Loaded `protobuf:"bytes,41,opt,name=loaded,proto3" json:"loaded,omitempty"` + Flush *Materialize_Flush `protobuf:"bytes,42,opt,name=flush,proto3" json:"flush,omitempty"` + Flushed *Materialize_Flushed `protobuf:"bytes,43,opt,name=flushed,proto3" json:"flushed,omitempty"` + Store *Materialize_Store `protobuf:"bytes,44,opt,name=store,proto3" json:"store,omitempty"` + Stored *Materialize_Stored `protobuf:"bytes,45,opt,name=stored,proto3" json:"stored,omitempty"` + StartCommit *Materialize_StartCommit `protobuf:"bytes,46,opt,name=start_commit,json=startCommit,proto3" json:"start_commit,omitempty"` + StartedCommit *Materialize_StartedCommit `protobuf:"bytes,47,opt,name=started_commit,json=startedCommit,proto3" json:"started_commit,omitempty"` + Acknowledge *Materialize_Acknowledge `protobuf:"bytes,48,opt,name=acknowledge,proto3" json:"acknowledge,omitempty"` + Acknowledged *Materialize_Acknowledged `protobuf:"bytes,49,opt,name=acknowledged,proto3" json:"acknowledged,omitempty"` + // Leader → Shard zero. Durably persist state. + Persist *Persist `protobuf:"bytes,50,opt,name=persist,proto3" json:"persist,omitempty"` + // Shard zero → Leader. State is durable. + Persisted *Persisted `protobuf:"bytes,51,opt,name=persisted,proto3" json:"persisted,omitempty"` + // Controller → Shard → Leader. Request immediate close of the + // currently-open transaction. + CloseNow *CloseNow `protobuf:"bytes,52,opt,name=close_now,json=closeNow,proto3" json:"close_now,omitempty"` + // Controller → Shard → Leader. Graceful shutdown request. + Stop *Stop `protobuf:"bytes,60,opt,name=stop,proto3" json:"stop,omitempty"` + // Leader → Shards → Controllers. Shutdown confirmed; EOF follows on + // each leg. + Stopped *Stopped `protobuf:"bytes,61,opt,name=stopped,proto3" json:"stopped,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ShuffleRequest) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err +func (m *Materialize) Reset() { *m = Materialize{} } +func (m *Materialize) String() string { return proto.CompactTextString(m) } +func (*Materialize) ProtoMessage() {} +func (*Materialize) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28} +} +func (m *Materialize) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return dAtA[:n], nil +} +func (m *Materialize) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize.Merge(m, src) +} +func (m *Materialize) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize.DiscardUnknown(m) } -func (m *ShuffleRequest) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +var xxx_messageInfo_Materialize proto.InternalMessageInfo + +// Each Shard → Leader, and each Shard → Controller. Connector is +// running and session startup is complete. +// +// The shard sends *two* Opened messages per session — one to each +// peer — populated differently: +// - To the leader: `container` is empty; `connector_checkpoint` is +// the C:Opened checkpoint (if any). Non-zero shards MUST send +// `Opened{}` (default) here, mirroring the Recover migration +// guard: a non-empty checkpoint from a non-zero shard is treated +// as evidence of stale per-shard state, and startup fails. +// - To the controller: `container` describes the running connector +// container; `connector_checkpoint` is empty. +type Materialize_Opened struct { + // Description of the running connector container. + Container *Container `protobuf:"bytes,1,opt,name=container,proto3" json:"container,omitempty"` + // Optional connector consumer.Checkpoint returned by C:Opened. + // + // Remote-authoritative connectors commit to their endpoint during + // StartCommit, and the leader compares the close-clock embedded in + // this checkpoint against the recovered `hinted_close_clock` to + // detect a commit that landed in the endpoint but not in the + // recovery log. See `Recover` for the full hinted/committed model. + // + // Migration: if no close-clock is embedded the checkpoint is + // assumed to be a V1 legacy checkpoint and replaces the recovered + // committed Frontier. The V2 runtime may write additional checkpoint + // keys for rollback compatibility with V1. + ConnectorCheckpoint *protocol1.Checkpoint `protobuf:"bytes,2,opt,name=connector_checkpoint,json=connectorCheckpoint,proto3" json:"connector_checkpoint,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ShuffleRequest) 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.Materialization != nil { - { - size, err := m.Materialization.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x5a - } - if m.Derivation != nil { - { - size, err := m.Derivation.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x52 - } - if m.ShuffleIndex != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.ShuffleIndex)) - i-- - dAtA[i] = 0x48 - } - if m.Resolution != nil { - { - size, err := m.Resolution.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) +func (m *Materialize_Opened) Reset() { *m = Materialize_Opened{} } +func (m *Materialize_Opened) String() string { return proto.CompactTextString(m) } +func (*Materialize_Opened) ProtoMessage() {} +func (*Materialize_Opened) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 0} +} +func (m *Materialize_Opened) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Opened) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Opened.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err } - i-- - dAtA[i] = 0x42 - } - if len(m.Coordinator) > 0 { - i -= len(m.Coordinator) - copy(dAtA[i:], m.Coordinator) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.Coordinator))) - i-- - dAtA[i] = 0x3a + return b[:n], nil } - { - size, err := m.Range.MarshalToSizedBuffer(dAtA[:i]) +} +func (m *Materialize_Opened) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Opened.Merge(m, src) +} +func (m *Materialize_Opened) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Opened) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Opened.DiscardUnknown(m) +} + +var xxx_messageInfo_Materialize_Opened proto.InternalMessageInfo + +// Leader → Shards. Incremental Frontier to process into transaction. +// Shards add source docs to their combiner and issue C:Load RPCs for +// documents that may already exist in the endpoint. +// Multiple Load messages may be sent per transaction. +type Materialize_Load struct { + Frontier *shuffle.Frontier `protobuf:"bytes,1,opt,name=frontier,proto3" json:"frontier,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Materialize_Load) Reset() { *m = Materialize_Load{} } +func (m *Materialize_Load) String() string { return proto.CompactTextString(m) } +func (*Materialize_Load) ProtoMessage() {} +func (*Materialize_Load) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 1} +} +func (m *Materialize_Load) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Load) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Load.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) if err != nil { - return 0, err + return nil, err } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x32 - if m.EndOffset != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.EndOffset)) - i-- - dAtA[i] = 0x28 - } - if m.Offset != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.Offset)) - i-- - dAtA[i] = 0x20 - } - if len(m.BuildId) > 0 { - i -= len(m.BuildId) - copy(dAtA[i:], m.BuildId) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.BuildId))) - i-- - dAtA[i] = 0x1a + return b[:n], nil } - if m.Replay { - i-- - if m.Replay { - dAtA[i] = 1 - } else { - dAtA[i] = 0 +} +func (m *Materialize_Load) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Load.Merge(m, src) +} +func (m *Materialize_Load) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Load) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Load.DiscardUnknown(m) +} + +var xxx_messageInfo_Materialize_Load proto.InternalMessageInfo + +// Shard → Leader. All frontier documents have been loaded into the +// combiner and all C:Load requests sent into the connector. +type Materialize_Loaded struct { + Bindings []*Materialize_Loaded_Binding `protobuf:"bytes,1,rep,name=bindings,proto3" json:"bindings,omitempty"` + // On-disk size of this shard's combiner, for transaction close policy. + CombinerUsageBytes uint64 `protobuf:"varint,2,opt,name=combiner_usage_bytes,json=combinerUsageBytes,proto3" json:"combiner_usage_bytes,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Materialize_Loaded) Reset() { *m = Materialize_Loaded{} } +func (m *Materialize_Loaded) String() string { return proto.CompactTextString(m) } +func (*Materialize_Loaded) ProtoMessage() {} +func (*Materialize_Loaded) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 2} +} +func (m *Materialize_Loaded) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Loaded) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Loaded.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err } - i-- - dAtA[i] = 0x10 - } - if len(m.Journal) > 0 { - i -= len(m.Journal) - copy(dAtA[i:], m.Journal) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.Journal))) - i-- - dAtA[i] = 0xa + return b[:n], nil } - return len(dAtA) - i, nil +} +func (m *Materialize_Loaded) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Loaded.Merge(m, src) +} +func (m *Materialize_Loaded) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Loaded) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Loaded.DiscardUnknown(m) +} + +var xxx_messageInfo_Materialize_Loaded proto.InternalMessageInfo + +// Binding which participated in this Loaded frontier. +type Materialize_Loaded_Binding struct { + // Binding index. + Index uint32 `protobuf:"varint,1,opt,name=index,proto3" json:"index,omitempty"` + // Minimum source document Clock observed + MinSourceClock uint64 `protobuf:"fixed64,2,opt,name=min_source_clock,json=minSourceClock,proto3" json:"min_source_clock,omitempty"` + // Maximum source document Clock observed + MaxSourceClock uint64 `protobuf:"fixed64,3,opt,name=max_source_clock,json=maxSourceClock,proto3" json:"max_source_clock,omitempty"` + // Number of source documents processed. + SourcedDocsTotal uint64 `protobuf:"varint,4,opt,name=sourced_docs_total,json=sourcedDocsTotal,proto3" json:"sourced_docs_total,omitempty"` + // Number of source document bytes processed. + SourcedBytesTotal uint64 `protobuf:"varint,5,opt,name=sourced_bytes_total,json=sourcedBytesTotal,proto3" json:"sourced_bytes_total,omitempty"` + // When non-empty, a new maximum key was observed. + MaxKeyDelta []byte `protobuf:"bytes,6,opt,name=max_key_delta,json=maxKeyDelta,proto3" json:"max_key_delta,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ShuffleResponse) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err +func (m *Materialize_Loaded_Binding) Reset() { *m = Materialize_Loaded_Binding{} } +func (m *Materialize_Loaded_Binding) String() string { return proto.CompactTextString(m) } +func (*Materialize_Loaded_Binding) ProtoMessage() {} +func (*Materialize_Loaded_Binding) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 2, 0} +} +func (m *Materialize_Loaded_Binding) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Loaded_Binding) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Loaded_Binding.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return dAtA[:n], nil } - -func (m *ShuffleResponse) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +func (m *Materialize_Loaded_Binding) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Loaded_Binding.Merge(m, src) +} +func (m *Materialize_Loaded_Binding) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Loaded_Binding) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Loaded_Binding.DiscardUnknown(m) } -func (m *ShuffleResponse) 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 len(m.PackedKey) > 0 { - for iNdEx := len(m.PackedKey) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.PackedKey[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x52 - } - } - if len(m.UuidParts) > 0 { - for iNdEx := len(m.UuidParts) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.UuidParts[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x4a - } - } - if len(m.Offsets) > 0 { - dAtA6 := make([]byte, len(m.Offsets)*10) - var j5 int - for _, num1 := range m.Offsets { - num := uint64(num1) - for num >= 1<<7 { - dAtA6[j5] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j5++ - } - dAtA6[j5] = uint8(num) - j5++ - } - i -= j5 - copy(dAtA[i:], dAtA6[:j5]) - i = encodeVarintRuntime(dAtA, i, uint64(j5)) - i-- - dAtA[i] = 0x42 - } - if len(m.Docs) > 0 { - for iNdEx := len(m.Docs) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Docs[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x3a +var xxx_messageInfo_Materialize_Loaded_Binding proto.InternalMessageInfo + +// Leader → Shards. Signals end of Load phase. +type Materialize_Flush struct { + // Prior transaction's aggregated C:Acknowledged state patches. + // State Update Wire Format. + ConnectorPatchesJson []byte `protobuf:"bytes,1,opt,name=connector_patches_json,json=connectorPatches,proto3" json:"connector_patches_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Materialize_Flush) Reset() { *m = Materialize_Flush{} } +func (m *Materialize_Flush) String() string { return proto.CompactTextString(m) } +func (*Materialize_Flush) ProtoMessage() {} +func (*Materialize_Flush) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 3} +} +func (m *Materialize_Flush) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Flush) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Flush.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err } + return b[:n], nil } - if len(m.Arena) > 0 { - i -= len(m.Arena) - copy(dAtA[i:], m.Arena) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.Arena))) - i-- - dAtA[i] = 0x32 - } - if m.WriteHead != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.WriteHead)) - i-- - dAtA[i] = 0x28 - } - if m.ReadThrough != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.ReadThrough)) - i-- - dAtA[i] = 0x20 - } - if len(m.TerminalError) > 0 { - i -= len(m.TerminalError) - copy(dAtA[i:], m.TerminalError) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.TerminalError))) - i-- - dAtA[i] = 0x1a - } - if m.Header != nil { - { - size, err := m.Header.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) +} +func (m *Materialize_Flush) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Flush.Merge(m, src) +} +func (m *Materialize_Flush) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Flush) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Flush.DiscardUnknown(m) +} + +var xxx_messageInfo_Materialize_Flush proto.InternalMessageInfo + +// Shard → Leader. Flush phase complete. +// Reports connector state patches and max-key deltas from C:Flushed. +type Materialize_Flushed struct { + Bindings []*Materialize_Flushed_Binding `protobuf:"bytes,1,rep,name=bindings,proto3" json:"bindings,omitempty"` + // Connector state patches from this shard's C:Flushed. + // State Update Wire Format. + ConnectorPatchesJson []byte `protobuf:"bytes,2,opt,name=connector_patches_json,json=connectorPatches,proto3" json:"connector_patches_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Materialize_Flushed) Reset() { *m = Materialize_Flushed{} } +func (m *Materialize_Flushed) String() string { return proto.CompactTextString(m) } +func (*Materialize_Flushed) ProtoMessage() {} +func (*Materialize_Flushed) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 4} +} +func (m *Materialize_Flushed) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Flushed) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Flushed.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err } - i-- - dAtA[i] = 0x12 - } - if m.Status != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.Status)) - i-- - dAtA[i] = 0x8 + return b[:n], nil } - return len(dAtA) - i, nil +} +func (m *Materialize_Flushed) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Flushed.Merge(m, src) +} +func (m *Materialize_Flushed) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Flushed) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Flushed.DiscardUnknown(m) } -func (m *RocksDBDescriptor) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err +var xxx_messageInfo_Materialize_Flushed proto.InternalMessageInfo + +// Binding which participated in this Flushed phase. +type Materialize_Flushed_Binding struct { + // Binding index. + Index uint32 `protobuf:"varint,1,opt,name=index,proto3" json:"index,omitempty"` + // Number of endpoint documents loaded by this binding + // between the last L:Loaded and C:Flushed. + LoadedDocsTotal uint64 `protobuf:"varint,2,opt,name=loaded_docs_total,json=loadedDocsTotal,proto3" json:"loaded_docs_total,omitempty"` + // Number of endpoint document bytes loaded by this binding + // between the last L:Loaded and C:Flushed. + LoadedBytesTotal uint64 `protobuf:"varint,3,opt,name=loaded_bytes_total,json=loadedBytesTotal,proto3" json:"loaded_bytes_total,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Materialize_Flushed_Binding) Reset() { *m = Materialize_Flushed_Binding{} } +func (m *Materialize_Flushed_Binding) String() string { return proto.CompactTextString(m) } +func (*Materialize_Flushed_Binding) ProtoMessage() {} +func (*Materialize_Flushed_Binding) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 4, 0} +} +func (m *Materialize_Flushed_Binding) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Flushed_Binding) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Flushed_Binding.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return dAtA[:n], nil +} +func (m *Materialize_Flushed_Binding) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Flushed_Binding.Merge(m, src) +} +func (m *Materialize_Flushed_Binding) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Flushed_Binding) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Flushed_Binding.DiscardUnknown(m) } -func (m *RocksDBDescriptor) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +var xxx_messageInfo_Materialize_Flushed_Binding proto.InternalMessageInfo + +// Leader → Shards. Idempotency Persist now complete; drain combiners +// into C:Store requests. +type Materialize_Store struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RocksDBDescriptor) 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 len(m.RocksdbPath) > 0 { - i -= len(m.RocksdbPath) - copy(dAtA[i:], m.RocksdbPath) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.RocksdbPath))) - i-- - dAtA[i] = 0x12 - } - if m.RocksdbEnvMemptr != 0 { - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.RocksdbEnvMemptr)) - i-- - dAtA[i] = 0x9 +func (m *Materialize_Store) Reset() { *m = Materialize_Store{} } +func (m *Materialize_Store) String() string { return proto.CompactTextString(m) } +func (*Materialize_Store) ProtoMessage() {} +func (*Materialize_Store) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 5} +} +func (m *Materialize_Store) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Store) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Store.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return len(dAtA) - i, nil +} +func (m *Materialize_Store) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Store.Merge(m, src) +} +func (m *Materialize_Store) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Store) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Store.DiscardUnknown(m) } -func (m *Container) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err +var xxx_messageInfo_Materialize_Store proto.InternalMessageInfo + +// Shard → Leader. Store phase complete; shard is ready to commit. +// Reports final per-binding transaction statistics. +type Materialize_Stored struct { + Bindings []*Materialize_Stored_Binding `protobuf:"bytes,1,rep,name=bindings,proto3" json:"bindings,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Materialize_Stored) Reset() { *m = Materialize_Stored{} } +func (m *Materialize_Stored) String() string { return proto.CompactTextString(m) } +func (*Materialize_Stored) ProtoMessage() {} +func (*Materialize_Stored) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 6} +} +func (m *Materialize_Stored) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Stored) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Stored.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return dAtA[:n], nil +} +func (m *Materialize_Stored) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Stored.Merge(m, src) +} +func (m *Materialize_Stored) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Stored) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Stored.DiscardUnknown(m) } -func (m *Container) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +var xxx_messageInfo_Materialize_Stored proto.InternalMessageInfo + +// Binding which participated in this Stored phase. +type Materialize_Stored_Binding struct { + // Binding index. + Index uint32 `protobuf:"varint,1,opt,name=index,proto3" json:"index,omitempty"` + // Number of endpoint documents stored by this binding. + StoredDocsTotal uint64 `protobuf:"varint,2,opt,name=stored_docs_total,json=storedDocsTotal,proto3" json:"stored_docs_total,omitempty"` + // Number of endpoint document bytes stored by this binding. + StoredBytesTotal uint64 `protobuf:"varint,3,opt,name=stored_bytes_total,json=storedBytesTotal,proto3" json:"stored_bytes_total,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Container) 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.UsageRate != 0 { - i -= 4 - encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.UsageRate)))) - i-- - dAtA[i] = 0x25 - } - if len(m.MappedHostPorts) > 0 { - for k := range m.MappedHostPorts { - v := m.MappedHostPorts[k] - baseI := i - i -= len(v) - copy(dAtA[i:], v) - i = encodeVarintRuntime(dAtA, i, uint64(len(v))) - i-- - dAtA[i] = 0x12 - i = encodeVarintRuntime(dAtA, i, uint64(k)) - i-- - dAtA[i] = 0x8 - i = encodeVarintRuntime(dAtA, i, uint64(baseI-i)) - i-- - dAtA[i] = 0x1a - } - } - if len(m.NetworkPorts) > 0 { - for iNdEx := len(m.NetworkPorts) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.NetworkPorts[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 +func (m *Materialize_Stored_Binding) Reset() { *m = Materialize_Stored_Binding{} } +func (m *Materialize_Stored_Binding) String() string { return proto.CompactTextString(m) } +func (*Materialize_Stored_Binding) ProtoMessage() {} +func (*Materialize_Stored_Binding) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 6, 0} +} +func (m *Materialize_Stored_Binding) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Stored_Binding) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Stored_Binding.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err } + return b[:n], nil } - if len(m.IpAddr) > 0 { - i -= len(m.IpAddr) - copy(dAtA[i:], m.IpAddr) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.IpAddr))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil } - -func (m *CaptureRequestExt) 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 *Materialize_Stored_Binding) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Stored_Binding.Merge(m, src) +} +func (m *Materialize_Stored_Binding) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Stored_Binding) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Stored_Binding.DiscardUnknown(m) } -func (m *CaptureRequestExt) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +var xxx_messageInfo_Materialize_Stored_Binding proto.InternalMessageInfo + +// Leader → Shards. Begin commit phase. +// Distributes aggregated L:Flushed patches from all shards so each connector +// instance can observe its peers' state for cooperative strategies (such as +// parallel file staging). +type Materialize_StartCommit struct { + // Aggregated Flushed connector state patches from all shards. + // State Update Wire Format. + ConnectorPatchesJson []byte `protobuf:"bytes,1,opt,name=connector_patches_json,json=connectorPatches,proto3" json:"connector_patches_json,omitempty"` + // Transaction Checkpoint for remote-authoritative connectors. + ConnectorCheckpoint *protocol1.Checkpoint `protobuf:"bytes,2,opt,name=connector_checkpoint,json=connectorCheckpoint,proto3" json:"connector_checkpoint,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *CaptureRequestExt) 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.StartCommit != nil { - { - size, err := m.StartCommit.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } - if m.RocksdbDescriptor != nil { - { - size, err := m.RocksdbDescriptor.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) +func (m *Materialize_StartCommit) Reset() { *m = Materialize_StartCommit{} } +func (m *Materialize_StartCommit) String() string { return proto.CompactTextString(m) } +func (*Materialize_StartCommit) ProtoMessage() {} +func (*Materialize_StartCommit) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 7} +} +func (m *Materialize_StartCommit) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_StartCommit) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_StartCommit.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err } - i-- - dAtA[i] = 0x12 - } - if m.LogLevel != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.LogLevel)) - i-- - dAtA[i] = 0x8 + return b[:n], nil } - return len(dAtA) - i, nil } - -func (m *CaptureRequestExt_StartCommit) 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 *Materialize_StartCommit) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_StartCommit.Merge(m, src) +} +func (m *Materialize_StartCommit) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_StartCommit) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_StartCommit.DiscardUnknown(m) } -func (m *CaptureRequestExt_StartCommit) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +var xxx_messageInfo_Materialize_StartCommit proto.InternalMessageInfo + +// Shard → Leader. Commit initiated. +// Remote-authoritative connectors will literally commit prior to return. +type Materialize_StartedCommit struct { + // Connector state patches from this shard's C:StartedCommit. + // State Update Wire Format. + ConnectorPatchesJson []byte `protobuf:"bytes,1,opt,name=connector_patches_json,json=connectorPatches,proto3" json:"connector_patches_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *CaptureRequestExt_StartCommit) 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.RuntimeCheckpoint != nil { - { - size, err := m.RuntimeCheckpoint.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) +func (m *Materialize_StartedCommit) Reset() { *m = Materialize_StartedCommit{} } +func (m *Materialize_StartedCommit) String() string { return proto.CompactTextString(m) } +func (*Materialize_StartedCommit) ProtoMessage() {} +func (*Materialize_StartedCommit) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 8} +} +func (m *Materialize_StartedCommit) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_StartedCommit) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_StartedCommit.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err } - i-- - dAtA[i] = 0xa + return b[:n], nil } - return len(dAtA) - i, nil +} +func (m *Materialize_StartedCommit) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_StartedCommit.Merge(m, src) +} +func (m *Materialize_StartedCommit) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_StartedCommit) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_StartedCommit.DiscardUnknown(m) } -func (m *CaptureResponseExt) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err +var xxx_messageInfo_Materialize_StartedCommit proto.InternalMessageInfo + +// Leader → Shards. Transaction complete. +type Materialize_Acknowledge struct { + // Aggregated StartedCommit connector state patches from all shards. + // State Update Wire Format. + ConnectorPatchesJson []byte `protobuf:"bytes,1,opt,name=connector_patches_json,json=connectorPatches,proto3" json:"connector_patches_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Materialize_Acknowledge) Reset() { *m = Materialize_Acknowledge{} } +func (m *Materialize_Acknowledge) String() string { return proto.CompactTextString(m) } +func (*Materialize_Acknowledge) ProtoMessage() {} +func (*Materialize_Acknowledge) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 9} +} +func (m *Materialize_Acknowledge) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Acknowledge) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Acknowledge.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil } - return dAtA[:n], nil +} +func (m *Materialize_Acknowledge) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Acknowledge.Merge(m, src) +} +func (m *Materialize_Acknowledge) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Acknowledge) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Acknowledge.DiscardUnknown(m) } -func (m *CaptureResponseExt) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +var xxx_messageInfo_Materialize_Acknowledge proto.InternalMessageInfo + +// Shard → Leader. Reports C:Acknowledged state from prior transaction. +// Sent by each shard after its connector responds to C:Acknowledge. +type Materialize_Acknowledged struct { + // Connector state patches from this shard's C:Acknowledged. + // State Update Wire Format. + ConnectorPatchesJson []byte `protobuf:"bytes,1,opt,name=connector_patches_json,json=connectorPatches,proto3" json:"connector_patches_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *CaptureResponseExt) 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.Checkpoint != nil { - { - size, err := m.Checkpoint.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) +func (m *Materialize_Acknowledged) Reset() { *m = Materialize_Acknowledged{} } +func (m *Materialize_Acknowledged) String() string { return proto.CompactTextString(m) } +func (*Materialize_Acknowledged) ProtoMessage() {} +func (*Materialize_Acknowledged) Descriptor() ([]byte, []int) { + return fileDescriptor_73af6e0737ce390c, []int{28, 10} +} +func (m *Materialize_Acknowledged) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Materialize_Acknowledged) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Materialize_Acknowledged.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err } - i-- - dAtA[i] = 0x22 + return b[:n], nil } - if m.Captured != nil { - { - size, err := m.Captured.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } - if m.Opened != nil { - { - size, err := m.Opened.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]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil +} +func (m *Materialize_Acknowledged) XXX_Merge(src proto.Message) { + xxx_messageInfo_Materialize_Acknowledged.Merge(m, src) +} +func (m *Materialize_Acknowledged) XXX_Size() int { + return m.ProtoSize() +} +func (m *Materialize_Acknowledged) XXX_DiscardUnknown() { + xxx_messageInfo_Materialize_Acknowledged.DiscardUnknown(m) } -func (m *CaptureResponseExt_Opened) 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 +var xxx_messageInfo_Materialize_Acknowledged proto.InternalMessageInfo + +func init() { + proto.RegisterEnum("runtime.Plane", Plane_name, Plane_value) + 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") + proto.RegisterType((*RocksDBDescriptor)(nil), "runtime.RocksDBDescriptor") + proto.RegisterType((*Container)(nil), "runtime.Container") + proto.RegisterMapType((map[uint32]string)(nil), "runtime.Container.MappedHostPortsEntry") + proto.RegisterType((*CaptureRequestExt)(nil), "runtime.CaptureRequestExt") + proto.RegisterType((*CaptureRequestExt_StartCommit)(nil), "runtime.CaptureRequestExt.StartCommit") + proto.RegisterType((*CaptureResponseExt)(nil), "runtime.CaptureResponseExt") + proto.RegisterType((*CaptureResponseExt_Opened)(nil), "runtime.CaptureResponseExt.Opened") + proto.RegisterType((*CaptureResponseExt_Captured)(nil), "runtime.CaptureResponseExt.Captured") + proto.RegisterType((*CaptureResponseExt_Checkpoint)(nil), "runtime.CaptureResponseExt.Checkpoint") + proto.RegisterType((*DeriveRequestExt)(nil), "runtime.DeriveRequestExt") + proto.RegisterType((*DeriveRequestExt_Open)(nil), "runtime.DeriveRequestExt.Open") + proto.RegisterType((*DeriveResponseExt)(nil), "runtime.DeriveResponseExt") + proto.RegisterType((*DeriveResponseExt_Opened)(nil), "runtime.DeriveResponseExt.Opened") + proto.RegisterType((*DeriveResponseExt_Published)(nil), "runtime.DeriveResponseExt.Published") + proto.RegisterType((*DeriveResponseExt_Flushed)(nil), "runtime.DeriveResponseExt.Flushed") + proto.RegisterType((*MaterializeRequestExt)(nil), "runtime.MaterializeRequestExt") + proto.RegisterType((*MaterializeResponseExt)(nil), "runtime.MaterializeResponseExt") + proto.RegisterType((*MaterializeResponseExt_Flushed)(nil), "runtime.MaterializeResponseExt.Flushed") + proto.RegisterType((*CombineRequest)(nil), "runtime.CombineRequest") + proto.RegisterType((*CombineRequest_Open)(nil), "runtime.CombineRequest.Open") + proto.RegisterType((*CombineRequest_Open_Binding)(nil), "runtime.CombineRequest.Open.Binding") + proto.RegisterType((*CombineRequest_Add)(nil), "runtime.CombineRequest.Add") + proto.RegisterType((*CombineResponse)(nil), "runtime.CombineResponse") + proto.RegisterType((*ConnectorProxyRequest)(nil), "runtime.ConnectorProxyRequest") + proto.RegisterType((*ConnectorProxyResponse)(nil), "runtime.ConnectorProxyResponse") + proto.RegisterType((*Join)(nil), "runtime.Join") + proto.RegisterType((*Join_Shard)(nil), "runtime.Join.Shard") + proto.RegisterType((*Joined)(nil), "runtime.Joined") + proto.RegisterType((*Task)(nil), "runtime.Task") + proto.RegisterType((*Recover)(nil), "runtime.Recover") + proto.RegisterMapType((map[string][]byte)(nil), "runtime.Recover.AckIntentsEntry") + proto.RegisterMapType((map[uint32][]byte)(nil), "runtime.Recover.MaxKeysEntry") + proto.RegisterType((*Persist)(nil), "runtime.Persist") + proto.RegisterMapType((map[string][]byte)(nil), "runtime.Persist.AckIntentsEntry") + proto.RegisterMapType((map[uint32][]byte)(nil), "runtime.Persist.MaxKeysEntry") + proto.RegisterType((*Persisted)(nil), "runtime.Persisted") + proto.RegisterType((*Apply)(nil), "runtime.Apply") + proto.RegisterType((*Applied)(nil), "runtime.Applied") + proto.RegisterType((*Open)(nil), "runtime.Open") + proto.RegisterMapType((map[uint32][]byte)(nil), "runtime.Open.MaxKeysEntry") + proto.RegisterType((*CloseNow)(nil), "runtime.CloseNow") + proto.RegisterType((*Stop)(nil), "runtime.Stop") + proto.RegisterType((*Stopped)(nil), "runtime.Stopped") + proto.RegisterType((*SessionLoop)(nil), "runtime.SessionLoop") + proto.RegisterType((*Materialize)(nil), "runtime.Materialize") + proto.RegisterType((*Materialize_Opened)(nil), "runtime.Materialize.Opened") + proto.RegisterType((*Materialize_Load)(nil), "runtime.Materialize.Load") + proto.RegisterType((*Materialize_Loaded)(nil), "runtime.Materialize.Loaded") + proto.RegisterType((*Materialize_Loaded_Binding)(nil), "runtime.Materialize.Loaded.Binding") + proto.RegisterType((*Materialize_Flush)(nil), "runtime.Materialize.Flush") + proto.RegisterType((*Materialize_Flushed)(nil), "runtime.Materialize.Flushed") + proto.RegisterType((*Materialize_Flushed_Binding)(nil), "runtime.Materialize.Flushed.Binding") + proto.RegisterType((*Materialize_Store)(nil), "runtime.Materialize.Store") + proto.RegisterType((*Materialize_Stored)(nil), "runtime.Materialize.Stored") + proto.RegisterType((*Materialize_Stored_Binding)(nil), "runtime.Materialize.Stored.Binding") + proto.RegisterType((*Materialize_StartCommit)(nil), "runtime.Materialize.StartCommit") + proto.RegisterType((*Materialize_StartedCommit)(nil), "runtime.Materialize.StartedCommit") + proto.RegisterType((*Materialize_Acknowledge)(nil), "runtime.Materialize.Acknowledge") + proto.RegisterType((*Materialize_Acknowledged)(nil), "runtime.Materialize.Acknowledged") } -func (m *CaptureResponseExt_Opened) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +func init() { + proto.RegisterFile("go/protocols/runtime/runtime.proto", fileDescriptor_73af6e0737ce390c) } -func (m *CaptureResponseExt_Opened) 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.RuntimeCheckpoint != nil { - { - size, err := m.RuntimeCheckpoint.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 +var fileDescriptor_73af6e0737ce390c = []byte{ + // 3784 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x3a, 0x4b, 0x6c, 0x1c, 0x47, + 0x76, 0x9e, 0xff, 0xf4, 0x9b, 0xe1, 0x70, 0xa6, 0x44, 0xc9, 0xed, 0xb1, 0x2d, 0x52, 0x63, 0x3b, + 0xe6, 0x4a, 0xd4, 0x90, 0x4b, 0x7b, 0x77, 0x6d, 0x61, 0xb5, 0x16, 0x39, 0x24, 0x6d, 0x6a, 0x29, + 0x89, 0x29, 0x52, 0x42, 0x92, 0x4b, 0xa3, 0xd9, 0x55, 0x1c, 0xb6, 0xd8, 0xd3, 0xd5, 0xdb, 0xd5, + 0x43, 0x6a, 0xf6, 0x9a, 0x4b, 0x80, 0x1c, 0x72, 0xc9, 0x2d, 0x97, 0xe4, 0x14, 0x20, 0x48, 0x0e, + 0xb9, 0xee, 0x21, 0xa7, 0x00, 0xf1, 0x31, 0x48, 0x80, 0x20, 0x27, 0x01, 0xd9, 0x5c, 0x73, 0x73, + 0x02, 0x24, 0x42, 0x0e, 0x8b, 0xfa, 0xf4, 0x67, 0x3e, 0xa4, 0x25, 0xda, 0x07, 0x1f, 0x24, 0x76, + 0xbd, 0x5f, 0xbf, 0x7a, 0xf5, 0x7e, 0xf5, 0x7a, 0xa0, 0xd3, 0x67, 0xab, 0x41, 0xc8, 0x22, 0xe6, + 0x30, 0x8f, 0xaf, 0x86, 0x43, 0x3f, 0x72, 0x07, 0x34, 0xfe, 0xdb, 0x95, 0x18, 0x54, 0xd1, 0xcb, + 0xf6, 0xcd, 0xa3, 0x90, 0x9d, 0xd2, 0x30, 0x61, 0x48, 0x1e, 0x14, 0x61, 0x7b, 0xc9, 0x61, 0x3e, + 0x1f, 0x0e, 0x2e, 0xa3, 0x18, 0x7b, 0x1d, 0xa1, 0xa1, 0x7b, 0x46, 0xf5, 0x1f, 0x4d, 0xf1, 0xde, + 0x18, 0xc5, 0xb1, 0xc7, 0xce, 0xe5, 0x7f, 0x1a, 0x7b, 0x7b, 0x0c, 0x3b, 0xb0, 0x23, 0x1a, 0xba, + 0xb6, 0xe7, 0xfe, 0x9a, 0x66, 0x9f, 0x35, 0x6d, 0x7b, 0x8c, 0x96, 0x05, 0xf2, 0x9f, 0xc6, 0x8d, + 0x6f, 0x9b, 0x9f, 0x0c, 0x8f, 0x8f, 0x3d, 0x1a, 0xff, 0xd5, 0x34, 0x0b, 0x7d, 0xd6, 0x67, 0xf2, + 0x71, 0x55, 0x3c, 0x29, 0x68, 0xe7, 0x1f, 0x72, 0xd0, 0x3a, 0xb4, 0xf9, 0xe9, 0x01, 0x0d, 0xcf, + 0x5c, 0x87, 0xf6, 0x98, 0x7f, 0xec, 0xf6, 0xd1, 0x4d, 0xa8, 0x79, 0xac, 0x6f, 0x1d, 0xbb, 0x1e, + 0xb5, 0x8e, 0x89, 0x99, 0x5b, 0xca, 0x2d, 0x97, 0xb0, 0xe1, 0xb1, 0xfe, 0x8e, 0xeb, 0xd1, 0x1d, + 0x82, 0xde, 0x05, 0x23, 0xb2, 0xf9, 0xa9, 0xe5, 0xdb, 0x03, 0x6a, 0xe6, 0x97, 0x72, 0xcb, 0x06, + 0xae, 0x0a, 0xc0, 0x63, 0x7b, 0x40, 0xd1, 0x3b, 0x50, 0x1d, 0x12, 0x6e, 0x05, 0x76, 0x74, 0x62, + 0x16, 0x24, 0xae, 0x32, 0x24, 0x7c, 0xdf, 0x8e, 0x4e, 0xd0, 0x1d, 0x68, 0x39, 0xcc, 0x8f, 0x6c, + 0xd7, 0xa7, 0xa1, 0xe5, 0xd3, 0xe8, 0x9c, 0x85, 0xa7, 0x66, 0x51, 0xd2, 0x34, 0x13, 0xc4, 0x63, + 0x05, 0x47, 0x1f, 0x42, 0x29, 0xf0, 0x6c, 0x9f, 0x9a, 0xe5, 0xa5, 0xdc, 0x72, 0x63, 0xbd, 0xd1, + 0x8d, 0x8f, 0x71, 0x5f, 0x40, 0xb1, 0x42, 0x76, 0xfe, 0xaf, 0x08, 0x8d, 0x03, 0xb5, 0x51, 0x4c, + 0x7f, 0x35, 0xa4, 0x3c, 0x42, 0xbb, 0x50, 0x79, 0xce, 0x86, 0xa1, 0x6f, 0x7b, 0x52, 0x73, 0x63, + 0x73, 0xf5, 0xd5, 0xcb, 0xc5, 0x3b, 0x7d, 0xd6, 0xed, 0xdb, 0xbf, 0xa6, 0x51, 0x44, 0xbb, 0x84, + 0x9e, 0xad, 0x3a, 0x2c, 0xa4, 0xab, 0x13, 0x0e, 0xd0, 0x7d, 0xa8, 0xd8, 0x70, 0xcc, 0x8f, 0x6e, + 0x40, 0x39, 0xa4, 0x81, 0x67, 0x8f, 0xe4, 0x2e, 0xab, 0x58, 0xaf, 0xc4, 0x1e, 0x8f, 0x86, 0xae, + 0x47, 0x2c, 0x97, 0xc4, 0x7b, 0x94, 0xeb, 0x5d, 0x82, 0x76, 0xa0, 0xcc, 0x8e, 0x8f, 0x39, 0x8d, + 0xe4, 0xc6, 0x0a, 0x9b, 0xdd, 0x57, 0x2f, 0x17, 0x6f, 0xbf, 0xce, 0xcb, 0x9f, 0x48, 0x2e, 0xac, + 0xb9, 0xd1, 0x23, 0x00, 0xea, 0x13, 0x4b, 0xcb, 0x2a, 0x5d, 0x49, 0x96, 0x41, 0x7d, 0xa2, 0x1e, + 0xd1, 0x1d, 0x28, 0x85, 0xb6, 0xdf, 0x57, 0xd6, 0xac, 0xad, 0xcf, 0x77, 0xa5, 0x1b, 0x62, 0x01, + 0x3a, 0x08, 0xa8, 0xb3, 0x59, 0xfc, 0xfa, 0xe5, 0xe2, 0x5b, 0x58, 0xd1, 0xa0, 0x03, 0xa8, 0x39, + 0x8c, 0x85, 0xc4, 0xf5, 0xed, 0x88, 0x85, 0x66, 0x45, 0x5a, 0xf1, 0xc7, 0xaf, 0x5e, 0x2e, 0xde, + 0x9d, 0xf5, 0xf2, 0xa9, 0x30, 0xe9, 0x1e, 0x9c, 0xd8, 0x21, 0xd9, 0xdd, 0xc2, 0x59, 0x29, 0x68, + 0x0d, 0x20, 0xa4, 0x9c, 0x79, 0xc3, 0xc8, 0x65, 0xbe, 0x59, 0x95, 0x6a, 0x34, 0xbb, 0x09, 0xcf, + 0x57, 0xd4, 0x26, 0x34, 0xc4, 0x19, 0x1a, 0xf4, 0x01, 0xcc, 0x69, 0x1f, 0xb6, 0x5c, 0x9f, 0xd0, + 0x17, 0xa6, 0xb1, 0x94, 0x5b, 0x9e, 0xc3, 0x75, 0x0d, 0xdc, 0x15, 0x30, 0xf4, 0x29, 0x80, 0x8c, + 0x38, 0x5b, 0x8a, 0x05, 0x29, 0x76, 0x41, 0xed, 0xae, 0xc7, 0x3c, 0x8f, 0x3a, 0x02, 0x2e, 0xb6, + 0x88, 0x33, 0x74, 0xa8, 0x07, 0xf3, 0x69, 0x88, 0x29, 0xd6, 0x9a, 0x64, 0x7d, 0x47, 0xb1, 0x3e, + 0x1a, 0x47, 0x4a, 0xfe, 0x49, 0x8e, 0xce, 0xbf, 0x14, 0x61, 0x3e, 0xf1, 0x3d, 0x1e, 0x30, 0x9f, + 0x53, 0xb4, 0x0c, 0x65, 0x1e, 0xd9, 0xd1, 0x90, 0x4b, 0xdf, 0x6b, 0xac, 0x37, 0xbb, 0xb1, 0x79, + 0xba, 0x07, 0x12, 0x8e, 0x35, 0x5e, 0x50, 0x9e, 0xc8, 0x3d, 0x4b, 0xdf, 0x9a, 0x65, 0x0b, 0x8d, + 0x47, 0x1f, 0x41, 0x23, 0xa2, 0xe1, 0xc0, 0xf5, 0x6d, 0xcf, 0xa2, 0x61, 0xc8, 0x42, 0xed, 0x73, + 0x73, 0x31, 0x74, 0x5b, 0x00, 0xd1, 0xef, 0x43, 0x3d, 0xa4, 0x36, 0xb1, 0xa2, 0x93, 0x90, 0x0d, + 0xfb, 0x27, 0x57, 0xf4, 0xbf, 0x9a, 0x90, 0x71, 0xa8, 0x44, 0x08, 0x27, 0x3c, 0x0f, 0xdd, 0x88, + 0x5a, 0x42, 0x93, 0xab, 0x3a, 0xa1, 0x94, 0x20, 0xb6, 0x84, 0x76, 0xa1, 0x64, 0x87, 0xd4, 0xb7, + 0xa5, 0x13, 0xd6, 0x37, 0x3f, 0x79, 0xf5, 0x72, 0x71, 0xb5, 0xef, 0x46, 0x27, 0xc3, 0xa3, 0xae, + 0xc3, 0x06, 0xab, 0x94, 0x47, 0x43, 0x3b, 0x1c, 0xa9, 0x34, 0x39, 0x95, 0x38, 0xbb, 0x1b, 0x82, + 0x15, 0x2b, 0x09, 0xe8, 0x23, 0x28, 0x12, 0xe6, 0x70, 0xb3, 0xb2, 0x54, 0x58, 0xae, 0xad, 0xd7, + 0xd4, 0xa9, 0x1d, 0x78, 0xae, 0x43, 0xb5, 0x2b, 0x4b, 0x34, 0xfa, 0x0a, 0x2a, 0x2a, 0x82, 0xb8, + 0x59, 0x5d, 0x2a, 0x5c, 0x41, 0xfb, 0x98, 0x5d, 0xf8, 0xd9, 0x70, 0xe8, 0x12, 0x2b, 0xb0, 0xc3, + 0x88, 0x9b, 0x86, 0x7c, 0xad, 0x8e, 0xa2, 0xa7, 0x4f, 0x77, 0xb7, 0xf6, 0x05, 0x58, 0xbf, 0xda, + 0x10, 0x84, 0x12, 0x20, 0x9c, 0x3e, 0xb0, 0x9d, 0x53, 0x4a, 0xac, 0x53, 0x3a, 0x32, 0xe1, 0x22, + 0x65, 0x0d, 0x45, 0xf4, 0x4b, 0x3a, 0xea, 0x10, 0x68, 0x61, 0xe6, 0x9c, 0xf2, 0xad, 0xcd, 0x2d, + 0xca, 0x9d, 0xd0, 0x0d, 0x44, 0xec, 0xac, 0x00, 0x0a, 0x05, 0x90, 0x1c, 0x59, 0xd4, 0x3f, 0xb3, + 0x06, 0x74, 0x10, 0x44, 0xa1, 0xf4, 0xb0, 0x32, 0x6e, 0x6a, 0xcc, 0xb6, 0x7f, 0xf6, 0x48, 0xc2, + 0xd1, 0x2d, 0xa8, 0xc7, 0xd4, 0x32, 0x0b, 0xab, 0x0c, 0x5d, 0xd3, 0x30, 0x91, 0x89, 0x3b, 0x7f, + 0x9e, 0x07, 0xa3, 0x17, 0x67, 0x5c, 0xf4, 0x36, 0x54, 0xdc, 0xc0, 0xb2, 0x09, 0x51, 0x32, 0x0d, + 0x5c, 0x76, 0x83, 0x0d, 0x42, 0x42, 0xf4, 0x53, 0x98, 0xd3, 0x69, 0xda, 0x0a, 0x98, 0xd8, 0x77, + 0x5e, 0xee, 0xa0, 0xa5, 0x76, 0xa0, 0x33, 0xf5, 0x3e, 0x0b, 0x23, 0x5c, 0xf7, 0xd3, 0x05, 0x47, + 0x07, 0xd0, 0x1a, 0xd8, 0x41, 0x40, 0x89, 0x75, 0xc2, 0x78, 0xa4, 0x79, 0x0b, 0x92, 0xf7, 0xe3, + 0x24, 0x8f, 0x27, 0xef, 0xef, 0x3e, 0x92, 0xb4, 0x5f, 0x31, 0x1e, 0x49, 0xf6, 0x6d, 0x3f, 0x0a, + 0x47, 0x22, 0xdc, 0xc6, 0xa0, 0xe8, 0x7d, 0x80, 0x21, 0xb7, 0xfb, 0xd4, 0x0a, 0xed, 0x88, 0x4a, + 0xef, 0xce, 0x63, 0x43, 0x42, 0xb0, 0x1d, 0xd1, 0xf6, 0x26, 0x2c, 0xcc, 0x92, 0x83, 0x9a, 0x50, + 0x10, 0xb6, 0xcf, 0xc9, 0xdc, 0x21, 0x1e, 0xd1, 0x02, 0x94, 0xce, 0x6c, 0x6f, 0x18, 0x97, 0x2e, + 0xb5, 0xb8, 0x97, 0xff, 0x2c, 0xd7, 0xf9, 0x9b, 0x3c, 0xb4, 0x7a, 0x76, 0x10, 0x0d, 0xc3, 0xb8, + 0x9a, 0x6c, 0xbf, 0x10, 0xb9, 0x53, 0xd4, 0x3e, 0xcb, 0xa3, 0x67, 0xd4, 0xd3, 0x61, 0xdd, 0xe8, + 0x8a, 0xea, 0xbb, 0xc7, 0xfa, 0xdd, 0x3d, 0x01, 0xc5, 0x55, 0x8f, 0xf5, 0xe5, 0x13, 0xda, 0x4d, + 0x8f, 0x8a, 0x24, 0x07, 0xa8, 0x43, 0xbc, 0x9d, 0xec, 0x7d, 0xea, 0x88, 0x71, 0x4b, 0x73, 0x65, + 0x4e, 0x7d, 0x17, 0xea, 0x3c, 0xb2, 0xc3, 0xc8, 0x72, 0xd8, 0x60, 0xe0, 0x46, 0x32, 0xea, 0x6b, + 0xeb, 0xbf, 0x97, 0x1a, 0x70, 0x52, 0x53, 0x91, 0x62, 0xc2, 0xa8, 0x27, 0xa9, 0x71, 0x8d, 0xa7, + 0x8b, 0x36, 0x86, 0x5a, 0x06, 0x87, 0x7a, 0x80, 0xb4, 0x10, 0xcb, 0x39, 0xa1, 0xce, 0x69, 0xc0, + 0x5c, 0x3f, 0x92, 0x5b, 0x13, 0xc9, 0x33, 0xc9, 0x58, 0xbd, 0x04, 0x87, 0x5b, 0x9a, 0x3e, 0x05, + 0x75, 0xfe, 0xbf, 0x08, 0x28, 0x51, 0x41, 0xa5, 0x3f, 0x61, 0xad, 0x35, 0x30, 0x92, 0x5a, 0xae, + 0x45, 0xa2, 0xe9, 0x33, 0xc7, 0x29, 0x11, 0xba, 0x07, 0x65, 0x16, 0x50, 0x9f, 0x12, 0x6d, 0xa6, + 0xce, 0xf4, 0x0e, 0x13, 0xf1, 0xdd, 0x27, 0x92, 0x12, 0x6b, 0x0e, 0xf4, 0x00, 0xaa, 0x8e, 0x22, + 0x22, 0xda, 0x3e, 0x1f, 0x5e, 0xc6, 0xad, 0x41, 0x04, 0x27, 0x5c, 0x68, 0x07, 0x20, 0x63, 0x83, + 0xe2, 0x45, 0x36, 0xce, 0xc8, 0x48, 0xad, 0x92, 0xe1, 0x6c, 0x3f, 0x82, 0xb2, 0xd2, 0xed, 0x7b, + 0xb1, 0x6e, 0xfb, 0x19, 0x54, 0x63, 0x65, 0x85, 0xe7, 0x9f, 0xd2, 0x91, 0xa5, 0x92, 0x84, 0x14, + 0x54, 0xc7, 0xc6, 0x29, 0x1d, 0xed, 0x4b, 0x80, 0x68, 0xab, 0x44, 0x56, 0x72, 0x45, 0x51, 0xe2, + 0x31, 0x55, 0x5e, 0x52, 0x35, 0x53, 0x84, 0x22, 0x6e, 0x9f, 0x03, 0xa4, 0x6f, 0x41, 0x4b, 0x50, + 0x12, 0xe5, 0x88, 0x6b, 0xed, 0x40, 0xba, 0xb5, 0x28, 0x54, 0x1c, 0x2b, 0x04, 0xfa, 0x12, 0x6a, + 0x01, 0xf3, 0x3c, 0x2b, 0xa4, 0x7c, 0xe8, 0x45, 0x52, 0x6c, 0xe3, 0x72, 0xfb, 0xec, 0x33, 0xcf, + 0xc3, 0x92, 0x1a, 0x43, 0x90, 0x3c, 0x77, 0x1e, 0x03, 0xa4, 0x18, 0x54, 0x83, 0xca, 0xee, 0xe3, + 0x67, 0x1b, 0x7b, 0xbb, 0x5b, 0xcd, 0xb7, 0x90, 0x01, 0x25, 0xbc, 0xbd, 0xb1, 0xf5, 0x87, 0xcd, + 0x1c, 0x9a, 0x03, 0xe3, 0xf1, 0x93, 0x43, 0x4b, 0x2d, 0xf3, 0xa8, 0x0e, 0xd5, 0xde, 0x93, 0x27, + 0x7b, 0xd6, 0x93, 0x9d, 0x9d, 0x66, 0x41, 0x30, 0xe1, 0xed, 0x83, 0xc3, 0x0d, 0x7c, 0xd8, 0x2c, + 0x76, 0xfe, 0x2b, 0x07, 0xcd, 0x2d, 0xd9, 0x6b, 0xff, 0x00, 0x42, 0x75, 0x1d, 0x8a, 0xc2, 0x21, + 0xb5, 0x0b, 0xde, 0x4c, 0x98, 0x27, 0x15, 0x94, 0xee, 0x8b, 0x25, 0x6d, 0x7b, 0x05, 0x8a, 0x62, + 0x85, 0x3e, 0x84, 0x06, 0xff, 0x95, 0x27, 0xaa, 0xec, 0xd9, 0x31, 0xb7, 0x86, 0xa1, 0xab, 0x93, + 0x70, 0x5d, 0x41, 0x9f, 0x1d, 0xf3, 0xa7, 0xa1, 0xdb, 0xf9, 0xef, 0x02, 0xb4, 0x62, 0x69, 0xdf, + 0x25, 0xd8, 0x3e, 0x9f, 0x08, 0xb6, 0x5b, 0x53, 0xba, 0x5e, 0x18, 0x6b, 0x9b, 0x60, 0x04, 0xc3, + 0x23, 0xcf, 0xe5, 0x27, 0x33, 0x82, 0x6d, 0x9a, 0x7b, 0x3f, 0xa6, 0xc5, 0x29, 0x1b, 0xfa, 0x39, + 0x54, 0x8e, 0xbd, 0xa1, 0x94, 0x50, 0x9c, 0x08, 0xf6, 0x69, 0x09, 0x3b, 0x8a, 0x12, 0xc7, 0x2c, + 0xdf, 0x77, 0x8c, 0x45, 0x60, 0x24, 0x4a, 0x8a, 0x4b, 0xcd, 0xc0, 0x7e, 0x61, 0x39, 0x1e, 0x73, + 0x4e, 0x75, 0x69, 0xad, 0x0e, 0xec, 0x17, 0x3d, 0xb1, 0x9e, 0x88, 0xc0, 0xfc, 0x6b, 0x45, 0x60, + 0xe1, 0x82, 0x08, 0xbc, 0x03, 0x15, 0xbd, 0xb1, 0x6f, 0x0f, 0xbf, 0xce, 0x9f, 0xe5, 0xe0, 0x7a, + 0xda, 0x8c, 0xfe, 0x00, 0x5c, 0xbd, 0xf3, 0x9b, 0x1c, 0xdc, 0x18, 0xd3, 0xe8, 0xbb, 0x78, 0xe3, + 0x46, 0xea, 0x0e, 0x4a, 0x99, 0xb4, 0x3d, 0x98, 0xfd, 0x8e, 0x69, 0x9f, 0x78, 0x23, 0x73, 0xfe, + 0xa6, 0x08, 0x8d, 0x1e, 0x1b, 0x1c, 0xb9, 0x7e, 0x72, 0x5d, 0x5c, 0xd3, 0xa1, 0xab, 0x78, 0xde, + 0xcb, 0xe8, 0x9b, 0x25, 0xcb, 0x04, 0x2e, 0xba, 0x0b, 0x05, 0x9b, 0xc4, 0x0a, 0xbf, 0x7b, 0x11, + 0xc3, 0x06, 0x21, 0x58, 0xd0, 0xb5, 0xff, 0x2d, 0xaf, 0x03, 0xfd, 0x01, 0x54, 0x8f, 0x5c, 0x9f, + 0xb8, 0x7e, 0x5f, 0x68, 0x58, 0x18, 0xaf, 0x55, 0xd3, 0x6f, 0xeb, 0x6e, 0x2a, 0x62, 0x9c, 0x70, + 0xb5, 0xff, 0x34, 0x0f, 0x15, 0x0d, 0x45, 0x08, 0x8a, 0xc7, 0x43, 0x4f, 0x1d, 0x7d, 0x15, 0xcb, + 0xe7, 0xb8, 0xd7, 0x11, 0x5d, 0x9a, 0xa1, 0x7a, 0x9d, 0xcf, 0xa0, 0x16, 0x84, 0xec, 0xb9, 0xba, + 0x06, 0xc5, 0x3d, 0x58, 0x53, 0xf5, 0x6f, 0xfb, 0x09, 0x42, 0xb7, 0xa1, 0x59, 0x52, 0x74, 0x1f, + 0x6a, 0xdc, 0x39, 0xa1, 0x03, 0xdb, 0x7a, 0xce, 0x99, 0x2f, 0xa3, 0xb5, 0xbe, 0xf9, 0xde, 0xab, + 0x97, 0x8b, 0x26, 0xf5, 0x1d, 0x26, 0x54, 0x58, 0x15, 0x88, 0x2e, 0xb6, 0xcf, 0x1f, 0x51, 0x2e, + 0xdb, 0x30, 0x50, 0x0c, 0x0f, 0x39, 0xf3, 0x51, 0x17, 0x80, 0xd3, 0xd0, 0x0a, 0x98, 0xe7, 0x3a, + 0x23, 0x79, 0x75, 0x48, 0xfa, 0xe5, 0x03, 0x1a, 0xee, 0x4b, 0x30, 0x36, 0x78, 0xfc, 0x28, 0xc7, + 0x06, 0xb2, 0xbf, 0x8e, 0x42, 0x79, 0x3d, 0x30, 0x70, 0x45, 0xb6, 0xd1, 0x51, 0x28, 0x6e, 0xe1, + 0xb2, 0x45, 0x53, 0xdd, 0xbe, 0x81, 0xf5, 0xaa, 0xed, 0x43, 0x61, 0x83, 0x10, 0x64, 0x42, 0x45, + 0x1b, 0x48, 0x37, 0x79, 0xf1, 0x12, 0xfd, 0x0c, 0xaa, 0x84, 0x39, 0x4a, 0xff, 0xfc, 0x6b, 0xe8, + 0x5f, 0x21, 0xcc, 0x91, 0xca, 0x2f, 0x40, 0xe9, 0x38, 0x64, 0xbe, 0x6a, 0xb9, 0xaa, 0x58, 0x2d, + 0x3a, 0xff, 0x9e, 0x83, 0xf9, 0xe4, 0x9c, 0xf4, 0x7d, 0xef, 0xe2, 0x97, 0x9b, 0x50, 0x21, 0xd4, + 0xa3, 0x91, 0x76, 0xed, 0x2a, 0x8e, 0x97, 0x63, 0x6a, 0x15, 0xae, 0xa4, 0x56, 0x31, 0xa3, 0xd6, + 0x44, 0x6e, 0x2a, 0x4d, 0xe6, 0xa6, 0x0f, 0x60, 0x4e, 0xd9, 0x2b, 0xa6, 0x90, 0x97, 0x2f, 0x5c, + 0x57, 0x40, 0x45, 0xd4, 0x79, 0x1b, 0xae, 0xf7, 0x98, 0xef, 0x53, 0x27, 0x62, 0xe1, 0x7e, 0xc8, + 0x5e, 0x8c, 0xb4, 0x23, 0x76, 0xfe, 0x22, 0x07, 0x37, 0x26, 0x31, 0x7a, 0xeb, 0x0f, 0xa1, 0x22, + 0xae, 0x0c, 0x94, 0x73, 0x3d, 0x67, 0x59, 0x7b, 0xf5, 0x72, 0x71, 0xe5, 0x75, 0xee, 0x56, 0xdb, + 0x3e, 0x51, 0x39, 0x39, 0x16, 0x20, 0x4e, 0x3f, 0x10, 0xc2, 0x2d, 0x97, 0xe8, 0xae, 0xbc, 0x22, + 0xd7, 0xbb, 0x04, 0xb5, 0xa1, 0xe0, 0xb1, 0xbe, 0xae, 0x37, 0xd5, 0x38, 0xc3, 0x61, 0x01, 0xec, + 0xfc, 0x7d, 0x01, 0x8a, 0x0f, 0x99, 0xeb, 0xa3, 0xdb, 0xd0, 0xa2, 0x91, 0x43, 0xac, 0x01, 0x23, + 0x56, 0x48, 0xcf, 0x5c, 0x2e, 0x6e, 0xf4, 0x42, 0xab, 0x02, 0x9e, 0x17, 0x88, 0x47, 0x8c, 0x60, + 0x0d, 0x46, 0x77, 0xa0, 0xcc, 0x4f, 0xec, 0x90, 0xc4, 0xb7, 0x99, 0x6b, 0x49, 0x10, 0x0a, 0x51, + 0x6a, 0x78, 0x81, 0x35, 0x09, 0x5a, 0x84, 0x9a, 0x7c, 0xd2, 0x13, 0x88, 0x82, 0x3c, 0x63, 0x90, + 0x20, 0x35, 0x7f, 0xb8, 0x03, 0xad, 0x78, 0x48, 0x41, 0xdc, 0x50, 0x9a, 0x69, 0x14, 0xcf, 0xb4, + 0x34, 0x62, 0x2b, 0x86, 0xa3, 0x1f, 0x41, 0x0c, 0xb3, 0xa8, 0xb6, 0x81, 0x3c, 0x30, 0x03, 0xcf, + 0x6b, 0x78, 0x6c, 0x1a, 0xf4, 0x31, 0xcc, 0x7b, 0xf2, 0xfa, 0x9f, 0x52, 0xaa, 0xb0, 0x68, 0x28, + 0x70, 0x4c, 0xd8, 0xfe, 0xbb, 0x1c, 0x94, 0xa4, 0xce, 0xa8, 0x01, 0x79, 0x97, 0xe8, 0xe6, 0x21, + 0xef, 0x12, 0xd4, 0x85, 0xaa, 0x67, 0x1f, 0x51, 0x4f, 0x38, 0x67, 0x5e, 0x67, 0x63, 0x99, 0x11, + 0x05, 0xf5, 0x9e, 0xc6, 0xe0, 0x84, 0x06, 0xad, 0x43, 0x25, 0xa4, 0xb6, 0xd0, 0x54, 0x5b, 0xdb, + 0x4c, 0x47, 0x12, 0xfb, 0x21, 0x73, 0x28, 0xe7, 0x07, 0x01, 0x75, 0xba, 0xbb, 0x5b, 0x38, 0x26, + 0x44, 0x6b, 0xb0, 0x20, 0x0d, 0xef, 0x84, 0xd4, 0x8e, 0x68, 0x6a, 0x7b, 0x39, 0x7c, 0xc0, 0x48, + 0xe0, 0x7a, 0x12, 0x15, 0x9b, 0xbf, 0xf3, 0x29, 0x94, 0x85, 0x9d, 0x29, 0x11, 0x87, 0x26, 0x2a, + 0xae, 0xe4, 0x9f, 0x3c, 0xb4, 0x81, 0xfd, 0x62, 0x3b, 0x72, 0x92, 0x43, 0xeb, 0xfc, 0x53, 0x0e, + 0x8a, 0x87, 0x36, 0x3f, 0x15, 0x69, 0x8f, 0x07, 0xd4, 0xd1, 0x5d, 0xb0, 0x7c, 0x16, 0x82, 0x58, + 0xc0, 0x2d, 0x99, 0xe2, 0xad, 0x78, 0xf6, 0xa7, 0xdc, 0x68, 0x9e, 0x05, 0x5c, 0x56, 0x00, 0x3d, + 0xdb, 0x43, 0xf7, 0xa0, 0x91, 0xd2, 0x4a, 0x49, 0x85, 0x4b, 0x66, 0x46, 0xf5, 0x98, 0x5d, 0xac, + 0x44, 0x48, 0x07, 0x42, 0x51, 0x7a, 0xae, 0x23, 0x30, 0x5e, 0x8a, 0x83, 0x15, 0x5b, 0x89, 0x42, + 0xdb, 0xe7, 0xb6, 0xce, 0xb5, 0x25, 0xe9, 0x2b, 0x62, 0x27, 0x87, 0x19, 0x70, 0xe7, 0xaf, 0x4a, + 0x50, 0xc1, 0xd4, 0x61, 0x67, 0xb2, 0xfc, 0xd5, 0x6c, 0xe7, 0xd4, 0x72, 0xfd, 0x88, 0xfa, 0x51, + 0x5c, 0x14, 0x96, 0xd2, 0x7a, 0xac, 0xc8, 0xba, 0x1b, 0xce, 0xe9, 0xae, 0x22, 0x51, 0x57, 0x63, + 0xb0, 0x13, 0x00, 0x5a, 0x87, 0xeb, 0xea, 0x7a, 0x18, 0x51, 0x22, 0x9a, 0x17, 0x4e, 0x75, 0x0b, + 0x93, 0x97, 0x2d, 0xcc, 0xb5, 0x04, 0xd9, 0x13, 0x38, 0xd5, 0xcd, 0x3c, 0x00, 0x94, 0xf2, 0xc8, + 0x24, 0xe2, 0xd2, 0xf8, 0xcc, 0x5b, 0xdd, 0x78, 0x6e, 0xbc, 0xa3, 0x11, 0xb8, 0x95, 0x10, 0xc7, + 0x20, 0xb4, 0x02, 0x0b, 0x4e, 0x9c, 0x15, 0xa4, 0x2d, 0x69, 0xa6, 0x4a, 0xe0, 0x46, 0x82, 0x13, + 0xb6, 0xa3, 0x68, 0x05, 0xd0, 0x89, 0xd8, 0xe3, 0xb8, 0x82, 0x25, 0x35, 0xbe, 0x50, 0x98, 0x8c, + 0x76, 0xf7, 0x60, 0x5e, 0x53, 0x27, 0xaa, 0x95, 0x2f, 0x52, 0xad, 0xa1, 0x28, 0x13, 0xbd, 0x6e, + 0x41, 0xdd, 0xb3, 0x79, 0x64, 0xd9, 0x41, 0xe0, 0xb9, 0x94, 0xc8, 0xd1, 0x65, 0x1d, 0xd7, 0x04, + 0x6c, 0x43, 0x81, 0xd0, 0x06, 0xb4, 0x3c, 0xda, 0xb7, 0x9d, 0x51, 0xb6, 0x71, 0xac, 0x5e, 0xd2, + 0x38, 0x36, 0x15, 0x79, 0xe6, 0xd6, 0xf4, 0x19, 0x88, 0xce, 0xd0, 0x3a, 0xa5, 0xa3, 0x78, 0x12, + 0xf4, 0xfe, 0xd4, 0x99, 0x3d, 0xb2, 0x5f, 0xfc, 0x92, 0x8e, 0xf4, 0x81, 0x55, 0x06, 0x6a, 0x85, + 0x6e, 0xc3, 0xb5, 0x28, 0x74, 0xfb, 0x7d, 0x51, 0x19, 0xed, 0xd0, 0x1e, 0x70, 0x65, 0x36, 0x90, + 0x6a, 0xce, 0x69, 0xd4, 0xbe, 0xc4, 0xb4, 0xef, 0xc3, 0xfc, 0xc4, 0xc1, 0x67, 0x67, 0x19, 0xc6, + 0x8c, 0x59, 0x46, 0x3d, 0x33, 0xcb, 0x68, 0xdf, 0x83, 0x7a, 0x56, 0x87, 0x6f, 0x9b, 0x83, 0x64, + 0x79, 0x3b, 0xff, 0x5a, 0x86, 0xca, 0x3e, 0x0d, 0xb9, 0xcb, 0x23, 0x41, 0xe5, 0x33, 0xdf, 0xa1, + 0x92, 0xb3, 0x88, 0xd5, 0x42, 0x1c, 0xa9, 0x2a, 0x67, 0x56, 0xd6, 0x81, 0x55, 0xa1, 0x6b, 0x2a, + 0x4c, 0xaa, 0xfc, 0xa4, 0x9f, 0x17, 0x26, 0xfc, 0x5c, 0xbf, 0xea, 0x6a, 0x7e, 0x5e, 0x7c, 0x53, + 0x3f, 0x2f, 0xbd, 0x81, 0x9f, 0xaf, 0xc1, 0x8d, 0xd4, 0xcf, 0x03, 0x3b, 0x72, 0x4e, 0xa8, 0x3e, + 0x32, 0x55, 0x45, 0x9b, 0x09, 0x76, 0x5f, 0x21, 0x2f, 0xf0, 0xf5, 0xca, 0x05, 0xbe, 0xfe, 0x29, + 0xdc, 0xd0, 0x66, 0x9c, 0x74, 0xf9, 0xaa, 0x34, 0xe5, 0x82, 0xc2, 0x7e, 0x35, 0xee, 0xe5, 0x33, + 0x22, 0xc4, 0xb8, 0x6a, 0x84, 0xc0, 0x6b, 0x46, 0x48, 0xed, 0xca, 0x11, 0x52, 0x9f, 0x88, 0x90, + 0xf8, 0xb4, 0x67, 0x47, 0xc8, 0x3a, 0x5c, 0xd7, 0x16, 0x19, 0x0f, 0x14, 0x73, 0x4e, 0x1a, 0xe4, + 0x9a, 0x42, 0x1e, 0x66, 0x23, 0xe5, 0xa2, 0xa8, 0x6a, 0xfc, 0xc0, 0xa2, 0xea, 0x16, 0x18, 0x7a, + 0xef, 0x94, 0xcc, 0x0e, 0xab, 0xce, 0x5f, 0xe7, 0xa0, 0x24, 0x8e, 0x61, 0x34, 0xb3, 0xce, 0x99, + 0x50, 0x39, 0x13, 0x02, 0x74, 0x3b, 0x6b, 0xe0, 0x78, 0x29, 0x2e, 0xaf, 0xf2, 0x54, 0x25, 0x8b, + 0x4a, 0xc2, 0x55, 0x01, 0x90, 0x65, 0x2b, 0x3e, 0xf2, 0x98, 0x57, 0x75, 0x1c, 0xf2, 0xc8, 0x9f, + 0x69, 0xfe, 0xb5, 0x0b, 0xf2, 0xb9, 0xf2, 0x72, 0x34, 0x9e, 0xcf, 0x45, 0x2f, 0xda, 0x79, 0x0e, + 0x95, 0xd8, 0x5f, 0xee, 0x02, 0x52, 0xc5, 0x2d, 0xb9, 0x5b, 0xc6, 0x85, 0xdc, 0xc0, 0x2d, 0x85, + 0xd9, 0x4a, 0x11, 0x97, 0xc4, 0x54, 0x7e, 0x76, 0x4c, 0x75, 0xbe, 0xc9, 0xe9, 0x1b, 0xd4, 0x9b, + 0x19, 0xe5, 0xa3, 0xf8, 0x9b, 0x57, 0x61, 0xe6, 0x37, 0xaf, 0xf8, 0x6b, 0xd7, 0x07, 0x97, 0xd6, + 0x32, 0x79, 0x71, 0xa4, 0xe8, 0x27, 0x19, 0x87, 0x2e, 0x49, 0x87, 0x4e, 0xaf, 0xcd, 0xf2, 0xb2, + 0x36, 0xd3, 0x9b, 0xbf, 0x93, 0xbb, 0x00, 0x54, 0x65, 0xa6, 0x78, 0xcc, 0xce, 0x3b, 0x65, 0x28, + 0x1e, 0x44, 0x2c, 0xe8, 0x18, 0x50, 0x11, 0x7f, 0x03, 0x4a, 0x3a, 0x7f, 0x00, 0xb5, 0x03, 0xca, + 0xc5, 0x46, 0xf7, 0x18, 0x0b, 0x2e, 0xb8, 0xe1, 0xe7, 0xae, 0x72, 0xc3, 0xff, 0xdb, 0x1b, 0x50, + 0xcb, 0xdc, 0xbe, 0xd1, 0xdd, 0x8c, 0xd5, 0x6b, 0xeb, 0xef, 0x74, 0xb3, 0x1f, 0xa7, 0xe3, 0x0b, + 0xab, 0x34, 0xa8, 0x3a, 0x90, 0x2f, 0x60, 0x4e, 0xfc, 0xb5, 0x42, 0x7d, 0x51, 0x48, 0xc6, 0x0c, + 0xe3, 0x7c, 0x0a, 0xa9, 0x18, 0xeb, 0x82, 0x21, 0xb9, 0x58, 0x7c, 0x0e, 0xd5, 0x33, 0xdb, 0x73, + 0x89, 0x1d, 0xc5, 0x47, 0xf7, 0xfe, 0xcc, 0x77, 0x3e, 0xd3, 0x44, 0x38, 0x21, 0x47, 0xf7, 0xc1, + 0x88, 0x9f, 0xe3, 0x01, 0xd3, 0xe2, 0xec, 0xf7, 0xc6, 0xcc, 0x04, 0xa7, 0x1c, 0xe8, 0x67, 0x50, + 0xe7, 0xca, 0xa6, 0x96, 0xc7, 0x58, 0x60, 0x2e, 0xe8, 0xa4, 0x17, 0x9b, 0x2f, 0x63, 0x70, 0x5c, + 0xe3, 0x19, 0xeb, 0xdf, 0x82, 0xe2, 0x73, 0xe6, 0xfa, 0xe6, 0x75, 0xc9, 0x30, 0x37, 0x76, 0xa3, + 0xc0, 0x12, 0x85, 0x3e, 0x86, 0xf2, 0x73, 0xd9, 0xf7, 0x9a, 0x37, 0xb4, 0x3b, 0x66, 0x89, 0x28, + 0xc1, 0x1a, 0x2d, 0x64, 0x45, 0x36, 0x3f, 0x35, 0xdf, 0x9e, 0x90, 0x25, 0xda, 0x5f, 0x2c, 0x51, + 0xe8, 0xb6, 0xe8, 0xd4, 0x65, 0x9f, 0x61, 0x9a, 0xfa, 0xe3, 0xe1, 0x44, 0xff, 0x81, 0x63, 0x02, + 0xf4, 0x21, 0x94, 0x44, 0xae, 0x1f, 0x99, 0xef, 0x48, 0xca, 0xf4, 0x3b, 0xba, 0xcc, 0x33, 0x58, + 0x21, 0x85, 0xc4, 0xb8, 0x22, 0xb4, 0x27, 0x24, 0xea, 0x30, 0xc7, 0x31, 0x81, 0x50, 0x50, 0x4e, + 0x4c, 0xde, 0x9d, 0x50, 0x30, 0x33, 0x22, 0xf9, 0x24, 0x99, 0x32, 0xbe, 0x37, 0x31, 0x25, 0xc9, + 0x38, 0xd6, 0xe4, 0x7c, 0xf1, 0x2e, 0x14, 0x3d, 0x66, 0x13, 0x73, 0x59, 0xfb, 0xd9, 0x2c, 0x96, + 0x3d, 0x66, 0x13, 0x2c, 0xc9, 0xc4, 0x3b, 0xc4, 0x5f, 0x4a, 0xcc, 0x1f, 0x5d, 0xf2, 0x8e, 0x3d, + 0x49, 0x82, 0x35, 0x29, 0x5a, 0x83, 0x92, 0x1c, 0x1c, 0x99, 0xb7, 0x27, 0x22, 0x23, 0xcb, 0x23, + 0xe7, 0x49, 0x58, 0x11, 0xa2, 0x9f, 0xa6, 0x23, 0xaa, 0x3b, 0x13, 0x23, 0xa2, 0x29, 0x9e, 0xcc, + 0x5c, 0x4a, 0xbc, 0x89, 0x47, 0x2c, 0xa4, 0xe6, 0xca, 0x25, 0x6f, 0x3a, 0x10, 0x14, 0x58, 0x11, + 0x8a, 0x0d, 0xc9, 0x07, 0x62, 0xde, 0xbd, 0x64, 0x43, 0x92, 0x45, 0x5c, 0x50, 0xe5, 0x5f, 0xd4, + 0x9b, 0xf8, 0x48, 0xd4, 0x95, 0xac, 0x4b, 0x17, 0xb0, 0xce, 0xfe, 0x3c, 0x84, 0x76, 0xa1, 0x21, + 0x97, 0xa2, 0x6b, 0x51, 0x62, 0x56, 0x27, 0x86, 0xb3, 0x53, 0x62, 0x28, 0xd1, 0x82, 0xe6, 0x78, + 0x76, 0x89, 0x36, 0x65, 0xab, 0xe7, 0xb3, 0x73, 0x8f, 0x92, 0x3e, 0x35, 0xd7, 0x2e, 0x51, 0x67, + 0x23, 0xa5, 0xc3, 0x59, 0x26, 0xb4, 0x0d, 0xf5, 0xcc, 0x92, 0x98, 0x3f, 0x9e, 0x98, 0x54, 0x5f, + 0x20, 0x84, 0xe0, 0x31, 0x36, 0xe1, 0xd3, 0x81, 0xaa, 0xb7, 0xe6, 0xfa, 0x84, 0x4f, 0xeb, 0x3a, + 0x8c, 0x63, 0x02, 0xb4, 0x06, 0x46, 0x10, 0xd7, 0x66, 0xf3, 0x93, 0x89, 0xd1, 0x65, 0x52, 0xb5, + 0x71, 0x4a, 0x84, 0xba, 0x60, 0xa8, 0x0e, 0xcf, 0x67, 0xe7, 0xe6, 0xa7, 0xba, 0xfd, 0x4a, 0xc6, + 0x79, 0x3a, 0x71, 0xe3, 0xaa, 0xa3, 0x9f, 0x44, 0xd4, 0xf0, 0x88, 0x05, 0xe6, 0xcf, 0x27, 0xa2, + 0x46, 0xe4, 0x73, 0x2c, 0x51, 0x42, 0x61, 0xae, 0xb2, 0xbb, 0x79, 0x7f, 0x42, 0x61, 0x9d, 0xf5, + 0x71, 0x4c, 0xd0, 0xfe, 0xe3, 0x5c, 0x32, 0x0b, 0x7f, 0xf3, 0xb1, 0xeb, 0x97, 0xd9, 0x92, 0x97, + 0x69, 0xf2, 0xf2, 0x97, 0x34, 0x79, 0xd7, 0x12, 0x8e, 0xcc, 0x04, 0xfd, 0x27, 0x50, 0x14, 0x01, + 0x86, 0xee, 0x42, 0x35, 0x69, 0x45, 0x73, 0x17, 0xb5, 0xa2, 0x09, 0x49, 0xfb, 0x9b, 0x3c, 0x94, + 0x55, 0x60, 0xa2, 0x2f, 0xa6, 0x86, 0xa2, 0x1f, 0x5c, 0x12, 0xc7, 0xd3, 0x33, 0x51, 0xd5, 0xba, + 0xc8, 0xa1, 0x5c, 0x68, 0xa9, 0xef, 0xc3, 0x47, 0xa3, 0x88, 0xaa, 0xbb, 0x48, 0x51, 0xb4, 0x2e, + 0x0a, 0xf7, 0x54, 0xa0, 0x36, 0x05, 0xa6, 0xfd, 0x3f, 0xb9, 0x74, 0x8a, 0xba, 0x00, 0x25, 0x35, + 0xd9, 0x51, 0x25, 0x59, 0x2d, 0xd0, 0x32, 0x34, 0x07, 0xae, 0x6f, 0x71, 0x36, 0x0c, 0x9d, 0xf1, + 0xfb, 0x74, 0x63, 0xe0, 0xfa, 0x07, 0x12, 0xac, 0x1a, 0xf8, 0x65, 0x75, 0xf1, 0x1f, 0xa3, 0x2c, + 0x68, 0x4a, 0xfb, 0x45, 0x96, 0x72, 0x05, 0x90, 0xa2, 0x22, 0x16, 0x61, 0x0e, 0xb7, 0x22, 0x16, + 0xd9, 0x9e, 0xac, 0x51, 0x45, 0xdc, 0xd4, 0x98, 0x2d, 0xe6, 0xf0, 0x43, 0x01, 0x47, 0x5d, 0xb8, + 0x16, 0x53, 0xcb, 0xed, 0x68, 0xf2, 0x92, 0x24, 0x6f, 0x69, 0x94, 0xdc, 0x8e, 0xa2, 0xef, 0xc0, + 0x9c, 0xee, 0x4f, 0x2c, 0x42, 0xbd, 0x48, 0xff, 0xc4, 0x02, 0xd7, 0x54, 0x23, 0xb2, 0x25, 0x40, + 0xed, 0xcf, 0xa1, 0x24, 0xb3, 0xd4, 0x25, 0x1d, 0x58, 0x6e, 0x76, 0x07, 0xd6, 0xfe, 0xdf, 0x5c, + 0x3a, 0x65, 0xbf, 0x6c, 0x8c, 0x3d, 0x23, 0x23, 0xce, 0x3c, 0xb2, 0x37, 0xec, 0x00, 0xdb, 0xa3, + 0x6f, 0x3b, 0xb1, 0xdb, 0xd0, 0x52, 0x19, 0x3e, 0x6b, 0x5c, 0xe5, 0x02, 0xf3, 0x0a, 0x91, 0xda, + 0x76, 0x05, 0x90, 0xa6, 0xcd, 0x9a, 0xb6, 0xa0, 0x4e, 0x42, 0x61, 0x52, 0xcb, 0xb6, 0x2b, 0x50, + 0x92, 0x29, 0xb7, 0xfd, 0x8f, 0x39, 0x28, 0xab, 0xe4, 0xfb, 0xda, 0x4e, 0xab, 0xc8, 0x67, 0x0c, + 0xf2, 0x5f, 0x67, 0x3f, 0x2a, 0xc1, 0xcf, 0xd8, 0x8f, 0x42, 0x8c, 0xed, 0x47, 0xd3, 0xce, 0xd8, + 0x8f, 0xc2, 0x64, 0xf6, 0xf3, 0x27, 0xb9, 0xf1, 0xdf, 0x02, 0xbc, 0xb1, 0x33, 0x7c, 0x7f, 0xd9, + 0x63, 0x03, 0xe6, 0xc6, 0x6a, 0xc9, 0x15, 0x1c, 0xf3, 0x0b, 0xa8, 0x65, 0x2a, 0xc0, 0x15, 0x04, + 0x3c, 0x80, 0x7a, 0xb6, 0x84, 0xbc, 0xb9, 0x84, 0xdb, 0x77, 0xa0, 0x24, 0x7f, 0x92, 0x88, 0x00, + 0xca, 0xfb, 0x4f, 0x37, 0xf7, 0x76, 0x7b, 0xcd, 0xb7, 0x50, 0x0d, 0x2a, 0xfb, 0x78, 0xf7, 0xd9, + 0xc6, 0xe1, 0x76, 0x33, 0x87, 0x0c, 0x28, 0xed, 0x3d, 0xe9, 0x6d, 0xec, 0x35, 0xf3, 0xeb, 0x0f, + 0xa1, 0xaa, 0x7f, 0x32, 0x16, 0xa2, 0x5f, 0x40, 0x45, 0x3f, 0xa3, 0xb7, 0xd3, 0x44, 0x3f, 0xf6, + 0x63, 0xc6, 0xb6, 0x39, 0x8d, 0x50, 0x0d, 0xec, 0x5a, 0x6e, 0x7d, 0x0f, 0xaa, 0xfa, 0x73, 0x44, + 0x88, 0x1e, 0x40, 0x45, 0x3f, 0x67, 0x64, 0x8d, 0x7f, 0x54, 0xca, 0xc8, 0x9a, 0xf8, 0x8a, 0xb1, + 0x9c, 0x5b, 0xcb, 0xad, 0x9f, 0x40, 0x63, 0x7c, 0xd0, 0x8f, 0x9e, 0xc1, 0xbc, 0x7c, 0x48, 0xc0, + 0x1c, 0xdd, 0xcc, 0xd6, 0x95, 0xe9, 0xcf, 0x05, 0xed, 0xc5, 0x0b, 0xf1, 0x99, 0x37, 0x7d, 0x09, + 0xe5, 0x3d, 0xf5, 0xcb, 0xb6, 0xfb, 0xe3, 0x17, 0x8d, 0x85, 0x59, 0x41, 0xd4, 0x9e, 0x09, 0x95, + 0x82, 0x76, 0xe2, 0xc9, 0xf7, 0x77, 0x93, 0xb3, 0xf9, 0x8b, 0xaf, 0xff, 0xe3, 0xe6, 0x5b, 0x5f, + 0xff, 0xf6, 0x66, 0xee, 0x9f, 0x7f, 0x7b, 0x33, 0xf7, 0x97, 0xff, 0x79, 0x33, 0xf7, 0x47, 0x2b, + 0xaf, 0xf5, 0xb3, 0x34, 0x2d, 0xef, 0xa8, 0x2c, 0x41, 0x9f, 0xfc, 0x2e, 0x00, 0x00, 0xff, 0xff, + 0xe4, 0x09, 0x98, 0xc9, 0x7f, 0x2c, 0x00, 0x00, } -func (m *CaptureResponseExt_Captured) 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 +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// ShufflerClient is the client API for Shuffler service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type ShufflerClient interface { + Shuffle(ctx context.Context, in *ShuffleRequest, opts ...grpc.CallOption) (Shuffler_ShuffleClient, error) } -func (m *CaptureResponseExt_Captured) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +type shufflerClient struct { + cc *grpc.ClientConn } -func (m *CaptureResponseExt_Captured) 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 len(m.PartitionsPacked) > 0 { - i -= len(m.PartitionsPacked) - copy(dAtA[i:], m.PartitionsPacked) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.PartitionsPacked))) - i-- - dAtA[i] = 0x12 - } - if len(m.KeyPacked) > 0 { - i -= len(m.KeyPacked) - copy(dAtA[i:], m.KeyPacked) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.KeyPacked))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil +func NewShufflerClient(cc *grpc.ClientConn) ShufflerClient { + return &shufflerClient{cc} } -func (m *CaptureResponseExt_Checkpoint) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) +func (c *shufflerClient) Shuffle(ctx context.Context, in *ShuffleRequest, opts ...grpc.CallOption) (Shuffler_ShuffleClient, error) { + stream, err := c.cc.NewStream(ctx, &_Shuffler_serviceDesc.Streams[0], "/runtime.Shuffler/Shuffle", opts...) if err != nil { return nil, err } - return dAtA[:n], nil + x := &shufflerShuffleClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil } -func (m *CaptureResponseExt_Checkpoint) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +type Shuffler_ShuffleClient interface { + Recv() (*ShuffleResponse, error) + grpc.ClientStream } -func (m *CaptureResponseExt_Checkpoint) 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.PollResult != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.PollResult)) - i-- - dAtA[i] = 0x10 - } - 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 +type shufflerShuffleClient struct { + grpc.ClientStream } -func (m *DeriveRequestExt) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { +func (x *shufflerShuffleClient) Recv() (*ShuffleResponse, error) { + m := new(ShuffleResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { return nil, err } - return dAtA[:n], nil + return m, nil } -func (m *DeriveRequestExt) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +// ShufflerServer is the server API for Shuffler service. +type ShufflerServer interface { + Shuffle(*ShuffleRequest, Shuffler_ShuffleServer) error } -func (m *DeriveRequestExt) 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.Open != nil { - { - size, err := m.Open.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a +// UnimplementedShufflerServer can be embedded to have forward compatible implementations. +type UnimplementedShufflerServer struct { +} + +func (*UnimplementedShufflerServer) Shuffle(req *ShuffleRequest, srv Shuffler_ShuffleServer) error { + return status.Errorf(codes.Unimplemented, "method Shuffle not implemented") +} + +func RegisterShufflerServer(s *grpc.Server, srv ShufflerServer) { + s.RegisterService(&_Shuffler_serviceDesc, srv) +} + +func _Shuffler_Shuffle_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(ShuffleRequest) + if err := stream.RecvMsg(m); err != nil { + return err } - if m.RocksdbDescriptor != nil { - { - size, err := m.RocksdbDescriptor.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - } - if m.LogLevel != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.LogLevel)) - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil + return srv.(ShufflerServer).Shuffle(m, &shufflerShuffleServer{stream}) } -func (m *DeriveRequestExt_Open) 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 +type Shuffler_ShuffleServer interface { + Send(*ShuffleResponse) error + grpc.ServerStream } -func (m *DeriveRequestExt_Open) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +type shufflerShuffleServer struct { + grpc.ServerStream } -func (m *DeriveRequestExt_Open) 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 len(m.SqliteVfsUri) > 0 { - i -= len(m.SqliteVfsUri) - copy(dAtA[i:], m.SqliteVfsUri) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.SqliteVfsUri))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil +func (x *shufflerShuffleServer) Send(m *ShuffleResponse) error { + return x.ServerStream.SendMsg(m) } -func (m *DeriveResponseExt) 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 +var _Shuffler_serviceDesc = grpc.ServiceDesc{ + ServiceName: "runtime.Shuffler", + HandlerType: (*ShufflerServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "Shuffle", + Handler: _Shuffler_Shuffle_Handler, + ServerStreams: true, + }, + }, + Metadata: "go/protocols/runtime/runtime.proto", } -func (m *DeriveResponseExt) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +// CombinerClient is the client API for Combiner service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type CombinerClient interface { + Combine(ctx context.Context, opts ...grpc.CallOption) (Combiner_CombineClient, error) } -func (m *DeriveResponseExt) 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.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] = 0x22 - } - if m.Published != nil { - { - size, err := m.Published.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } - if m.Opened != nil { - { - size, err := m.Opened.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]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil +type combinerClient struct { + cc *grpc.ClientConn } -func (m *DeriveResponseExt_Opened) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) +func NewCombinerClient(cc *grpc.ClientConn) CombinerClient { + return &combinerClient{cc} +} + +func (c *combinerClient) Combine(ctx context.Context, opts ...grpc.CallOption) (Combiner_CombineClient, error) { + stream, err := c.cc.NewStream(ctx, &_Combiner_serviceDesc.Streams[0], "/runtime.Combiner/Combine", opts...) if err != nil { return nil, err } - return dAtA[:n], nil + x := &combinerCombineClient{stream} + return x, nil } -func (m *DeriveResponseExt_Opened) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +type Combiner_CombineClient interface { + Send(*CombineRequest) error + Recv() (*CombineResponse, error) + grpc.ClientStream } -func (m *DeriveResponseExt_Opened) 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.RuntimeCheckpoint != nil { - { - size, err := m.RuntimeCheckpoint.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 +type combinerCombineClient struct { + grpc.ClientStream } -func (m *DeriveResponseExt_Published) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { +func (x *combinerCombineClient) Send(m *CombineRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *combinerCombineClient) Recv() (*CombineResponse, error) { + m := new(CombineResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { return nil, err } - return dAtA[:n], nil + return m, nil } -func (m *DeriveResponseExt_Published) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +// CombinerServer is the server API for Combiner service. +type CombinerServer interface { + Combine(Combiner_CombineServer) error } -func (m *DeriveResponseExt_Published) 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 len(m.PartitionsPacked) > 0 { - i -= len(m.PartitionsPacked) - copy(dAtA[i:], m.PartitionsPacked) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.PartitionsPacked))) - i-- - dAtA[i] = 0x1a - } - if len(m.KeyPacked) > 0 { - i -= len(m.KeyPacked) - copy(dAtA[i:], m.KeyPacked) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.KeyPacked))) - i-- - dAtA[i] = 0x12 - } - if m.MaxClock != 0 { - i -= 8 - encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.MaxClock)) - i-- - dAtA[i] = 0x9 - } - return len(dAtA) - i, nil +// UnimplementedCombinerServer can be embedded to have forward compatible implementations. +type UnimplementedCombinerServer struct { } -func (m *DeriveResponseExt_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 (*UnimplementedCombinerServer) Combine(srv Combiner_CombineServer) error { + return status.Errorf(codes.Unimplemented, "method Combine not implemented") } -func (m *DeriveResponseExt_Flushed) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +func RegisterCombinerServer(s *grpc.Server, srv CombinerServer) { + s.RegisterService(&_Combiner_serviceDesc, srv) } -func (m *DeriveResponseExt_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 _Combiner_Combine_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(CombinerServer).Combine(&combinerCombineServer{stream}) } -func (m *MaterializeRequestExt) 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 +type Combiner_CombineServer interface { + Send(*CombineResponse) error + Recv() (*CombineRequest, error) + grpc.ServerStream } -func (m *MaterializeRequestExt) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +type combinerCombineServer struct { + grpc.ServerStream } -func (m *MaterializeRequestExt) 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.RocksdbDescriptor != nil { - { - size, err := m.RocksdbDescriptor.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - } - if m.LogLevel != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.LogLevel)) - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil +func (x *combinerCombineServer) Send(m *CombineResponse) error { + return x.ServerStream.SendMsg(m) } -func (m *MaterializeResponseExt) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { +func (x *combinerCombineServer) Recv() (*CombineRequest, error) { + m := new(CombineRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { return nil, err } - return dAtA[:n], nil + return m, nil } -func (m *MaterializeResponseExt) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +var _Combiner_serviceDesc = grpc.ServiceDesc{ + ServiceName: "runtime.Combiner", + HandlerType: (*CombinerServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "Combine", + Handler: _Combiner_Combine_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "go/protocols/runtime/runtime.proto", } -func (m *MaterializeResponseExt) 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.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]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil +// ConnectorProxyClient is the client API for ConnectorProxy service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type ConnectorProxyClient interface { + ProxyConnectors(ctx context.Context, opts ...grpc.CallOption) (ConnectorProxy_ProxyConnectorsClient, error) } -func (m *MaterializeResponseExt_Flushed) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) +type connectorProxyClient struct { + cc *grpc.ClientConn +} + +func NewConnectorProxyClient(cc *grpc.ClientConn) ConnectorProxyClient { + return &connectorProxyClient{cc} +} + +func (c *connectorProxyClient) ProxyConnectors(ctx context.Context, opts ...grpc.CallOption) (ConnectorProxy_ProxyConnectorsClient, error) { + stream, err := c.cc.NewStream(ctx, &_ConnectorProxy_serviceDesc.Streams[0], "/runtime.ConnectorProxy/ProxyConnectors", opts...) if err != nil { return nil, err } - return dAtA[:n], nil + x := &connectorProxyProxyConnectorsClient{stream} + return x, nil } -func (m *MaterializeResponseExt_Flushed) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +type ConnectorProxy_ProxyConnectorsClient interface { + Send(*ConnectorProxyRequest) error + Recv() (*ConnectorProxyResponse, error) + grpc.ClientStream } -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 +type connectorProxyProxyConnectorsClient struct { + grpc.ClientStream } -func (m *CombineRequest) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { +func (x *connectorProxyProxyConnectorsClient) Send(m *ConnectorProxyRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *connectorProxyProxyConnectorsClient) Recv() (*ConnectorProxyResponse, error) { + m := new(ConnectorProxyResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { return nil, err } - return dAtA[:n], nil + return m, nil } -func (m *CombineRequest) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +// ConnectorProxyServer is the server API for ConnectorProxy service. +type ConnectorProxyServer interface { + ProxyConnectors(ConnectorProxy_ProxyConnectorsServer) error } -func (m *CombineRequest) 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.Add != nil { - { - size, err := m.Add.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - } - if m.Open != nil { - { - size, err := m.Open.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 +// UnimplementedConnectorProxyServer can be embedded to have forward compatible implementations. +type UnimplementedConnectorProxyServer struct { } -func (m *CombineRequest_Open) 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 (*UnimplementedConnectorProxyServer) ProxyConnectors(srv ConnectorProxy_ProxyConnectorsServer) error { + return status.Errorf(codes.Unimplemented, "method ProxyConnectors not implemented") } -func (m *CombineRequest_Open) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +func RegisterConnectorProxyServer(s *grpc.Server, srv ConnectorProxyServer) { + s.RegisterService(&_ConnectorProxy_serviceDesc, srv) } -func (m *CombineRequest_Open) 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 len(m.Bindings) > 0 { - for iNdEx := len(m.Bindings) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Bindings[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } +func _ConnectorProxy_ProxyConnectors_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(ConnectorProxyServer).ProxyConnectors(&connectorProxyProxyConnectorsServer{stream}) +} + +type ConnectorProxy_ProxyConnectorsServer interface { + Send(*ConnectorProxyResponse) error + Recv() (*ConnectorProxyRequest, error) + grpc.ServerStream +} + +type connectorProxyProxyConnectorsServer struct { + grpc.ServerStream +} + +func (x *connectorProxyProxyConnectorsServer) Send(m *ConnectorProxyResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *connectorProxyProxyConnectorsServer) Recv() (*ConnectorProxyRequest, error) { + m := new(ConnectorProxyRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err } - return len(dAtA) - i, nil + return m, nil } -func (m *CombineRequest_Open_Binding) Marshal() (dAtA []byte, err error) { - size := m.ProtoSize() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) +var _ConnectorProxy_serviceDesc = grpc.ServiceDesc{ + ServiceName: "runtime.ConnectorProxy", + HandlerType: (*ConnectorProxyServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "ProxyConnectors", + Handler: _ConnectorProxy_ProxyConnectors_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "go/protocols/runtime/runtime.proto", +} + +// LeaderClient is the client API for Leader service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type LeaderClient interface { + // rpc Derive(stream .runtime.Derive) returns (stream .runtime.Derive); + Materialize(ctx context.Context, opts ...grpc.CallOption) (Leader_MaterializeClient, error) +} + +type leaderClient struct { + cc *grpc.ClientConn +} + +func NewLeaderClient(cc *grpc.ClientConn) LeaderClient { + return &leaderClient{cc} +} + +func (c *leaderClient) Materialize(ctx context.Context, opts ...grpc.CallOption) (Leader_MaterializeClient, error) { + stream, err := c.cc.NewStream(ctx, &_Leader_serviceDesc.Streams[0], "/runtime.Leader/Materialize", opts...) if err != nil { return nil, err } - return dAtA[:n], nil + x := &leaderMaterializeClient{stream} + return x, nil } -func (m *CombineRequest_Open_Binding) MarshalTo(dAtA []byte) (int, error) { - size := m.ProtoSize() - return m.MarshalToSizedBuffer(dAtA[:size]) +type Leader_MaterializeClient interface { + Send(*Materialize) error + Recv() (*Materialize, error) + grpc.ClientStream } -func (m *CombineRequest_Open_Binding) 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 len(m.Values) > 0 { - for iNdEx := len(m.Values) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Values[iNdEx]) - copy(dAtA[i:], m.Values[iNdEx]) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.Values[iNdEx]))) - i-- - dAtA[i] = 0x3a - } +type leaderMaterializeClient struct { + grpc.ClientStream +} + +func (x *leaderMaterializeClient) Send(m *Materialize) error { + return x.ClientStream.SendMsg(m) +} + +func (x *leaderMaterializeClient) Recv() (*Materialize, error) { + m := new(Materialize) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err } - if len(m.UuidPtr) > 0 { - i -= len(m.UuidPtr) - copy(dAtA[i:], m.UuidPtr) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.UuidPtr))) - i-- - dAtA[i] = 0x32 + return m, nil +} + +// LeaderServer is the server API for Leader service. +type LeaderServer interface { + // rpc Derive(stream .runtime.Derive) returns (stream .runtime.Derive); + Materialize(Leader_MaterializeServer) error +} + +// UnimplementedLeaderServer can be embedded to have forward compatible implementations. +type UnimplementedLeaderServer struct { +} + +func (*UnimplementedLeaderServer) Materialize(srv Leader_MaterializeServer) error { + return status.Errorf(codes.Unimplemented, "method Materialize not implemented") +} + +func RegisterLeaderServer(s *grpc.Server, srv LeaderServer) { + s.RegisterService(&_Leader_serviceDesc, srv) +} + +func _Leader_Materialize_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(LeaderServer).Materialize(&leaderMaterializeServer{stream}) +} + +type Leader_MaterializeServer interface { + Send(*Materialize) error + Recv() (*Materialize, error) + grpc.ServerStream +} + +type leaderMaterializeServer struct { + grpc.ServerStream +} + +func (x *leaderMaterializeServer) Send(m *Materialize) error { + return x.ServerStream.SendMsg(m) +} + +func (x *leaderMaterializeServer) Recv() (*Materialize, error) { + m := new(Materialize) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err } - if m.SerPolicy != nil { + return m, nil +} + +var _Leader_serviceDesc = grpc.ServiceDesc{ + ServiceName: "runtime.Leader", + HandlerType: (*LeaderServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ { - size, err := m.SerPolicy.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x2a - } - if len(m.SchemaJson) > 0 { - i -= len(m.SchemaJson) - copy(dAtA[i:], m.SchemaJson) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.SchemaJson))) - i-- - dAtA[i] = 0x22 - } - if len(m.Projections) > 0 { - for iNdEx := len(m.Projections) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Projections[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } + StreamName: "Materialize", + Handler: _Leader_Materialize_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "go/protocols/runtime/runtime.proto", +} + +// ShardClient is the client API for Shard service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type ShardClient interface { + // rpc Derive(stream .runtime.Derive) returns (stream .runtime.Derive); + Materialize(ctx context.Context, opts ...grpc.CallOption) (Shard_MaterializeClient, error) +} + +type shardClient struct { + cc *grpc.ClientConn +} + +func NewShardClient(cc *grpc.ClientConn) ShardClient { + return &shardClient{cc} +} + +func (c *shardClient) Materialize(ctx context.Context, opts ...grpc.CallOption) (Shard_MaterializeClient, error) { + stream, err := c.cc.NewStream(ctx, &_Shard_serviceDesc.Streams[0], "/runtime.Shard/Materialize", opts...) + if err != nil { + return nil, err } - if len(m.Key) > 0 { - for iNdEx := len(m.Key) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Key[iNdEx]) - copy(dAtA[i:], m.Key[iNdEx]) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.Key[iNdEx]))) - i-- - dAtA[i] = 0x12 - } + x := &shardMaterializeClient{stream} + return x, nil +} + +type Shard_MaterializeClient interface { + Send(*Materialize) error + Recv() (*Materialize, error) + grpc.ClientStream +} + +type shardMaterializeClient struct { + grpc.ClientStream +} + +func (x *shardMaterializeClient) Send(m *Materialize) error { + return x.ClientStream.SendMsg(m) +} + +func (x *shardMaterializeClient) Recv() (*Materialize, error) { + m := new(Materialize) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err } - if m.Full { - i-- - if m.Full { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x8 + return m, nil +} + +// ShardServer is the server API for Shard service. +type ShardServer interface { + // rpc Derive(stream .runtime.Derive) returns (stream .runtime.Derive); + Materialize(Shard_MaterializeServer) error +} + +// UnimplementedShardServer can be embedded to have forward compatible implementations. +type UnimplementedShardServer struct { +} + +func (*UnimplementedShardServer) Materialize(srv Shard_MaterializeServer) error { + return status.Errorf(codes.Unimplemented, "method Materialize not implemented") +} + +func RegisterShardServer(s *grpc.Server, srv ShardServer) { + s.RegisterService(&_Shard_serviceDesc, srv) +} + +func _Shard_Materialize_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(ShardServer).Materialize(&shardMaterializeServer{stream}) +} + +type Shard_MaterializeServer interface { + Send(*Materialize) error + Recv() (*Materialize, error) + grpc.ServerStream +} + +type shardMaterializeServer struct { + grpc.ServerStream +} + +func (x *shardMaterializeServer) Send(m *Materialize) error { + return x.ServerStream.SendMsg(m) +} + +func (x *shardMaterializeServer) Recv() (*Materialize, error) { + m := new(Materialize) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err } - return len(dAtA) - i, nil + return m, nil } -func (m *CombineRequest_Add) Marshal() (dAtA []byte, err error) { +var _Shard_serviceDesc = grpc.ServiceDesc{ + ServiceName: "runtime.Shard", + HandlerType: (*ShardServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "Materialize", + Handler: _Shard_Materialize_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "go/protocols/runtime/runtime.proto", +} + +func (m *TaskServiceConfig) Marshal() (dAtA []byte, err error) { size := m.ProtoSize() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -3165,12 +3697,12 @@ func (m *CombineRequest_Add) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *CombineRequest_Add) MarshalTo(dAtA []byte) (int, error) { +func (m *TaskServiceConfig) MarshalTo(dAtA []byte) (int, error) { size := m.ProtoSize() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *CombineRequest_Add) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *TaskServiceConfig) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -3179,32 +3711,41 @@ func (m *CombineRequest_Add) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } - if m.Front { + if m.Plane != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.Plane)) i-- - if m.Front { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } + dAtA[i] = 0x30 + } + if len(m.ContainerNetwork) > 0 { + i -= len(m.ContainerNetwork) + copy(dAtA[i:], m.ContainerNetwork) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ContainerNetwork))) i-- - dAtA[i] = 0x18 + dAtA[i] = 0x22 } - if len(m.DocJson) > 0 { - i -= len(m.DocJson) - copy(dAtA[i:], m.DocJson) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.DocJson))) + if len(m.UdsPath) > 0 { + i -= len(m.UdsPath) + copy(dAtA[i:], m.UdsPath) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.UdsPath))) + i-- + dAtA[i] = 0x1a + } + if len(m.TaskName) > 0 { + i -= len(m.TaskName) + copy(dAtA[i:], m.TaskName) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.TaskName))) i-- dAtA[i] = 0x12 } - if m.Binding != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.Binding)) + if m.LogFileFd != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.LogFileFd)) i-- dAtA[i] = 0x8 } return len(dAtA) - i, nil } -func (m *CombineResponse) Marshal() (dAtA []byte, err error) { +func (m *ShuffleRequest) Marshal() (dAtA []byte, err error) { size := m.ProtoSize() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -3214,12 +3755,12 @@ func (m *CombineResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *CombineResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *ShuffleRequest) MarshalTo(dAtA []byte) (int, error) { size := m.ProtoSize() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *CombineResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *ShuffleRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -3228,40 +3769,84 @@ func (m *CombineResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } - if len(m.ValuesPacked) > 0 { - i -= len(m.ValuesPacked) - copy(dAtA[i:], m.ValuesPacked) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.ValuesPacked))) + if m.Materialization != nil { + { + size, err := m.Materialization.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } i-- - dAtA[i] = 0x32 + dAtA[i] = 0x5a } - if len(m.KeyPacked) > 0 { - i -= len(m.KeyPacked) - copy(dAtA[i:], m.KeyPacked) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.KeyPacked))) + if m.Derivation != nil { + { + size, err := m.Derivation.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } i-- - dAtA[i] = 0x2a + dAtA[i] = 0x52 } - if m.Front { + if m.ShuffleIndex != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.ShuffleIndex)) i-- - if m.Front { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + dAtA[i] = 0x48 + } + if m.Resolution != nil { + { + size, err := m.Resolution.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + if len(m.Coordinator) > 0 { + i -= len(m.Coordinator) + copy(dAtA[i:], m.Coordinator) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Coordinator))) + i-- + dAtA[i] = 0x3a + } + { + size, err := m.Range.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + if m.EndOffset != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.EndOffset)) + i-- + dAtA[i] = 0x28 + } + if m.Offset != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.Offset)) i-- dAtA[i] = 0x20 } - if len(m.DocJson) > 0 { - i -= len(m.DocJson) - copy(dAtA[i:], m.DocJson) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.DocJson))) + if len(m.BuildId) > 0 { + i -= len(m.BuildId) + copy(dAtA[i:], m.BuildId) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.BuildId))) i-- dAtA[i] = 0x1a } - if m.Deleted { + if m.Replay { i-- - if m.Deleted { + if m.Replay { dAtA[i] = 1 } else { dAtA[i] = 0 @@ -3269,15 +3854,17 @@ func (m *CombineResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x10 } - if m.Binding != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.Binding)) + if len(m.Journal) > 0 { + i -= len(m.Journal) + copy(dAtA[i:], m.Journal) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Journal))) i-- - dAtA[i] = 0x8 + dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *ConnectorProxyRequest) Marshal() (dAtA []byte, err error) { +func (m *ShuffleResponse) Marshal() (dAtA []byte, err error) { size := m.ProtoSize() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -3287,12 +3874,12 @@ func (m *ConnectorProxyRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *ConnectorProxyRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *ShuffleResponse) MarshalTo(dAtA []byte) (int, error) { size := m.ProtoSize() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *ConnectorProxyRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *ShuffleResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -3301,10 +3888,112 @@ func (m *ConnectorProxyRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.PackedKey) > 0 { + for iNdEx := len(m.PackedKey) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.PackedKey[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x52 + } + } + if len(m.UuidParts) > 0 { + for iNdEx := len(m.UuidParts) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.UuidParts[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } + } + if len(m.Offsets) > 0 { + dAtA6 := make([]byte, len(m.Offsets)*10) + var j5 int + for _, num1 := range m.Offsets { + num := uint64(num1) + for num >= 1<<7 { + dAtA6[j5] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j5++ + } + dAtA6[j5] = uint8(num) + j5++ + } + i -= j5 + copy(dAtA[i:], dAtA6[:j5]) + i = encodeVarintRuntime(dAtA, i, uint64(j5)) + i-- + dAtA[i] = 0x42 + } + if len(m.Docs) > 0 { + for iNdEx := len(m.Docs) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Docs[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3a + } + } + if len(m.Arena) > 0 { + i -= len(m.Arena) + copy(dAtA[i:], m.Arena) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Arena))) + i-- + dAtA[i] = 0x32 + } + if m.WriteHead != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.WriteHead)) + i-- + dAtA[i] = 0x28 + } + if m.ReadThrough != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.ReadThrough)) + i-- + dAtA[i] = 0x20 + } + if len(m.TerminalError) > 0 { + i -= len(m.TerminalError) + copy(dAtA[i:], m.TerminalError) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.TerminalError))) + i-- + dAtA[i] = 0x1a + } + if m.Header != nil { + { + size, err := m.Header.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.Status != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.Status)) + i-- + dAtA[i] = 0x8 + } return len(dAtA) - i, nil } -func (m *ConnectorProxyResponse) Marshal() (dAtA []byte, err error) { +func (m *RocksDBDescriptor) Marshal() (dAtA []byte, err error) { size := m.ProtoSize() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -3314,12 +4003,12 @@ func (m *ConnectorProxyResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *ConnectorProxyResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *RocksDBDescriptor) MarshalTo(dAtA []byte) (int, error) { size := m.ProtoSize() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *ConnectorProxyResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *RocksDBDescriptor) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -3328,716 +4017,9267 @@ func (m *ConnectorProxyResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } - if m.Log != nil { - { - size, err := m.Log.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintRuntime(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } - if len(m.ProxyId) > 0 { - i -= len(m.ProxyId) - copy(dAtA[i:], m.ProxyId) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.ProxyId))) + if len(m.RocksdbPath) > 0 { + i -= len(m.RocksdbPath) + copy(dAtA[i:], m.RocksdbPath) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.RocksdbPath))) i-- dAtA[i] = 0x12 } - if len(m.Address) > 0 { - i -= len(m.Address) - copy(dAtA[i:], m.Address) - i = encodeVarintRuntime(dAtA, i, uint64(len(m.Address))) + if m.RocksdbEnvMemptr != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.RocksdbEnvMemptr)) i-- - dAtA[i] = 0xa + dAtA[i] = 0x9 } return len(dAtA) - i, nil } -func encodeVarintRuntime(dAtA []byte, offset int, v uint64) int { - offset -= sovRuntime(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ +func (m *Container) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err } - dAtA[offset] = uint8(v) - return base + return dAtA[:n], nil } -func (m *TaskServiceConfig) ProtoSize() (n int) { - if m == nil { - return 0 - } + +func (m *Container) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Container) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if m.LogFileFd != 0 { - n += 1 + sovRuntime(uint64(m.LogFileFd)) - } - l = len(m.TaskName) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - l = len(m.UdsPath) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) + if m.UsageRate != 0 { + i -= 4 + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.UsageRate)))) + i-- + dAtA[i] = 0x25 } - l = len(m.ContainerNetwork) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) + if len(m.MappedHostPorts) > 0 { + for k := range m.MappedHostPorts { + v := m.MappedHostPorts[k] + baseI := i + i -= len(v) + copy(dAtA[i:], v) + i = encodeVarintRuntime(dAtA, i, uint64(len(v))) + i-- + dAtA[i] = 0x12 + i = encodeVarintRuntime(dAtA, i, uint64(k)) + i-- + dAtA[i] = 0x8 + i = encodeVarintRuntime(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0x1a + } } - if m.Plane != 0 { - n += 1 + sovRuntime(uint64(m.Plane)) + if len(m.NetworkPorts) > 0 { + for iNdEx := len(m.NetworkPorts) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.NetworkPorts[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + if len(m.IpAddr) > 0 { + i -= len(m.IpAddr) + copy(dAtA[i:], m.IpAddr) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.IpAddr))) + i-- + dAtA[i] = 0xa } - return n + return len(dAtA) - i, nil } -func (m *ShuffleRequest) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *CaptureRequestExt) 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 *CaptureRequestExt) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CaptureRequestExt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - l = len(m.Journal) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if m.Replay { - n += 2 - } - l = len(m.BuildId) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if m.Offset != 0 { - n += 1 + sovRuntime(uint64(m.Offset)) - } - if m.EndOffset != 0 { - n += 1 + sovRuntime(uint64(m.EndOffset)) - } - l = m.Range.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - l = len(m.Coordinator) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - if m.Resolution != nil { - l = m.Resolution.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.StartCommit != nil { + { + size, err := m.StartCommit.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a } - if m.ShuffleIndex != 0 { - n += 1 + sovRuntime(uint64(m.ShuffleIndex)) + if m.RocksdbDescriptor != nil { + { + size, err := m.RocksdbDescriptor.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 } - if m.Derivation != nil { - l = m.Derivation.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.LogLevel != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.LogLevel)) + i-- + dAtA[i] = 0x8 } - if m.Materialization != nil { - l = m.Materialization.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + return len(dAtA) - i, nil +} + +func (m *CaptureRequestExt_StartCommit) 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 *CaptureRequestExt_StartCommit) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CaptureRequestExt_StartCommit) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - return n + if m.RuntimeCheckpoint != nil { + { + size, err := m.RuntimeCheckpoint.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 (m *ShuffleResponse) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *CaptureResponseExt) 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 *CaptureResponseExt) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CaptureResponseExt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if m.Status != 0 { - n += 1 + sovRuntime(uint64(m.Status)) + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - if m.Header != nil { - l = m.Header.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } - l = len(m.TerminalError) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if m.ReadThrough != 0 { - n += 1 + sovRuntime(uint64(m.ReadThrough)) - } - if m.WriteHead != 0 { - n += 1 + sovRuntime(uint64(m.WriteHead)) - } - l = len(m.Arena) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if len(m.Docs) > 0 { - for _, e := range m.Docs { - l = e.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.Checkpoint != nil { + { + size, err := m.Checkpoint.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) } + i-- + dAtA[i] = 0x22 } - if len(m.Offsets) > 0 { - l = 0 - for _, e := range m.Offsets { - l += sovRuntime(uint64(e)) + if m.Captured != nil { + { + size, err := m.Captured.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) } - n += 1 + sovRuntime(uint64(l)) + l + i-- + dAtA[i] = 0x1a } - if len(m.UuidParts) > 0 { - for _, e := range m.UuidParts { - l = e.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.Opened != nil { + { + size, err := m.Opened.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) } + i-- + dAtA[i] = 0x12 } - if len(m.PackedKey) > 0 { - for _, e := range m.PackedKey { - l = e.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.Container != nil { + { + size, err := m.Container.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) } + i-- + dAtA[i] = 0xa } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) - } - return n + return len(dAtA) - i, nil } -func (m *RocksDBDescriptor) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *CaptureResponseExt_Opened) 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 *CaptureResponseExt_Opened) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CaptureResponseExt_Opened) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if m.RocksdbEnvMemptr != 0 { - n += 9 - } - l = len(m.RocksdbPath) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - return n + if m.RuntimeCheckpoint != nil { + { + size, err := m.RuntimeCheckpoint.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 (m *Container) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *CaptureResponseExt_Captured) 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 *CaptureResponseExt_Captured) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CaptureResponseExt_Captured) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - l = len(m.IpAddr) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if len(m.NetworkPorts) > 0 { - for _, e := range m.NetworkPorts { - l = e.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } - } - if len(m.MappedHostPorts) > 0 { - for k, v := range m.MappedHostPorts { - _ = k - _ = v - mapEntrySize := 1 + sovRuntime(uint64(k)) + 1 + len(v) + sovRuntime(uint64(len(v))) - n += mapEntrySize + 1 + sovRuntime(uint64(mapEntrySize)) - } + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - if m.UsageRate != 0 { - n += 5 + if len(m.PartitionsPacked) > 0 { + i -= len(m.PartitionsPacked) + copy(dAtA[i:], m.PartitionsPacked) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.PartitionsPacked))) + i-- + dAtA[i] = 0x12 } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + if len(m.KeyPacked) > 0 { + i -= len(m.KeyPacked) + copy(dAtA[i:], m.KeyPacked) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.KeyPacked))) + i-- + dAtA[i] = 0xa } - return n + return len(dAtA) - i, nil } -func (m *CaptureRequestExt) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *CaptureResponseExt_Checkpoint) 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 *CaptureResponseExt_Checkpoint) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CaptureResponseExt_Checkpoint) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if m.LogLevel != 0 { - n += 1 + sovRuntime(uint64(m.LogLevel)) - } - if m.RocksdbDescriptor != nil { - l = m.RocksdbDescriptor.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - if m.StartCommit != nil { - l = m.StartCommit.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.PollResult != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.PollResult)) + i-- + dAtA[i] = 0x10 } - if m.XXX_unrecognized != nil { - n += len(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 n + return len(dAtA) - i, nil } -func (m *CaptureRequestExt_StartCommit) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *DeriveRequestExt) 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 *DeriveRequestExt) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DeriveRequestExt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if m.RuntimeCheckpoint != nil { - l = m.RuntimeCheckpoint.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) - } - return n -} - -func (m *CaptureResponseExt) ProtoSize() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Container != nil { - l = m.Container.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } - if m.Opened != nil { - l = m.Opened.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - if m.Captured != nil { - l = m.Captured.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.Open != nil { + { + size, err := m.Open.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a } - if m.Checkpoint != nil { - l = m.Checkpoint.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.RocksdbDescriptor != nil { + { + size, err := m.RocksdbDescriptor.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + if m.LogLevel != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.LogLevel)) + i-- + dAtA[i] = 0x8 } - return n + return len(dAtA) - i, nil } -func (m *CaptureResponseExt_Opened) ProtoSize() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.RuntimeCheckpoint != nil { - l = m.RuntimeCheckpoint.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) +func (m *DeriveRequestExt_Open) 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 n + return dAtA[:n], nil } -func (m *CaptureResponseExt_Captured) ProtoSize() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.KeyPacked) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - l = len(m.PartitionsPacked) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) - } - return n +func (m *DeriveRequestExt_Open) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *CaptureResponseExt_Checkpoint) ProtoSize() (n int) { - if m == nil { - return 0 - } +func (m *DeriveRequestExt_Open) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if m.Stats != nil { - 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) + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - return n + if len(m.SqliteVfsUri) > 0 { + i -= len(m.SqliteVfsUri) + copy(dAtA[i:], m.SqliteVfsUri) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.SqliteVfsUri))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil } -func (m *DeriveRequestExt) ProtoSize() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.LogLevel != 0 { - n += 1 + sovRuntime(uint64(m.LogLevel)) - } - if m.RocksdbDescriptor != nil { - l = m.RocksdbDescriptor.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } - if m.Open != nil { - l = m.Open.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) +func (m *DeriveResponseExt) 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 n + return dAtA[:n], nil } -func (m *DeriveRequestExt_Open) ProtoSize() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.SqliteVfsUri) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) - } - return n +func (m *DeriveResponseExt) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *DeriveResponseExt) ProtoSize() (n int) { - if m == nil { - return 0 - } +func (m *DeriveResponseExt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if m.Container != nil { - l = m.Container.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - if m.Opened != nil { - l = m.Opened.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + 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] = 0x22 } if m.Published != nil { - l = m.Published.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + { + size, err := m.Published.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a } - if m.Flushed != nil { - l = m.Flushed.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.Opened != nil { + { + size, err := m.Opened.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + if m.Container != nil { + { + size, err := m.Container.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa } - return n + return len(dAtA) - i, nil } -func (m *DeriveResponseExt_Opened) ProtoSize() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.RuntimeCheckpoint != nil { - l = m.RuntimeCheckpoint.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) +func (m *DeriveResponseExt_Opened) 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 n + return dAtA[:n], nil } -func (m *DeriveResponseExt_Published) ProtoSize() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.MaxClock != 0 { - n += 9 - } - l = len(m.KeyPacked) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - l = len(m.PartitionsPacked) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) - } - return n +func (m *DeriveResponseExt_Opened) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *DeriveResponseExt_Flushed) ProtoSize() (n int) { - if m == nil { - return 0 - } +func (m *DeriveResponseExt_Opened) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i 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) + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - return n + if m.RuntimeCheckpoint != nil { + { + size, err := m.RuntimeCheckpoint.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 (m *MaterializeRequestExt) ProtoSize() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.LogLevel != 0 { - n += 1 + sovRuntime(uint64(m.LogLevel)) - } - if m.RocksdbDescriptor != nil { - l = m.RocksdbDescriptor.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) +func (m *DeriveResponseExt_Published) 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 n + return dAtA[:n], nil } -func (m *MaterializeResponseExt) ProtoSize() (n int) { - if m == nil { - return 0 - } +func (m *DeriveResponseExt_Published) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DeriveResponseExt_Published) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if m.Container != nil { - 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) + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - return n -} - -func (m *MaterializeResponseExt_Flushed) ProtoSize() (n int) { - if m == nil { - return 0 + if len(m.PartitionsPacked) > 0 { + i -= len(m.PartitionsPacked) + copy(dAtA[i:], m.PartitionsPacked) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.PartitionsPacked))) + i-- + dAtA[i] = 0x1a } - var l int - _ = l - if m.Stats != nil { - l = m.Stats.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if len(m.KeyPacked) > 0 { + i -= len(m.KeyPacked) + copy(dAtA[i:], m.KeyPacked) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.KeyPacked))) + i-- + dAtA[i] = 0x12 } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + if m.MaxClock != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.MaxClock)) + i-- + dAtA[i] = 0x9 } - return n + return len(dAtA) - i, nil } -func (m *CombineRequest) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *DeriveResponseExt_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 *DeriveResponseExt_Flushed) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DeriveResponseExt_Flushed) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if m.Open != nil { - l = m.Open.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } - if m.Add != nil { - l = m.Add.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - return n + 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 (m *CombineRequest_Open) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *MaterializeRequestExt) 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 *MaterializeRequestExt) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *MaterializeRequestExt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if len(m.Bindings) > 0 { - for _, e := range m.Bindings { - l = e.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.RocksdbDescriptor != nil { + { + size, err := m.RocksdbDescriptor.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) } + i-- + dAtA[i] = 0x12 } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + if m.LogLevel != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.LogLevel)) + i-- + dAtA[i] = 0x8 } - return n + return len(dAtA) - i, nil } -func (m *CombineRequest_Open_Binding) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *MaterializeResponseExt) 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) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *MaterializeResponseExt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if m.Full { - n += 2 - } - if len(m.Key) > 0 { - for _, s := range m.Key { - l = len(s) - n += 1 + l + sovRuntime(uint64(l)) - } + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - if len(m.Projections) > 0 { - for _, e := range m.Projections { - l = e.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } - } - l = len(m.SchemaJson) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if m.SerPolicy != nil { - l = m.SerPolicy.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) - } - l = len(m.UuidPtr) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if len(m.Values) > 0 { - for _, s := range m.Values { - l = len(s) - n += 1 + l + sovRuntime(uint64(l)) + 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.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + if m.Container != nil { + { + size, err := m.Container.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa } - return n + return len(dAtA) - i, nil } -func (m *CombineRequest_Add) ProtoSize() (n int) { - if m == nil { - return 0 +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.Binding != 0 { - n += 1 + sovRuntime(uint64(m.Binding)) - } - l = len(m.DocJson) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if m.Front { - n += 2 - } if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - return n + 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 (m *CombineResponse) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *CombineRequest) 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 *CombineRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CombineRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - if m.Binding != 0 { - n += 1 + sovRuntime(uint64(m.Binding)) - } - if m.Deleted { - n += 2 - } - l = len(m.DocJson) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } - if m.Front { - n += 2 - } - l = len(m.KeyPacked) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - l = len(m.ValuesPacked) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) + if m.Add != nil { + { + size, err := m.Add.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + if m.Open != nil { + { + size, err := m.Open.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa } - return n + return len(dAtA) - i, nil } -func (m *ConnectorProxyRequest) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *CombineRequest_Open) 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 *CombineRequest_Open) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CombineRequest_Open) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - return n + if len(m.Bindings) > 0 { + for iNdEx := len(m.Bindings) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Bindings[iNdEx].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 (m *ConnectorProxyResponse) ProtoSize() (n int) { - if m == nil { - return 0 +func (m *CombineRequest_Open_Binding) 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 *CombineRequest_Open_Binding) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CombineRequest_Open_Binding) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i var l int _ = l - l = len(m.Address) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) } - l = len(m.ProxyId) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) + if len(m.Values) > 0 { + for iNdEx := len(m.Values) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Values[iNdEx]) + copy(dAtA[i:], m.Values[iNdEx]) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Values[iNdEx]))) + i-- + dAtA[i] = 0x3a + } } - if m.Log != nil { - l = m.Log.ProtoSize() - n += 1 + l + sovRuntime(uint64(l)) + if len(m.UuidPtr) > 0 { + i -= len(m.UuidPtr) + copy(dAtA[i:], m.UuidPtr) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.UuidPtr))) + i-- + dAtA[i] = 0x32 } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) + if m.SerPolicy != nil { + { + size, err := m.SerPolicy.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a } - return n + if len(m.SchemaJson) > 0 { + i -= len(m.SchemaJson) + copy(dAtA[i:], m.SchemaJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.SchemaJson))) + i-- + dAtA[i] = 0x22 + } + if len(m.Projections) > 0 { + for iNdEx := len(m.Projections) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Projections[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if len(m.Key) > 0 { + for iNdEx := len(m.Key) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Key[iNdEx]) + copy(dAtA[i:], m.Key[iNdEx]) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Key[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if m.Full { + i-- + if m.Full { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *CombineRequest_Add) 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 *CombineRequest_Add) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CombineRequest_Add) 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.Front { + i-- + if m.Front { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x18 + } + if len(m.DocJson) > 0 { + i -= len(m.DocJson) + copy(dAtA[i:], m.DocJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.DocJson))) + i-- + dAtA[i] = 0x12 + } + if m.Binding != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.Binding)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *CombineResponse) 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 *CombineResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CombineResponse) 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 len(m.ValuesPacked) > 0 { + i -= len(m.ValuesPacked) + copy(dAtA[i:], m.ValuesPacked) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ValuesPacked))) + i-- + dAtA[i] = 0x32 + } + if len(m.KeyPacked) > 0 { + i -= len(m.KeyPacked) + copy(dAtA[i:], m.KeyPacked) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.KeyPacked))) + i-- + dAtA[i] = 0x2a + } + if m.Front { + i-- + if m.Front { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x20 + } + if len(m.DocJson) > 0 { + i -= len(m.DocJson) + copy(dAtA[i:], m.DocJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.DocJson))) + i-- + dAtA[i] = 0x1a + } + if m.Deleted { + i-- + if m.Deleted { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } + if m.Binding != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.Binding)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ConnectorProxyRequest) 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 *ConnectorProxyRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ConnectorProxyRequest) 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) + } + return len(dAtA) - i, nil +} + +func (m *ConnectorProxyResponse) 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 *ConnectorProxyResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ConnectorProxyResponse) 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.Log != nil { + { + size, err := m.Log.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if len(m.ProxyId) > 0 { + i -= len(m.ProxyId) + copy(dAtA[i:], m.ProxyId) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ProxyId))) + i-- + dAtA[i] = 0x12 + } + if len(m.Address) > 0 { + i -= len(m.Address) + copy(dAtA[i:], m.Address) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Address))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Join) 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 *Join) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Join) 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 len(m.LeaderEndpoint) > 0 { + i -= len(m.LeaderEndpoint) + copy(dAtA[i:], m.LeaderEndpoint) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.LeaderEndpoint))) + i-- + dAtA[i] = 0x32 + } + if len(m.ShuffleEndpoint) > 0 { + i -= len(m.ShuffleEndpoint) + copy(dAtA[i:], m.ShuffleEndpoint) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ShuffleEndpoint))) + i-- + dAtA[i] = 0x2a + } + if len(m.ShuffleDirectory) > 0 { + i -= len(m.ShuffleDirectory) + copy(dAtA[i:], m.ShuffleDirectory) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ShuffleDirectory))) + i-- + dAtA[i] = 0x22 + } + if m.ShardIndex != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.ShardIndex)) + i-- + dAtA[i] = 0x18 + } + if len(m.Shards) > 0 { + for iNdEx := len(m.Shards) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Shards[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if m.EtcdModRevision != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.EtcdModRevision)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Join_Shard) 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 *Join_Shard) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Join_Shard) 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.EtcdCreateRevision != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.EtcdCreateRevision)) + i-- + dAtA[i] = 0x20 + } + if m.Reactor != nil { + { + size, err := m.Reactor.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.Labeling != nil { + { + size, err := m.Labeling.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if len(m.Id) > 0 { + i -= len(m.Id) + copy(dAtA[i:], m.Id) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Id))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Joined) 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 *Joined) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Joined) 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.MaxEtcdRevision != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.MaxEtcdRevision)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Task) 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 *Task) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Task) 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.MaxTransactions != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.MaxTransactions)) + i-- + dAtA[i] = 0x28 + } + if m.Preview { + i-- + if m.Preview { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x20 + } + if m.OpsStatsSpec != nil { + { + size, err := m.OpsStatsSpec.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if len(m.OpsStatsJournal) > 0 { + i -= len(m.OpsStatsJournal) + copy(dAtA[i:], m.OpsStatsJournal) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.OpsStatsJournal))) + i-- + dAtA[i] = 0x12 + } + if len(m.Spec) > 0 { + i -= len(m.Spec) + copy(dAtA[i:], m.Spec) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Spec))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Recover) 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 *Recover) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Recover) 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 len(m.TriggerParamsJson) > 0 { + i -= len(m.TriggerParamsJson) + copy(dAtA[i:], m.TriggerParamsJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.TriggerParamsJson))) + i-- + dAtA[i] = 0x52 + } + if len(m.MaxKeys) > 0 { + for k := range m.MaxKeys { + v := m.MaxKeys[k] + baseI := i + if len(v) > 0 { + i -= len(v) + copy(dAtA[i:], v) + i = encodeVarintRuntime(dAtA, i, uint64(len(v))) + i-- + dAtA[i] = 0x12 + } + i = encodeVarintRuntime(dAtA, i, uint64(k)) + i-- + dAtA[i] = 0x8 + i = encodeVarintRuntime(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0x4a + } + } + if m.LegacyCheckpoint != nil { + { + size, err := m.LegacyCheckpoint.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + if len(m.LastApplied) > 0 { + i -= len(m.LastApplied) + copy(dAtA[i:], m.LastApplied) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.LastApplied))) + i-- + dAtA[i] = 0x3a + } + if m.HintedFrontier != nil { + { + size, err := m.HintedFrontier.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + if m.HintedCloseClock != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.HintedCloseClock)) + i-- + dAtA[i] = 0x29 + } + if len(m.ConnectorStateJson) > 0 { + i -= len(m.ConnectorStateJson) + copy(dAtA[i:], m.ConnectorStateJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ConnectorStateJson))) + i-- + dAtA[i] = 0x22 + } + if m.CommittedFrontier != nil { + { + size, err := m.CommittedFrontier.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.CommittedCloseClock != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.CommittedCloseClock)) + i-- + dAtA[i] = 0x11 + } + if len(m.AckIntents) > 0 { + for k := range m.AckIntents { + v := m.AckIntents[k] + baseI := i + if len(v) > 0 { + i -= len(v) + copy(dAtA[i:], v) + i = encodeVarintRuntime(dAtA, i, uint64(len(v))) + i-- + dAtA[i] = 0x12 + } + i -= len(k) + copy(dAtA[i:], k) + i = encodeVarintRuntime(dAtA, i, uint64(len(k))) + i-- + dAtA[i] = 0xa + i = encodeVarintRuntime(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *Persist) 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 *Persist) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Persist) 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 len(m.TriggerParamsJson) > 0 { + i -= len(m.TriggerParamsJson) + copy(dAtA[i:], m.TriggerParamsJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.TriggerParamsJson))) + i-- + dAtA[i] = 0x72 + } + if m.DeleteTriggerParams { + i-- + if m.DeleteTriggerParams { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x68 + } + if len(m.MaxKeys) > 0 { + for k := range m.MaxKeys { + v := m.MaxKeys[k] + baseI := i + if len(v) > 0 { + i -= len(v) + copy(dAtA[i:], v) + i = encodeVarintRuntime(dAtA, i, uint64(len(v))) + i-- + dAtA[i] = 0x12 + } + i = encodeVarintRuntime(dAtA, i, uint64(k)) + i-- + dAtA[i] = 0x8 + i = encodeVarintRuntime(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0x62 + } + } + if m.LegacyCheckpoint != nil { + { + size, err := m.LegacyCheckpoint.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x5a + } + if len(m.LastApplied) > 0 { + i -= len(m.LastApplied) + copy(dAtA[i:], m.LastApplied) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.LastApplied))) + i-- + dAtA[i] = 0x52 + } + if m.HintedFrontier != nil { + { + size, err := m.HintedFrontier.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } + if m.DeleteHintedFrontier { + i-- + if m.DeleteHintedFrontier { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x40 + } + if m.HintedCloseClock != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.HintedCloseClock)) + i-- + dAtA[i] = 0x39 + } + if len(m.ConnectorPatchesJson) > 0 { + i -= len(m.ConnectorPatchesJson) + copy(dAtA[i:], m.ConnectorPatchesJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ConnectorPatchesJson))) + i-- + dAtA[i] = 0x32 + } + if m.CommittedFrontier != nil { + { + size, err := m.CommittedFrontier.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + if m.CommittedCloseClock != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.CommittedCloseClock)) + i-- + dAtA[i] = 0x21 + } + if len(m.AckIntents) > 0 { + for k := range m.AckIntents { + v := m.AckIntents[k] + baseI := i + if len(v) > 0 { + i -= len(v) + copy(dAtA[i:], v) + i = encodeVarintRuntime(dAtA, i, uint64(len(v))) + i-- + dAtA[i] = 0x12 + } + i -= len(k) + copy(dAtA[i:], k) + i = encodeVarintRuntime(dAtA, i, uint64(len(k))) + i-- + dAtA[i] = 0xa + i = encodeVarintRuntime(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0x1a + } + } + if m.DeleteAckIntents { + i-- + if m.DeleteAckIntents { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } + if m.Nonce != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.Nonce)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Persisted) 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 *Persisted) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Persisted) 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.Nonce != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.Nonce)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Apply) 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 *Apply) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Apply) 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 len(m.ConnectorStateJson) > 0 { + i -= len(m.ConnectorStateJson) + copy(dAtA[i:], m.ConnectorStateJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ConnectorStateJson))) + i-- + dAtA[i] = 0x32 + } + if len(m.LastVersion) > 0 { + i -= len(m.LastVersion) + copy(dAtA[i:], m.LastVersion) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.LastVersion))) + i-- + dAtA[i] = 0x2a + } + if len(m.LastSpec) > 0 { + i -= len(m.LastSpec) + copy(dAtA[i:], m.LastSpec) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.LastSpec))) + i-- + dAtA[i] = 0x22 + } + if len(m.Version) > 0 { + i -= len(m.Version) + copy(dAtA[i:], m.Version) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Version))) + i-- + dAtA[i] = 0x12 + } + if len(m.Spec) > 0 { + i -= len(m.Spec) + copy(dAtA[i:], m.Spec) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Spec))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Applied) 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 *Applied) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Applied) 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 len(m.ConnectorPatchesJson) > 0 { + i -= len(m.ConnectorPatchesJson) + copy(dAtA[i:], m.ConnectorPatchesJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ConnectorPatchesJson))) + i-- + dAtA[i] = 0x12 + } + if len(m.ActionDescription) > 0 { + i -= len(m.ActionDescription) + copy(dAtA[i:], m.ActionDescription) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ActionDescription))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Open) 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 *Open) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Open) 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 len(m.MaxKeys) > 0 { + for k := range m.MaxKeys { + v := m.MaxKeys[k] + baseI := i + if len(v) > 0 { + i -= len(v) + copy(dAtA[i:], v) + i = encodeVarintRuntime(dAtA, i, uint64(len(v))) + i-- + dAtA[i] = 0x12 + } + i = encodeVarintRuntime(dAtA, i, uint64(k)) + i-- + dAtA[i] = 0x8 + i = encodeVarintRuntime(dAtA, i, uint64(baseI-i)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.ConnectorStateJson) > 0 { + i -= len(m.ConnectorStateJson) + copy(dAtA[i:], m.ConnectorStateJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ConnectorStateJson))) + i-- + dAtA[i] = 0x22 + } + if m.Range != nil { + { + size, err := m.Range.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if len(m.Version) > 0 { + i -= len(m.Version) + copy(dAtA[i:], m.Version) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Version))) + i-- + dAtA[i] = 0x12 + } + if len(m.Spec) > 0 { + i -= len(m.Spec) + copy(dAtA[i:], m.Spec) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.Spec))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CloseNow) 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 *CloseNow) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CloseNow) 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) + } + return len(dAtA) - i, nil +} + +func (m *Stop) 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 *Stop) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Stop) 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) + } + return len(dAtA) - i, nil +} + +func (m *Stopped) 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 *Stopped) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Stopped) 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) + } + return len(dAtA) - i, nil +} + +func (m *SessionLoop) 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 *SessionLoop) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SessionLoop) 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.RocksdbDescriptor != nil { + { + size, err := m.RocksdbDescriptor.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 (m *Materialize) 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 *Materialize) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize) 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.Stopped != nil { + { + size, err := m.Stopped.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xea + } + if m.Stop != nil { + { + size, err := m.Stop.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xe2 + } + if m.CloseNow != nil { + { + size, err := m.CloseNow.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0xa2 + } + if m.Persisted != nil { + { + size, err := m.Persisted.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x9a + } + if m.Persist != nil { + { + size, err := m.Persist.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x92 + } + if m.Acknowledged != nil { + { + size, err := m.Acknowledged.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x8a + } + if m.Acknowledge != nil { + { + size, err := m.Acknowledge.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3 + i-- + dAtA[i] = 0x82 + } + if m.StartedCommit != nil { + { + size, err := m.StartedCommit.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xfa + } + if m.StartCommit != nil { + { + size, err := m.StartCommit.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xf2 + } + if m.Stored != nil { + { + size, err := m.Stored.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xea + } + if m.Store != nil { + { + size, err := m.Store.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xe2 + } + 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] = 0x2 + i-- + dAtA[i] = 0xda + } + if m.Flush != nil { + { + size, err := m.Flush.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xd2 + } + if m.Loaded != nil { + { + size, err := m.Loaded.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xca + } + if m.Load != nil { + { + size, err := m.Load.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2 + i-- + dAtA[i] = 0xc2 + } + if m.Opened != nil { + { + size, err := m.Opened.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xe2 + } + if m.Open != nil { + { + size, err := m.Open.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xda + } + if m.Applied != nil { + { + size, err := m.Applied.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xd2 + } + if m.Apply != nil { + { + size, err := m.Apply.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xca + } + if m.Recover != nil { + { + size, err := m.Recover.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xc2 + } + if m.Task != nil { + { + size, err := m.Task.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xba + } + if m.Joined != nil { + { + size, err := m.Joined.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xb2 + } + if m.Join != nil { + { + size, err := m.Join.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xaa + } + if m.SessionLoop != nil { + { + size, err := m.SessionLoop.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xa2 + } + if m.Validated != nil { + { + size, err := m.Validated.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + if m.Validate != nil { + { + size, err := m.Validate.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.SpecResponse != nil { + { + size, err := m.SpecResponse.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.Spec != nil { + { + size, err := m.Spec.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 (m *Materialize_Opened) 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 *Materialize_Opened) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_Opened) 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.ConnectorCheckpoint != nil { + { + size, err := m.ConnectorCheckpoint.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]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Materialize_Load) 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 *Materialize_Load) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_Load) 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.Frontier != nil { + { + size, err := m.Frontier.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 (m *Materialize_Loaded) 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 *Materialize_Loaded) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_Loaded) 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.CombinerUsageBytes != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.CombinerUsageBytes)) + i-- + dAtA[i] = 0x10 + } + if len(m.Bindings) > 0 { + for iNdEx := len(m.Bindings) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Bindings[iNdEx].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 (m *Materialize_Loaded_Binding) 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 *Materialize_Loaded_Binding) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_Loaded_Binding) 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 len(m.MaxKeyDelta) > 0 { + i -= len(m.MaxKeyDelta) + copy(dAtA[i:], m.MaxKeyDelta) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.MaxKeyDelta))) + i-- + dAtA[i] = 0x32 + } + if m.SourcedBytesTotal != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.SourcedBytesTotal)) + i-- + dAtA[i] = 0x28 + } + if m.SourcedDocsTotal != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.SourcedDocsTotal)) + i-- + dAtA[i] = 0x20 + } + if m.MaxSourceClock != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.MaxSourceClock)) + i-- + dAtA[i] = 0x19 + } + if m.MinSourceClock != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.MinSourceClock)) + i-- + dAtA[i] = 0x11 + } + if m.Index != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.Index)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Materialize_Flush) 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 *Materialize_Flush) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_Flush) 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 len(m.ConnectorPatchesJson) > 0 { + i -= len(m.ConnectorPatchesJson) + copy(dAtA[i:], m.ConnectorPatchesJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ConnectorPatchesJson))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Materialize_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 *Materialize_Flushed) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_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 len(m.ConnectorPatchesJson) > 0 { + i -= len(m.ConnectorPatchesJson) + copy(dAtA[i:], m.ConnectorPatchesJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ConnectorPatchesJson))) + i-- + dAtA[i] = 0x12 + } + if len(m.Bindings) > 0 { + for iNdEx := len(m.Bindings) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Bindings[iNdEx].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 (m *Materialize_Flushed_Binding) 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 *Materialize_Flushed_Binding) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_Flushed_Binding) 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.LoadedBytesTotal != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.LoadedBytesTotal)) + i-- + dAtA[i] = 0x18 + } + if m.LoadedDocsTotal != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.LoadedDocsTotal)) + i-- + dAtA[i] = 0x10 + } + if m.Index != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.Index)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Materialize_Store) 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 *Materialize_Store) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_Store) 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) + } + return len(dAtA) - i, nil +} + +func (m *Materialize_Stored) 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 *Materialize_Stored) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_Stored) 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 len(m.Bindings) > 0 { + for iNdEx := len(m.Bindings) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Bindings[iNdEx].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 (m *Materialize_Stored_Binding) 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 *Materialize_Stored_Binding) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_Stored_Binding) 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.StoredBytesTotal != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.StoredBytesTotal)) + i-- + dAtA[i] = 0x18 + } + if m.StoredDocsTotal != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.StoredDocsTotal)) + i-- + dAtA[i] = 0x10 + } + if m.Index != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.Index)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Materialize_StartCommit) 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 *Materialize_StartCommit) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_StartCommit) 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.ConnectorCheckpoint != nil { + { + size, err := m.ConnectorCheckpoint.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRuntime(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if len(m.ConnectorPatchesJson) > 0 { + i -= len(m.ConnectorPatchesJson) + copy(dAtA[i:], m.ConnectorPatchesJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ConnectorPatchesJson))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Materialize_StartedCommit) 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 *Materialize_StartedCommit) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_StartedCommit) 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 len(m.ConnectorPatchesJson) > 0 { + i -= len(m.ConnectorPatchesJson) + copy(dAtA[i:], m.ConnectorPatchesJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ConnectorPatchesJson))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Materialize_Acknowledge) 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 *Materialize_Acknowledge) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_Acknowledge) 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 len(m.ConnectorPatchesJson) > 0 { + i -= len(m.ConnectorPatchesJson) + copy(dAtA[i:], m.ConnectorPatchesJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ConnectorPatchesJson))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Materialize_Acknowledged) 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 *Materialize_Acknowledged) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Materialize_Acknowledged) 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 len(m.ConnectorPatchesJson) > 0 { + i -= len(m.ConnectorPatchesJson) + copy(dAtA[i:], m.ConnectorPatchesJson) + i = encodeVarintRuntime(dAtA, i, uint64(len(m.ConnectorPatchesJson))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func encodeVarintRuntime(dAtA []byte, offset int, v uint64) int { + offset -= sovRuntime(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *TaskServiceConfig) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.LogFileFd != 0 { + n += 1 + sovRuntime(uint64(m.LogFileFd)) + } + l = len(m.TaskName) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.UdsPath) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.ContainerNetwork) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Plane != 0 { + n += 1 + sovRuntime(uint64(m.Plane)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *ShuffleRequest) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Journal) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Replay { + n += 2 + } + l = len(m.BuildId) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Offset != 0 { + n += 1 + sovRuntime(uint64(m.Offset)) + } + if m.EndOffset != 0 { + n += 1 + sovRuntime(uint64(m.EndOffset)) + } + l = m.Range.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + l = len(m.Coordinator) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Resolution != nil { + l = m.Resolution.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.ShuffleIndex != 0 { + n += 1 + sovRuntime(uint64(m.ShuffleIndex)) + } + if m.Derivation != nil { + l = m.Derivation.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Materialization != nil { + l = m.Materialization.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *ShuffleResponse) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Status != 0 { + n += 1 + sovRuntime(uint64(m.Status)) + } + if m.Header != nil { + l = m.Header.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.TerminalError) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.ReadThrough != 0 { + n += 1 + sovRuntime(uint64(m.ReadThrough)) + } + if m.WriteHead != 0 { + n += 1 + sovRuntime(uint64(m.WriteHead)) + } + l = len(m.Arena) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if len(m.Docs) > 0 { + for _, e := range m.Docs { + l = e.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + } + if len(m.Offsets) > 0 { + l = 0 + for _, e := range m.Offsets { + l += sovRuntime(uint64(e)) + } + n += 1 + sovRuntime(uint64(l)) + l + } + if len(m.UuidParts) > 0 { + for _, e := range m.UuidParts { + l = e.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + } + if len(m.PackedKey) > 0 { + for _, e := range m.PackedKey { + l = e.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *RocksDBDescriptor) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RocksdbEnvMemptr != 0 { + n += 9 + } + l = len(m.RocksdbPath) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Container) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.IpAddr) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if len(m.NetworkPorts) > 0 { + for _, e := range m.NetworkPorts { + l = e.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + } + if len(m.MappedHostPorts) > 0 { + for k, v := range m.MappedHostPorts { + _ = k + _ = v + mapEntrySize := 1 + sovRuntime(uint64(k)) + 1 + len(v) + sovRuntime(uint64(len(v))) + n += mapEntrySize + 1 + sovRuntime(uint64(mapEntrySize)) + } + } + if m.UsageRate != 0 { + n += 5 + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CaptureRequestExt) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.LogLevel != 0 { + n += 1 + sovRuntime(uint64(m.LogLevel)) + } + if m.RocksdbDescriptor != nil { + l = m.RocksdbDescriptor.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.StartCommit != nil { + l = m.StartCommit.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CaptureRequestExt_StartCommit) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RuntimeCheckpoint != nil { + l = m.RuntimeCheckpoint.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CaptureResponseExt) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Container != nil { + l = m.Container.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Opened != nil { + l = m.Opened.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Captured != nil { + l = m.Captured.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Checkpoint != nil { + l = m.Checkpoint.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CaptureResponseExt_Opened) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RuntimeCheckpoint != nil { + l = m.RuntimeCheckpoint.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CaptureResponseExt_Captured) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.KeyPacked) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.PartitionsPacked) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CaptureResponseExt_Checkpoint) 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.PollResult != 0 { + n += 1 + sovRuntime(uint64(m.PollResult)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *DeriveRequestExt) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.LogLevel != 0 { + n += 1 + sovRuntime(uint64(m.LogLevel)) + } + if m.RocksdbDescriptor != nil { + l = m.RocksdbDescriptor.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Open != nil { + l = m.Open.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *DeriveRequestExt_Open) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.SqliteVfsUri) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *DeriveResponseExt) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Container != nil { + l = m.Container.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Opened != nil { + l = m.Opened.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Published != nil { + l = m.Published.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 *DeriveResponseExt_Opened) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RuntimeCheckpoint != nil { + l = m.RuntimeCheckpoint.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *DeriveResponseExt_Published) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MaxClock != 0 { + n += 9 + } + l = len(m.KeyPacked) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.PartitionsPacked) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *DeriveResponseExt_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) + } + return n +} + +func (m *MaterializeRequestExt) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.LogLevel != 0 { + n += 1 + sovRuntime(uint64(m.LogLevel)) + } + if m.RocksdbDescriptor != nil { + l = m.RocksdbDescriptor.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *MaterializeResponseExt) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Container != nil { + 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) + } + return n +} + +func (m *CombineRequest) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Open != nil { + l = m.Open.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Add != nil { + l = m.Add.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CombineRequest_Open) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Bindings) > 0 { + for _, e := range m.Bindings { + l = e.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CombineRequest_Open_Binding) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Full { + n += 2 + } + if len(m.Key) > 0 { + for _, s := range m.Key { + l = len(s) + n += 1 + l + sovRuntime(uint64(l)) + } + } + if len(m.Projections) > 0 { + for _, e := range m.Projections { + l = e.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + } + l = len(m.SchemaJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.SerPolicy != nil { + l = m.SerPolicy.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.UuidPtr) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if len(m.Values) > 0 { + for _, s := range m.Values { + l = len(s) + n += 1 + l + sovRuntime(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CombineRequest_Add) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Binding != 0 { + n += 1 + sovRuntime(uint64(m.Binding)) + } + l = len(m.DocJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Front { + n += 2 + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CombineResponse) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Binding != 0 { + n += 1 + sovRuntime(uint64(m.Binding)) + } + if m.Deleted { + n += 2 + } + l = len(m.DocJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Front { + n += 2 + } + l = len(m.KeyPacked) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.ValuesPacked) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *ConnectorProxyRequest) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *ConnectorProxyResponse) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Address) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.ProxyId) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Log != nil { + l = m.Log.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Join) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EtcdModRevision != 0 { + n += 1 + sovRuntime(uint64(m.EtcdModRevision)) + } + if len(m.Shards) > 0 { + for _, e := range m.Shards { + l = e.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + } + if m.ShardIndex != 0 { + n += 1 + sovRuntime(uint64(m.ShardIndex)) + } + l = len(m.ShuffleDirectory) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.ShuffleEndpoint) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.LeaderEndpoint) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Join_Shard) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Id) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Labeling != nil { + l = m.Labeling.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Reactor != nil { + l = m.Reactor.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.EtcdCreateRevision != 0 { + n += 1 + sovRuntime(uint64(m.EtcdCreateRevision)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Joined) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MaxEtcdRevision != 0 { + n += 1 + sovRuntime(uint64(m.MaxEtcdRevision)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Task) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Spec) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.OpsStatsJournal) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.OpsStatsSpec != nil { + l = m.OpsStatsSpec.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Preview { + n += 2 + } + if m.MaxTransactions != 0 { + n += 1 + sovRuntime(uint64(m.MaxTransactions)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Recover) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.AckIntents) > 0 { + for k, v := range m.AckIntents { + _ = k + _ = v + l = 0 + if len(v) > 0 { + l = 1 + len(v) + sovRuntime(uint64(len(v))) + } + mapEntrySize := 1 + len(k) + sovRuntime(uint64(len(k))) + l + n += mapEntrySize + 1 + sovRuntime(uint64(mapEntrySize)) + } + } + if m.CommittedCloseClock != 0 { + n += 9 + } + if m.CommittedFrontier != nil { + l = m.CommittedFrontier.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.ConnectorStateJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.HintedCloseClock != 0 { + n += 9 + } + if m.HintedFrontier != nil { + l = m.HintedFrontier.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.LastApplied) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.LegacyCheckpoint != nil { + l = m.LegacyCheckpoint.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if len(m.MaxKeys) > 0 { + for k, v := range m.MaxKeys { + _ = k + _ = v + l = 0 + if len(v) > 0 { + l = 1 + len(v) + sovRuntime(uint64(len(v))) + } + mapEntrySize := 1 + sovRuntime(uint64(k)) + l + n += mapEntrySize + 1 + sovRuntime(uint64(mapEntrySize)) + } + } + l = len(m.TriggerParamsJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Persist) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Nonce != 0 { + n += 1 + sovRuntime(uint64(m.Nonce)) + } + if m.DeleteAckIntents { + n += 2 + } + if len(m.AckIntents) > 0 { + for k, v := range m.AckIntents { + _ = k + _ = v + l = 0 + if len(v) > 0 { + l = 1 + len(v) + sovRuntime(uint64(len(v))) + } + mapEntrySize := 1 + len(k) + sovRuntime(uint64(len(k))) + l + n += mapEntrySize + 1 + sovRuntime(uint64(mapEntrySize)) + } + } + if m.CommittedCloseClock != 0 { + n += 9 + } + if m.CommittedFrontier != nil { + l = m.CommittedFrontier.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.ConnectorPatchesJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.HintedCloseClock != 0 { + n += 9 + } + if m.DeleteHintedFrontier { + n += 2 + } + if m.HintedFrontier != nil { + l = m.HintedFrontier.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.LastApplied) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.LegacyCheckpoint != nil { + l = m.LegacyCheckpoint.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if len(m.MaxKeys) > 0 { + for k, v := range m.MaxKeys { + _ = k + _ = v + l = 0 + if len(v) > 0 { + l = 1 + len(v) + sovRuntime(uint64(len(v))) + } + mapEntrySize := 1 + sovRuntime(uint64(k)) + l + n += mapEntrySize + 1 + sovRuntime(uint64(mapEntrySize)) + } + } + if m.DeleteTriggerParams { + n += 2 + } + l = len(m.TriggerParamsJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Persisted) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Nonce != 0 { + n += 1 + sovRuntime(uint64(m.Nonce)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Apply) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Spec) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.Version) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.LastSpec) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.LastVersion) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.ConnectorStateJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Applied) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ActionDescription) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.ConnectorPatchesJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Open) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Spec) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.Version) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Range != nil { + l = m.Range.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + l = len(m.ConnectorStateJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if len(m.MaxKeys) > 0 { + for k, v := range m.MaxKeys { + _ = k + _ = v + l = 0 + if len(v) > 0 { + l = 1 + len(v) + sovRuntime(uint64(len(v))) + } + mapEntrySize := 1 + sovRuntime(uint64(k)) + l + n += mapEntrySize + 1 + sovRuntime(uint64(mapEntrySize)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CloseNow) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Stop) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Stopped) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SessionLoop) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RocksdbDescriptor != nil { + l = m.RocksdbDescriptor.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Spec != nil { + l = m.Spec.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.SpecResponse != nil { + l = m.SpecResponse.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Validate != nil { + l = m.Validate.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.Validated != nil { + l = m.Validated.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.SessionLoop != nil { + l = m.SessionLoop.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Join != nil { + l = m.Join.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Joined != nil { + l = m.Joined.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Task != nil { + l = m.Task.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Recover != nil { + l = m.Recover.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Apply != nil { + l = m.Apply.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Applied != nil { + l = m.Applied.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Open != nil { + l = m.Open.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Opened != nil { + l = m.Opened.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Load != nil { + l = m.Load.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Loaded != nil { + l = m.Loaded.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Flush != nil { + l = m.Flush.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Flushed != nil { + l = m.Flushed.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Store != nil { + l = m.Store.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Stored != nil { + l = m.Stored.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.StartCommit != nil { + l = m.StartCommit.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.StartedCommit != nil { + l = m.StartedCommit.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Acknowledge != nil { + l = m.Acknowledge.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Acknowledged != nil { + l = m.Acknowledged.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Persist != nil { + l = m.Persist.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Persisted != nil { + l = m.Persisted.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.CloseNow != nil { + l = m.CloseNow.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Stop != nil { + l = m.Stop.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.Stopped != nil { + l = m.Stopped.ProtoSize() + n += 2 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Opened) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Container != nil { + l = m.Container.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.ConnectorCheckpoint != nil { + l = m.ConnectorCheckpoint.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Load) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Frontier != nil { + l = m.Frontier.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Loaded) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Bindings) > 0 { + for _, e := range m.Bindings { + l = e.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + } + if m.CombinerUsageBytes != 0 { + n += 1 + sovRuntime(uint64(m.CombinerUsageBytes)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Loaded_Binding) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Index != 0 { + n += 1 + sovRuntime(uint64(m.Index)) + } + if m.MinSourceClock != 0 { + n += 9 + } + if m.MaxSourceClock != 0 { + n += 9 + } + if m.SourcedDocsTotal != 0 { + n += 1 + sovRuntime(uint64(m.SourcedDocsTotal)) + } + if m.SourcedBytesTotal != 0 { + n += 1 + sovRuntime(uint64(m.SourcedBytesTotal)) + } + l = len(m.MaxKeyDelta) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Flush) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ConnectorPatchesJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Flushed) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Bindings) > 0 { + for _, e := range m.Bindings { + l = e.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + } + l = len(m.ConnectorPatchesJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Flushed_Binding) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Index != 0 { + n += 1 + sovRuntime(uint64(m.Index)) + } + if m.LoadedDocsTotal != 0 { + n += 1 + sovRuntime(uint64(m.LoadedDocsTotal)) + } + if m.LoadedBytesTotal != 0 { + n += 1 + sovRuntime(uint64(m.LoadedBytesTotal)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Store) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Stored) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Bindings) > 0 { + for _, e := range m.Bindings { + l = e.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Stored_Binding) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Index != 0 { + n += 1 + sovRuntime(uint64(m.Index)) + } + if m.StoredDocsTotal != 0 { + n += 1 + sovRuntime(uint64(m.StoredDocsTotal)) + } + if m.StoredBytesTotal != 0 { + n += 1 + sovRuntime(uint64(m.StoredBytesTotal)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_StartCommit) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ConnectorPatchesJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.ConnectorCheckpoint != nil { + l = m.ConnectorCheckpoint.ProtoSize() + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_StartedCommit) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ConnectorPatchesJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Acknowledge) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ConnectorPatchesJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Materialize_Acknowledged) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ConnectorPatchesJson) + if l > 0 { + n += 1 + l + sovRuntime(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func sovRuntime(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozRuntime(x uint64) (n int) { + return sovRuntime(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *TaskServiceConfig) 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: TaskServiceConfig: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TaskServiceConfig: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LogFileFd", wireType) + } + m.LogFileFd = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LogFileFd |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TaskName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TaskName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UdsPath", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.UdsPath = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ContainerNetwork", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ContainerNetwork = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Plane", wireType) + } + m.Plane = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Plane |= Plane(b&0x7F) << shift + if b < 0x80 { + break + } + } + 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 *ShuffleRequest) 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: ShuffleRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ShuffleRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Journal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Journal = go_gazette_dev_core_broker_protocol.Journal(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Replay", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Replay = bool(v != 0) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BuildId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BuildId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Offset", wireType) + } + m.Offset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Offset |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EndOffset", wireType) + } + m.EndOffset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.EndOffset |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Range", 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 err := m.Range.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Coordinator", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Coordinator = go_gazette_dev_core_consumer_protocol.ShardID(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Resolution", 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.Resolution == nil { + m.Resolution = &protocol.Header{} + } + if err := m.Resolution.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ShuffleIndex", wireType) + } + m.ShuffleIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ShuffleIndex |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Derivation", 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.Derivation == nil { + m.Derivation = &flow.CollectionSpec{} + } + if err := m.Derivation.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Materialization", 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.Materialization == nil { + m.Materialization = &flow.MaterializationSpec{} + } + if err := m.Materialization.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 *ShuffleResponse) 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: ShuffleResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ShuffleResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + m.Status = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Status |= protocol1.Status(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Header", 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.Header == nil { + m.Header = &protocol.Header{} + } + if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TerminalError", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TerminalError = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ReadThrough", wireType) + } + m.ReadThrough = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ReadThrough |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WriteHead", wireType) + } + m.WriteHead = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.WriteHead |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Arena", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Arena = append(m.Arena[:0], dAtA[iNdEx:postIndex]...) + if m.Arena == nil { + m.Arena = []byte{} + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Docs", 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 + } + m.Docs = append(m.Docs, flow.Slice{}) + if err := m.Docs[len(m.Docs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType == 0 { + var v go_gazette_dev_core_broker_protocol.Offset + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Offsets = append(m.Offsets, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Offsets) == 0 { + m.Offsets = make([]go_gazette_dev_core_broker_protocol.Offset, 0, elementCount) + } + for iNdEx < postIndex { + var v go_gazette_dev_core_broker_protocol.Offset + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Offsets = append(m.Offsets, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Offsets", wireType) + } + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UuidParts", 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 + } + m.UuidParts = append(m.UuidParts, flow.UUIDParts{}) + if err := m.UuidParts[len(m.UuidParts)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PackedKey", 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 + } + m.PackedKey = append(m.PackedKey, flow.Slice{}) + if err := m.PackedKey[len(m.PackedKey)-1].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 *RocksDBDescriptor) 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: RocksDBDescriptor: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RocksDBDescriptor: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field RocksdbEnvMemptr", wireType) + } + m.RocksdbEnvMemptr = 0 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + m.RocksdbEnvMemptr = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RocksdbPath", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RocksdbPath = string(dAtA[iNdEx:postIndex]) + 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 *Container) 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: Container: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Container: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IpAddr", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.IpAddr = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NetworkPorts", 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 + } + m.NetworkPorts = append(m.NetworkPorts, &flow.NetworkPort{}) + if err := m.NetworkPorts[len(m.NetworkPorts)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MappedHostPorts", 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.MappedHostPorts == nil { + m.MappedHostPorts = make(map[uint32]string) + } + var mapkey uint32 + var mapvalue string + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + } else if fieldNum == 2 { + var stringLenmapvalue uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapvalue |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapvalue := int(stringLenmapvalue) + if intStringLenmapvalue < 0 { + return ErrInvalidLengthRuntime + } + postStringIndexmapvalue := iNdEx + intStringLenmapvalue + if postStringIndexmapvalue < 0 { + return ErrInvalidLengthRuntime + } + if postStringIndexmapvalue > l { + return io.ErrUnexpectedEOF + } + mapvalue = string(dAtA[iNdEx:postStringIndexmapvalue]) + iNdEx = postStringIndexmapvalue + } else { + iNdEx = entryPreIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthRuntime + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.MappedHostPorts[mapkey] = mapvalue + iNdEx = postIndex + case 4: + if wireType != 5 { + return fmt.Errorf("proto: wrong wireType = %d for field UsageRate", wireType) + } + var v uint32 + if (iNdEx + 4) > l { + return io.ErrUnexpectedEOF + } + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) + iNdEx += 4 + m.UsageRate = float32(math.Float32frombits(v)) + 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 *CaptureRequestExt) 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: CaptureRequestExt: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CaptureRequestExt: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LogLevel", wireType) + } + m.LogLevel = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LogLevel |= ops.Log_Level(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RocksdbDescriptor", 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.RocksdbDescriptor == nil { + m.RocksdbDescriptor = &RocksDBDescriptor{} + } + if err := m.RocksdbDescriptor.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StartCommit", 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.StartCommit == nil { + m.StartCommit = &CaptureRequestExt_StartCommit{} + } + if err := m.StartCommit.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 *CaptureRequestExt_StartCommit) 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: StartCommit: wiretype end group for non-group") + } + if fieldNum <= 0 { + 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 RuntimeCheckpoint", 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.RuntimeCheckpoint == nil { + m.RuntimeCheckpoint = &protocol1.Checkpoint{} + } + if err := m.RuntimeCheckpoint.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 *CaptureResponseExt) 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: CaptureResponseExt: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CaptureResponseExt: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Container", 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.Container == nil { + m.Container = &Container{} + } + if err := m.Container.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Opened", 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.Opened == nil { + m.Opened = &CaptureResponseExt_Opened{} + } + if err := m.Opened.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Captured", 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.Captured == nil { + m.Captured = &CaptureResponseExt_Captured{} + } + if err := m.Captured.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Checkpoint", 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.Checkpoint == nil { + m.Checkpoint = &CaptureResponseExt_Checkpoint{} + } + if err := m.Checkpoint.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 *CaptureResponseExt_Opened) 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: Opened: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Opened: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RuntimeCheckpoint", 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.RuntimeCheckpoint == nil { + m.RuntimeCheckpoint = &protocol1.Checkpoint{} + } + if err := m.RuntimeCheckpoint.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 *CaptureResponseExt_Captured) 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: Captured: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Captured: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KeyPacked", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.KeyPacked = append(m.KeyPacked[:0], dAtA[iNdEx:postIndex]...) + if m.KeyPacked == nil { + m.KeyPacked = []byte{} + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PartitionsPacked", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PartitionsPacked = append(m.PartitionsPacked[:0], dAtA[iNdEx:postIndex]...) + if m.PartitionsPacked == nil { + m.PartitionsPacked = []byte{} + } + 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 *CaptureResponseExt_Checkpoint) 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: Checkpoint: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Checkpoint: 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 + 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:]) + 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 *DeriveRequestExt) 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: DeriveRequestExt: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DeriveRequestExt: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LogLevel", wireType) + } + m.LogLevel = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LogLevel |= ops.Log_Level(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RocksdbDescriptor", 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.RocksdbDescriptor == nil { + m.RocksdbDescriptor = &RocksDBDescriptor{} + } + if err := m.RocksdbDescriptor.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Open", 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.Open == nil { + m.Open = &DeriveRequestExt_Open{} + } + if err := m.Open.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 *DeriveRequestExt_Open) 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: Open: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Open: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SqliteVfsUri", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SqliteVfsUri = string(dAtA[iNdEx:postIndex]) + 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 *DeriveResponseExt) 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: DeriveResponseExt: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DeriveResponseExt: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Container", 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.Container == nil { + m.Container = &Container{} + } + if err := m.Container.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Opened", 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.Opened == nil { + m.Opened = &DeriveResponseExt_Opened{} + } + if err := m.Opened.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Published", 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.Published == nil { + m.Published = &DeriveResponseExt_Published{} + } + if err := m.Published.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + 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 = &DeriveResponseExt_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 *DeriveResponseExt_Opened) 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: Opened: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Opened: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RuntimeCheckpoint", 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.RuntimeCheckpoint == nil { + m.RuntimeCheckpoint = &protocol1.Checkpoint{} + } + if err := m.RuntimeCheckpoint.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 *DeriveResponseExt_Published) 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: Published: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Published: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxClock", wireType) + } + m.MaxClock = 0 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + m.MaxClock = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KeyPacked", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.KeyPacked = append(m.KeyPacked[:0], dAtA[iNdEx:postIndex]...) + if m.KeyPacked == nil { + m.KeyPacked = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PartitionsPacked", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PartitionsPacked = append(m.PartitionsPacked[:0], dAtA[iNdEx:postIndex]...) + if m.PartitionsPacked == nil { + m.PartitionsPacked = []byte{} + } + 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 *DeriveResponseExt_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:]) + 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 *MaterializeRequestExt) 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: MaterializeRequestExt: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MaterializeRequestExt: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LogLevel", wireType) + } + m.LogLevel = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LogLevel |= ops.Log_Level(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RocksdbDescriptor", 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.RocksdbDescriptor == nil { + m.RocksdbDescriptor = &RocksDBDescriptor{} + } + if err := m.RocksdbDescriptor.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) 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: MaterializeResponseExt: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MaterializeResponseExt: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Container", 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.Container == nil { + m.Container = &Container{} + } + if err := m.Container.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + 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:]) + 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 *CombineRequest) 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: CombineRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CombineRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Open", 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.Open == nil { + m.Open = &CombineRequest_Open{} + } + if err := m.Open.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Add", 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.Add == nil { + m.Add = &CombineRequest_Add{} + } + if err := m.Add.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 *CombineRequest_Open) 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: Open: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Open: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Bindings", 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 + } + m.Bindings = append(m.Bindings, &CombineRequest_Open_Binding{}) + if err := m.Bindings[len(m.Bindings)-1].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 *CombineRequest_Open_Binding) 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: Binding: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Binding: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Full", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Full = bool(v != 0) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = append(m.Key, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Projections", 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 + } + m.Projections = append(m.Projections, flow.Projection{}) + if err := m.Projections[len(m.Projections)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SchemaJson", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SchemaJson = append(m.SchemaJson[:0], dAtA[iNdEx:postIndex]...) + if m.SchemaJson == nil { + m.SchemaJson = []byte{} + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SerPolicy", 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.SerPolicy == nil { + m.SerPolicy = &flow.SerPolicy{} + } + if err := m.SerPolicy.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field UuidPtr", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.UuidPtr = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Values", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Values = append(m.Values, string(dAtA[iNdEx:postIndex])) + 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 *CombineRequest_Add) 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: Add: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Add: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Binding", wireType) + } + m.Binding = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Binding |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DocJson", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DocJson = append(m.DocJson[:0], dAtA[iNdEx:postIndex]...) + if m.DocJson == nil { + m.DocJson = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Front", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Front = bool(v != 0) + 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 *CombineResponse) 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: CombineResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CombineResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Binding", wireType) + } + m.Binding = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Binding |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Deleted", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Deleted = bool(v != 0) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DocJson", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DocJson = append(m.DocJson[:0], dAtA[iNdEx:postIndex]...) + if m.DocJson == nil { + m.DocJson = []byte{} + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Front", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Front = bool(v != 0) + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KeyPacked", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.KeyPacked = append(m.KeyPacked[:0], dAtA[iNdEx:postIndex]...) + if m.KeyPacked == nil { + m.KeyPacked = []byte{} + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValuesPacked", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ValuesPacked = append(m.ValuesPacked[:0], dAtA[iNdEx:postIndex]...) + if m.ValuesPacked == nil { + m.ValuesPacked = []byte{} + } + 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 *ConnectorProxyRequest) 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: ConnectorProxyRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ConnectorProxyRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + 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 *ConnectorProxyResponse) 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: ConnectorProxyResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ConnectorProxyResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Address", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Address = go_gazette_dev_core_broker_protocol.Endpoint(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ProxyId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ProxyId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Log", 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.Log == nil { + m.Log = &ops.Log{} + } + if err := m.Log.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 *Join) 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: Join: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Join: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EtcdModRevision", wireType) + } + m.EtcdModRevision = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.EtcdModRevision |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shards", 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 + } + m.Shards = append(m.Shards, &Join_Shard{}) + if err := m.Shards[len(m.Shards)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ShardIndex", wireType) + } + m.ShardIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ShardIndex |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ShuffleDirectory", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ShuffleDirectory = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ShuffleEndpoint", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ShuffleEndpoint = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LeaderEndpoint", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LeaderEndpoint = string(dAtA[iNdEx:postIndex]) + 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 *Join_Shard) 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: Shard: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Shard: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Id = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labeling", 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.Labeling == nil { + m.Labeling = &ops.ShardLabeling{} + } + if err := m.Labeling.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Reactor", 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.Reactor == nil { + m.Reactor = &protocol.ProcessSpec_ID{} + } + if err := m.Reactor.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EtcdCreateRevision", wireType) + } + m.EtcdCreateRevision = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.EtcdCreateRevision |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + 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 *Joined) 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: Joined: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Joined: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxEtcdRevision", wireType) + } + m.MaxEtcdRevision = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxEtcdRevision |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + 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 *Task) 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: Task: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Task: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Spec", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Spec = append(m.Spec[:0], dAtA[iNdEx:postIndex]...) + if m.Spec == nil { + m.Spec = []byte{} + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OpsStatsJournal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OpsStatsJournal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OpsStatsSpec", 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.OpsStatsSpec == nil { + m.OpsStatsSpec = &flow.CollectionSpec{} + } + if err := m.OpsStatsSpec.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Preview", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Preview = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxTransactions", wireType) + } + m.MaxTransactions = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxTransactions |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + 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 + } + } -func sovRuntime(x uint64) (n int) { - return (math_bits.Len64(x|1) + 6) / 7 -} -func sozRuntime(x uint64) (n int) { - return sovRuntime(uint64((x << 1) ^ uint64((int64(x) >> 63)))) + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil } -func (m *TaskServiceConfig) Unmarshal(dAtA []byte) error { +func (m *Recover) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -4047,30 +13287,204 @@ func (m *TaskServiceConfig) Unmarshal(dAtA []byte) error { if shift >= 64 { return ErrIntOverflowRuntime } - if iNdEx >= l { + 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: Recover: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Recover: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AckIntents", 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.AckIntents == nil { + m.AckIntents = make(map[string][]byte) + } + var mapkey string + mapvalue := []byte{} + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLengthRuntime + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLengthRuntime + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapbyteLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapbyteLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intMapbyteLen := int(mapbyteLen) + if intMapbyteLen < 0 { + return ErrInvalidLengthRuntime + } + postbytesIndex := iNdEx + intMapbyteLen + if postbytesIndex < 0 { + return ErrInvalidLengthRuntime + } + if postbytesIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = make([]byte, mapbyteLen) + copy(mapvalue, dAtA[iNdEx:postbytesIndex]) + iNdEx = postbytesIndex + } else { + iNdEx = entryPreIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthRuntime + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.AckIntents[mapkey] = mapvalue + iNdEx = postIndex + case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field CommittedCloseClock", wireType) + } + m.CommittedCloseClock = 0 + if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + m.CommittedCloseClock = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommittedFrontier", wireType) } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: TaskServiceConfig: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: TaskServiceConfig: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LogFileFd", 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 + } } - m.LogFileFd = 0 + if msglen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CommittedFrontier == nil { + m.CommittedFrontier = &shuffle.Frontier{} + } + if err := m.CommittedFrontier.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorStateJson", wireType) + } + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4080,16 +13494,41 @@ func (m *TaskServiceConfig) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.LogFileFd |= int32(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 2: + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ConnectorStateJson = append(m.ConnectorStateJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorStateJson == nil { + m.ConnectorStateJson = []byte{} + } + iNdEx = postIndex + case 5: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field HintedCloseClock", wireType) + } + m.HintedCloseClock = 0 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + m.HintedCloseClock = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TaskName", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field HintedFrontier", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4099,29 +13538,33 @@ func (m *TaskServiceConfig) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.TaskName = string(dAtA[iNdEx:postIndex]) + if m.HintedFrontier == nil { + m.HintedFrontier = &shuffle.Frontier{} + } + if err := m.HintedFrontier.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 3: + case 7: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field UdsPath", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LastApplied", wireType) } - var stringLen uint64 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4131,29 +13574,181 @@ func (m *TaskServiceConfig) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LastApplied = append(m.LastApplied[:0], dAtA[iNdEx:postIndex]...) + if m.LastApplied == nil { + m.LastApplied = []byte{} + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LegacyCheckpoint", 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.LegacyCheckpoint == nil { + m.LegacyCheckpoint = &protocol1.Checkpoint{} + } + if err := m.LegacyCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxKeys", 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 } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthRuntime - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthRuntime - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.UdsPath = string(dAtA[iNdEx:postIndex]) + if msglen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MaxKeys == nil { + m.MaxKeys = make(map[uint32][]byte) + } + var mapkey uint32 + mapvalue := []byte{} + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + } else if fieldNum == 2 { + var mapbyteLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapbyteLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intMapbyteLen := int(mapbyteLen) + if intMapbyteLen < 0 { + return ErrInvalidLengthRuntime + } + postbytesIndex := iNdEx + intMapbyteLen + if postbytesIndex < 0 { + return ErrInvalidLengthRuntime + } + if postbytesIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = make([]byte, mapbyteLen) + copy(mapvalue, dAtA[iNdEx:postbytesIndex]) + iNdEx = postbytesIndex + } else { + iNdEx = entryPreIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthRuntime + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.MaxKeys[mapkey] = mapvalue iNdEx = postIndex - case 4: + case 10: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ContainerNetwork", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TriggerParamsJson", wireType) } - var stringLen uint64 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4163,43 +13758,26 @@ func (m *TaskServiceConfig) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if byteLen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + byteLen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.ContainerNetwork = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Plane", wireType) - } - m.Plane = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRuntime - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Plane |= Plane(b&0x7F) << shift - if b < 0x80 { - break - } + m.TriggerParamsJson = append(m.TriggerParamsJson[:0], dAtA[iNdEx:postIndex]...) + if m.TriggerParamsJson == nil { + m.TriggerParamsJson = []byte{} } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRuntime(dAtA[iNdEx:]) @@ -4222,7 +13800,7 @@ func (m *TaskServiceConfig) Unmarshal(dAtA []byte) error { } return nil } -func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { +func (m *Persist) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -4245,17 +13823,17 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ShuffleRequest: wiretype end group for non-group") + return fmt.Errorf("proto: Persist: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ShuffleRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Persist: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Journal", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Nonce", wireType) } - var stringLen uint64 + m.Nonce = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4265,27 +13843,14 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.Nonce |= uint64(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthRuntime - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthRuntime - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Journal = go_gazette_dev_core_broker_protocol.Journal(dAtA[iNdEx:postIndex]) - iNdEx = postIndex case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Replay", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DeleteAckIntents", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -4302,12 +13867,12 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { break } } - m.Replay = bool(v != 0) + m.DeleteAckIntents = bool(v != 0) case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BuildId", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field AckIntents", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4317,29 +13882,135 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.BuildId = string(dAtA[iNdEx:postIndex]) + if m.AckIntents == nil { + m.AckIntents = make(map[string][]byte) + } + var mapkey string + mapvalue := []byte{} + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLenmapkey |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLengthRuntime + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey < 0 { + return ErrInvalidLengthRuntime + } + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey = string(dAtA[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + } else if fieldNum == 2 { + var mapbyteLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapbyteLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intMapbyteLen := int(mapbyteLen) + if intMapbyteLen < 0 { + return ErrInvalidLengthRuntime + } + postbytesIndex := iNdEx + intMapbyteLen + if postbytesIndex < 0 { + return ErrInvalidLengthRuntime + } + if postbytesIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = make([]byte, mapbyteLen) + copy(mapvalue, dAtA[iNdEx:postbytesIndex]) + iNdEx = postbytesIndex + } else { + iNdEx = entryPreIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthRuntime + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.AckIntents[mapkey] = mapvalue iNdEx = postIndex case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Offset", wireType) + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field CommittedCloseClock", wireType) } - m.Offset = 0 + m.CommittedCloseClock = 0 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + m.CommittedCloseClock = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommittedFrontier", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4349,16 +14020,77 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Offset |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 5: + if msglen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CommittedFrontier == nil { + m.CommittedFrontier = &shuffle.Frontier{} + } + if err := m.CommittedFrontier.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorPatchesJson", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ConnectorPatchesJson = append(m.ConnectorPatchesJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorPatchesJson == nil { + m.ConnectorPatchesJson = []byte{} + } + iNdEx = postIndex + case 7: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field HintedCloseClock", wireType) + } + m.HintedCloseClock = 0 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + m.HintedCloseClock = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + case 8: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field EndOffset", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DeleteHintedFrontier", wireType) } - m.EndOffset = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4368,14 +14100,15 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.EndOffset |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - case 6: + m.DeleteHintedFrontier = bool(v != 0) + case 9: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Range", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field HintedFrontier", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -4402,15 +14135,18 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.Range.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if m.HintedFrontier == nil { + m.HintedFrontier = &shuffle.Frontier{} + } + if err := m.HintedFrontier.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 7: + case 10: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Coordinator", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LastApplied", wireType) } - var stringLen uint64 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4420,27 +14156,29 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if byteLen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + byteLen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.Coordinator = go_gazette_dev_core_consumer_protocol.ShardID(dAtA[iNdEx:postIndex]) + m.LastApplied = append(m.LastApplied[:0], dAtA[iNdEx:postIndex]...) + if m.LastApplied == nil { + m.LastApplied = []byte{} + } iNdEx = postIndex - case 8: + case 11: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Resolution", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LegacyCheckpoint", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -4467,35 +14205,16 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Resolution == nil { - m.Resolution = &protocol.Header{} + if m.LegacyCheckpoint == nil { + m.LegacyCheckpoint = &protocol1.Checkpoint{} } - if err := m.Resolution.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.LegacyCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 9: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ShuffleIndex", wireType) - } - m.ShuffleIndex = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRuntime - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.ShuffleIndex |= uint32(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 10: + case 12: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Derivation", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field MaxKeys", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -4522,18 +14241,116 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Derivation == nil { - m.Derivation = &flow.CollectionSpec{} + if m.MaxKeys == nil { + m.MaxKeys = make(map[uint32][]byte) } - if err := m.Derivation.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + var mapkey uint32 + mapvalue := []byte{} + for iNdEx < postIndex { + entryPreIndex := 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) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + } else if fieldNum == 2 { + var mapbyteLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapbyteLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intMapbyteLen := int(mapbyteLen) + if intMapbyteLen < 0 { + return ErrInvalidLengthRuntime + } + postbytesIndex := iNdEx + intMapbyteLen + if postbytesIndex < 0 { + return ErrInvalidLengthRuntime + } + if postbytesIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = make([]byte, mapbyteLen) + copy(mapvalue, dAtA[iNdEx:postbytesIndex]) + iNdEx = postbytesIndex + } else { + iNdEx = entryPreIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthRuntime + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } } + m.MaxKeys[mapkey] = mapvalue iNdEx = postIndex - case 11: + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DeleteTriggerParams", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.DeleteTriggerParams = bool(v != 0) + case 14: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Materialization", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TriggerParamsJson", wireType) } - var msglen int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4543,26 +14360,24 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + if byteLen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + msglen + postIndex := iNdEx + byteLen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - if m.Materialization == nil { - m.Materialization = &flow.MaterializationSpec{} - } - if err := m.Materialization.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.TriggerParamsJson = append(m.TriggerParamsJson[:0], dAtA[iNdEx:postIndex]...) + if m.TriggerParamsJson == nil { + m.TriggerParamsJson = []byte{} } iNdEx = postIndex default: @@ -4587,7 +14402,7 @@ func (m *ShuffleRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *ShuffleResponse) Unmarshal(dAtA []byte) error { +func (m *Persisted) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -4610,36 +14425,17 @@ func (m *ShuffleResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ShuffleResponse: wiretype end group for non-group") + return fmt.Errorf("proto: Persisted: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ShuffleResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Persisted: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) - } - m.Status = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRuntime - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Status |= protocol1.Status(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Nonce", wireType) } - var msglen int + m.Nonce = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4649,84 +14445,67 @@ func (m *ShuffleResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.Nonce |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthRuntime + default: + iNdEx = preIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + msglen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthRuntime } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - if m.Header == nil { - m.Header = &protocol.Header{} - } - if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TerminalError", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRuntime - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthRuntime - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthRuntime + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Apply) 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 postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - m.TerminalError = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ReadThrough", wireType) - } - m.ReadThrough = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRuntime - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.ReadThrough |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift - if b < 0x80 { - break - } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field WriteHead", wireType) + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Apply: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Apply: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Spec", wireType) } - m.WriteHead = 0 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4736,16 +14515,31 @@ func (m *ShuffleResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.WriteHead |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 6: + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Spec = append(m.Spec[:0], dAtA[iNdEx:postIndex]...) + if m.Spec == nil { + m.Spec = []byte{} + } + iNdEx = postIndex + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Arena", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) } - var byteLen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4755,31 +14549,29 @@ func (m *ShuffleResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + byteLen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.Arena = append(m.Arena[:0], dAtA[iNdEx:postIndex]...) - if m.Arena == nil { - m.Arena = []byte{} - } + m.Version = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 7: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Docs", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LastSpec", wireType) } - var msglen int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4789,107 +14581,31 @@ func (m *ShuffleResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + if byteLen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + msglen + postIndex := iNdEx + byteLen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.Docs = append(m.Docs, flow.Slice{}) - if err := m.Docs[len(m.Docs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.LastSpec = append(m.LastSpec[:0], dAtA[iNdEx:postIndex]...) + if m.LastSpec == nil { + m.LastSpec = []byte{} } iNdEx = postIndex - case 8: - if wireType == 0 { - var v go_gazette_dev_core_broker_protocol.Offset - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRuntime - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Offsets = append(m.Offsets, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRuntime - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthRuntime - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthRuntime - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.Offsets) == 0 { - m.Offsets = make([]go_gazette_dev_core_broker_protocol.Offset, 0, elementCount) - } - for iNdEx < postIndex { - var v go_gazette_dev_core_broker_protocol.Offset - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRuntime - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= go_gazette_dev_core_broker_protocol.Offset(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Offsets = append(m.Offsets, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field Offsets", wireType) - } - case 9: + case 5: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field UuidParts", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LastVersion", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4899,31 +14615,29 @@ func (m *ShuffleResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.UuidParts = append(m.UuidParts, flow.UUIDParts{}) - if err := m.UuidParts[len(m.UuidParts)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.LastVersion = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 10: + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PackedKey", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorStateJson", wireType) } - var msglen int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -4933,24 +14647,24 @@ func (m *ShuffleResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + if byteLen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + msglen + postIndex := iNdEx + byteLen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.PackedKey = append(m.PackedKey, flow.Slice{}) - if err := m.PackedKey[len(m.PackedKey)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.ConnectorStateJson = append(m.ConnectorStateJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorStateJson == nil { + m.ConnectorStateJson = []byte{} } iNdEx = postIndex default: @@ -4975,7 +14689,7 @@ func (m *ShuffleResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *RocksDBDescriptor) Unmarshal(dAtA []byte) error { +func (m *Applied) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -4998,25 +14712,15 @@ func (m *RocksDBDescriptor) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RocksDBDescriptor: wiretype end group for non-group") + return fmt.Errorf("proto: Applied: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RocksDBDescriptor: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Applied: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field RocksdbEnvMemptr", wireType) - } - m.RocksdbEnvMemptr = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.RocksdbEnvMemptr = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RocksdbPath", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ActionDescription", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -5044,7 +14748,41 @@ func (m *RocksDBDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.RocksdbPath = string(dAtA[iNdEx:postIndex]) + m.ActionDescription = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorPatchesJson", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ConnectorPatchesJson = append(m.ConnectorPatchesJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorPatchesJson == nil { + m.ConnectorPatchesJson = []byte{} + } iNdEx = postIndex default: iNdEx = preIndex @@ -5068,7 +14806,7 @@ func (m *RocksDBDescriptor) Unmarshal(dAtA []byte) error { } return nil } -func (m *Container) Unmarshal(dAtA []byte) error { +func (m *Open) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5081,25 +14819,59 @@ func (m *Container) Unmarshal(dAtA []byte) error { if iNdEx >= l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + 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: Open: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Open: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Spec", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Spec = append(m.Spec[:0], dAtA[iNdEx:postIndex]...) + if m.Spec == nil { + m.Spec = []byte{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: Container: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Container: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + iNdEx = postIndex + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field IpAddr", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -5127,11 +14899,11 @@ func (m *Container) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.IpAddr = string(dAtA[iNdEx:postIndex]) + m.Version = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field NetworkPorts", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Range", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5158,14 +14930,50 @@ func (m *Container) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.NetworkPorts = append(m.NetworkPorts, &flow.NetworkPort{}) - if err := m.NetworkPorts[len(m.NetworkPorts)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if m.Range == nil { + m.Range = &flow.RangeSpec{} + } + if err := m.Range.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 3: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field MappedHostPorts", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorStateJson", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ConnectorStateJson = append(m.ConnectorStateJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorStateJson == nil { + m.ConnectorStateJson = []byte{} + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxKeys", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5192,11 +15000,11 @@ func (m *Container) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.MappedHostPorts == nil { - m.MappedHostPorts = make(map[uint32]string) + if m.MaxKeys == nil { + m.MaxKeys = make(map[uint32][]byte) } var mapkey uint32 - var mapvalue string + mapvalue := []byte{} for iNdEx < postIndex { entryPreIndex := iNdEx var wire uint64 @@ -5231,7 +15039,7 @@ func (m *Container) Unmarshal(dAtA []byte) error { } } } else if fieldNum == 2 { - var stringLenmapvalue uint64 + var mapbyteLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -5241,24 +15049,25 @@ func (m *Container) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLenmapvalue |= uint64(b&0x7F) << shift + mapbyteLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - intStringLenmapvalue := int(stringLenmapvalue) - if intStringLenmapvalue < 0 { + intMapbyteLen := int(mapbyteLen) + if intMapbyteLen < 0 { return ErrInvalidLengthRuntime } - postStringIndexmapvalue := iNdEx + intStringLenmapvalue - if postStringIndexmapvalue < 0 { + postbytesIndex := iNdEx + intMapbyteLen + if postbytesIndex < 0 { return ErrInvalidLengthRuntime } - if postStringIndexmapvalue > l { + if postbytesIndex > l { return io.ErrUnexpectedEOF } - mapvalue = string(dAtA[iNdEx:postStringIndexmapvalue]) - iNdEx = postStringIndexmapvalue + mapvalue = make([]byte, mapbyteLen) + copy(mapvalue, dAtA[iNdEx:postbytesIndex]) + iNdEx = postbytesIndex } else { iNdEx = entryPreIndex skippy, err := skipRuntime(dAtA[iNdEx:]) @@ -5274,19 +15083,8 @@ func (m *Container) Unmarshal(dAtA []byte) error { iNdEx += skippy } } - m.MappedHostPorts[mapkey] = mapvalue + m.MaxKeys[mapkey] = mapvalue iNdEx = postIndex - case 4: - if wireType != 5 { - return fmt.Errorf("proto: wrong wireType = %d for field UsageRate", wireType) - } - var v uint32 - if (iNdEx + 4) > l { - return io.ErrUnexpectedEOF - } - v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) - iNdEx += 4 - m.UsageRate = float32(math.Float32frombits(v)) default: iNdEx = preIndex skippy, err := skipRuntime(dAtA[iNdEx:]) @@ -5309,7 +15107,7 @@ func (m *Container) Unmarshal(dAtA []byte) error { } return nil } -func (m *CaptureRequestExt) Unmarshal(dAtA []byte) error { +func (m *CloseNow) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5332,103 +15130,114 @@ func (m *CaptureRequestExt) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CaptureRequestExt: wiretype end group for non-group") + return fmt.Errorf("proto: CloseNow: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CaptureRequestExt: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: CloseNow: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LogLevel", wireType) - } - m.LogLevel = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRuntime - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.LogLevel |= ops.Log_Level(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RocksdbDescriptor", 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 + default: + iNdEx = preIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + msglen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthRuntime } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - if m.RocksdbDescriptor == nil { - m.RocksdbDescriptor = &RocksDBDescriptor{} - } - if err := m.RocksdbDescriptor.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Stop) 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 } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field StartCommit", wireType) + if iNdEx >= l { + return io.ErrUnexpectedEOF } - 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 - } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - if msglen < 0 { - return ErrInvalidLengthRuntime + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Stop: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Stop: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + msglen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthRuntime } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - if m.StartCommit == nil { - m.StartCommit = &CaptureRequestExt_StartCommit{} + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Stopped) 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 err := m.StartCommit.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + if iNdEx >= l { + return io.ErrUnexpectedEOF } - iNdEx = postIndex + 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: Stopped: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Stopped: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { default: iNdEx = preIndex skippy, err := skipRuntime(dAtA[iNdEx:]) @@ -5451,7 +15260,7 @@ func (m *CaptureRequestExt) Unmarshal(dAtA []byte) error { } return nil } -func (m *CaptureRequestExt_StartCommit) Unmarshal(dAtA []byte) error { +func (m *SessionLoop) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5474,15 +15283,15 @@ func (m *CaptureRequestExt_StartCommit) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StartCommit: wiretype end group for non-group") + return fmt.Errorf("proto: SessionLoop: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StartCommit: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SessionLoop: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RuntimeCheckpoint", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RocksdbDescriptor", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5509,10 +15318,10 @@ func (m *CaptureRequestExt_StartCommit) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.RuntimeCheckpoint == nil { - m.RuntimeCheckpoint = &protocol1.Checkpoint{} + if m.RocksdbDescriptor == nil { + m.RocksdbDescriptor = &RocksDBDescriptor{} } - if err := m.RuntimeCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.RocksdbDescriptor.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -5538,7 +15347,7 @@ func (m *CaptureRequestExt_StartCommit) Unmarshal(dAtA []byte) error { } return nil } -func (m *CaptureResponseExt) Unmarshal(dAtA []byte) error { +func (m *Materialize) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5561,15 +15370,15 @@ func (m *CaptureResponseExt) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CaptureResponseExt: wiretype end group for non-group") + return fmt.Errorf("proto: Materialize: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CaptureResponseExt: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Materialize: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Container", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Spec", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5596,16 +15405,16 @@ func (m *CaptureResponseExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Container == nil { - m.Container = &Container{} + if m.Spec == nil { + m.Spec = &materialize.Request_Spec{} } - if err := m.Container.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Spec.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Opened", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SpecResponse", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5632,16 +15441,16 @@ func (m *CaptureResponseExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Opened == nil { - m.Opened = &CaptureResponseExt_Opened{} + if m.SpecResponse == nil { + m.SpecResponse = &materialize.Response_Spec{} } - if err := m.Opened.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.SpecResponse.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Captured", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Validate", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5668,16 +15477,16 @@ func (m *CaptureResponseExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Captured == nil { - m.Captured = &CaptureResponseExt_Captured{} + if m.Validate == nil { + m.Validate = &materialize.Request_Validate{} } - if err := m.Captured.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Validate.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Checkpoint", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Validated", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5704,67 +15513,52 @@ func (m *CaptureResponseExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Checkpoint == nil { - m.Checkpoint = &CaptureResponseExt_Checkpoint{} + if m.Validated == nil { + m.Validated = &materialize.Response_Validated{} } - if err := m.Checkpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Validated.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipRuntime(dAtA[iNdEx:]) - if err != nil { - return err + case 20: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SessionLoop", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthRuntime + 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 (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + if msglen < 0 { + return ErrInvalidLengthRuntime } - m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *CaptureResponseExt_Opened) 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 + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime } - if iNdEx >= l { + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if m.SessionLoop == nil { + m.SessionLoop = &SessionLoop{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: Opened: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Opened: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + if err := m.SessionLoop.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 21: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RuntimeCheckpoint", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Join", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5791,69 +15585,54 @@ func (m *CaptureResponseExt_Opened) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.RuntimeCheckpoint == nil { - m.RuntimeCheckpoint = &protocol1.Checkpoint{} + if m.Join == nil { + m.Join = &Join{} } - if err := m.RuntimeCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Join.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipRuntime(dAtA[iNdEx:]) - if err != nil { - return err + case 22: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Joined", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthRuntime + 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 (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + if msglen < 0 { + return ErrInvalidLengthRuntime } - m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *CaptureResponseExt_Captured) 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 + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime } - if iNdEx >= l { + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if m.Joined == nil { + m.Joined = &Joined{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: Captured: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Captured: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + if err := m.Joined.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 23: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field KeyPacked", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -5863,31 +15642,33 @@ func (m *CaptureResponseExt_Captured) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.KeyPacked = append(m.KeyPacked[:0], dAtA[iNdEx:postIndex]...) - if m.KeyPacked == nil { - m.KeyPacked = []byte{} + if m.Task == nil { + m.Task = &Task{} + } + if err := m.Task.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } iNdEx = postIndex - case 2: + case 24: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PartitionsPacked", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Recover", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -5897,80 +15678,67 @@ func (m *CaptureResponseExt_Captured) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.PartitionsPacked = append(m.PartitionsPacked[:0], dAtA[iNdEx:postIndex]...) - if m.PartitionsPacked == nil { - m.PartitionsPacked = []byte{} + if m.Recover == nil { + m.Recover = &Recover{} } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipRuntime(dAtA[iNdEx:]) - if err != nil { + if err := m.Recover.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthRuntime - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + iNdEx = postIndex + case 25: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Apply", wireType) } - m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *CaptureResponseExt_Checkpoint) 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 + 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 iNdEx >= l { - return io.ErrUnexpectedEOF + if msglen < 0 { + return ErrInvalidLengthRuntime } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: Checkpoint: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Checkpoint: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Apply == nil { + m.Apply = &Apply{} + } + if err := m.Apply.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 26: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Stats", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Applied", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5997,18 +15765,18 @@ func (m *CaptureResponseExt_Checkpoint) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Stats == nil { - m.Stats = &ops.Stats{} + if m.Applied == nil { + m.Applied = &Applied{} } - if err := m.Stats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Applied.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field PollResult", wireType) + case 27: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Open", wireType) } - m.PollResult = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -6018,67 +15786,33 @@ func (m *CaptureResponseExt_Checkpoint) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.PollResult |= CaptureResponseExt_PollResult(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - default: - iNdEx = preIndex - skippy, err := skipRuntime(dAtA[iNdEx:]) - if err != nil { - return err + if msglen < 0 { + return ErrInvalidLengthRuntime } - if (skippy < 0) || (iNdEx+skippy) < 0 { + postIndex := iNdEx + msglen + if postIndex < 0 { return ErrInvalidLengthRuntime } - if (iNdEx + skippy) > l { + if postIndex > 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 *DeriveRequestExt) 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 + if m.Open == nil { + m.Open = &Open{} } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if err := m.Open.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: DeriveRequestExt: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: DeriveRequestExt: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LogLevel", wireType) + iNdEx = postIndex + case 28: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Opened", wireType) } - m.LogLevel = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -6088,14 +15822,31 @@ func (m *DeriveRequestExt) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.LogLevel |= ops.Log_Level(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 2: + if msglen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Opened == nil { + m.Opened = &Materialize_Opened{} + } + if err := m.Opened.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 40: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RocksdbDescriptor", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Load", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6122,16 +15873,16 @@ func (m *DeriveRequestExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.RocksdbDescriptor == nil { - m.RocksdbDescriptor = &RocksDBDescriptor{} + if m.Load == nil { + m.Load = &Materialize_Load{} } - if err := m.RocksdbDescriptor.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Load.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 3: + case 41: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Open", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Loaded", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6158,69 +15909,54 @@ func (m *DeriveRequestExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Open == nil { - m.Open = &DeriveRequestExt_Open{} + if m.Loaded == nil { + m.Loaded = &Materialize_Loaded{} } - if err := m.Open.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Loaded.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipRuntime(dAtA[iNdEx:]) - if err != nil { - return err + case 42: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Flush", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthRuntime + 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 (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF + if msglen < 0 { + return ErrInvalidLengthRuntime } - m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *DeriveRequestExt_Open) 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 + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime } - if iNdEx >= l { + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if m.Flush == nil { + m.Flush = &Materialize_Flush{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: Open: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Open: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + if err := m.Flush.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 43: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SqliteVfsUri", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Flushed", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -6230,78 +15966,31 @@ func (m *DeriveRequestExt_Open) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.SqliteVfsUri = string(dAtA[iNdEx:postIndex]) - 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 *DeriveResponseExt) 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 + if m.Flushed == nil { + m.Flushed = &Materialize_Flushed{} } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if err := m.Flushed.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: DeriveResponseExt: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: DeriveResponseExt: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + iNdEx = postIndex + case 44: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Container", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Store", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6328,16 +16017,16 @@ func (m *DeriveResponseExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Container == nil { - m.Container = &Container{} + if m.Store == nil { + m.Store = &Materialize_Store{} } - if err := m.Container.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Store.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: + case 45: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Opened", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Stored", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6364,16 +16053,16 @@ func (m *DeriveResponseExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Opened == nil { - m.Opened = &DeriveResponseExt_Opened{} + if m.Stored == nil { + m.Stored = &Materialize_Stored{} } - if err := m.Opened.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Stored.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 3: + case 46: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Published", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartCommit", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6400,16 +16089,16 @@ func (m *DeriveResponseExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Published == nil { - m.Published = &DeriveResponseExt_Published{} + if m.StartCommit == nil { + m.StartCommit = &Materialize_StartCommit{} } - if err := m.Published.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.StartCommit.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 4: + case 47: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Flushed", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartedCommit", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6436,67 +16125,16 @@ func (m *DeriveResponseExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Flushed == nil { - m.Flushed = &DeriveResponseExt_Flushed{} + if m.StartedCommit == nil { + m.StartedCommit = &Materialize_StartedCommit{} } - if err := m.Flushed.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.StartedCommit.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 *DeriveResponseExt_Opened) 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: Opened: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Opened: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 48: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RuntimeCheckpoint", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Acknowledge", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6523,79 +16161,18 @@ func (m *DeriveResponseExt_Opened) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.RuntimeCheckpoint == nil { - m.RuntimeCheckpoint = &protocol1.Checkpoint{} + if m.Acknowledge == nil { + m.Acknowledge = &Materialize_Acknowledge{} } - if err := m.RuntimeCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Acknowledge.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 *DeriveResponseExt_Published) 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: Published: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Published: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxClock", wireType) - } - m.MaxClock = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - m.MaxClock = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - case 2: + case 49: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field KeyPacked", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Acknowledged", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -6605,31 +16182,33 @@ func (m *DeriveResponseExt_Published) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.KeyPacked = append(m.KeyPacked[:0], dAtA[iNdEx:postIndex]...) - if m.KeyPacked == nil { - m.KeyPacked = []byte{} + if m.Acknowledged == nil { + m.Acknowledged = &Materialize_Acknowledged{} + } + if err := m.Acknowledged.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } iNdEx = postIndex - case 3: + case 50: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PartitionsPacked", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Persist", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -6639,80 +16218,31 @@ func (m *DeriveResponseExt_Published) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.PartitionsPacked = append(m.PartitionsPacked[:0], dAtA[iNdEx:postIndex]...) - if m.PartitionsPacked == nil { - m.PartitionsPacked = []byte{} + if m.Persist == nil { + m.Persist = &Persist{} } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipRuntime(dAtA[iNdEx:]) - if err != nil { + if err := m.Persist.Unmarshal(dAtA[iNdEx:postIndex]); 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 *DeriveResponseExt_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: + iNdEx = postIndex + case 51: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Stats", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Persisted", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6739,69 +16269,54 @@ func (m *DeriveResponseExt_Flushed) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Stats == nil { - m.Stats = &ops.Stats{} + if m.Persisted == nil { + m.Persisted = &Persisted{} } - if err := m.Stats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Persisted.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 + case 52: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CloseNow", wireType) } - m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *MaterializeRequestExt) 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 + 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 iNdEx >= l { + if msglen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if m.CloseNow == nil { + m.CloseNow = &CloseNow{} } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: MaterializeRequestExt: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: MaterializeRequestExt: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LogLevel", wireType) + if err := m.CloseNow.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.LogLevel = 0 + iNdEx = postIndex + case 60: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Stop", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -6811,14 +16326,31 @@ func (m *MaterializeRequestExt) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.LogLevel |= ops.Log_Level(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 2: + if msglen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Stop == nil { + m.Stop = &Stop{} + } + if err := m.Stop.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 61: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RocksdbDescriptor", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Stopped", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6845,10 +16377,10 @@ func (m *MaterializeRequestExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.RocksdbDescriptor == nil { - m.RocksdbDescriptor = &RocksDBDescriptor{} + if m.Stopped == nil { + m.Stopped = &Stopped{} } - if err := m.RocksdbDescriptor.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Stopped.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -6874,7 +16406,7 @@ func (m *MaterializeRequestExt) Unmarshal(dAtA []byte) error { } return nil } -func (m *MaterializeResponseExt) Unmarshal(dAtA []byte) error { +func (m *Materialize_Opened) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6897,10 +16429,10 @@ func (m *MaterializeResponseExt) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: MaterializeResponseExt: wiretype end group for non-group") + return fmt.Errorf("proto: Opened: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: MaterializeResponseExt: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Opened: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -6941,7 +16473,7 @@ func (m *MaterializeResponseExt) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Flushed", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorCheckpoint", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6968,10 +16500,10 @@ func (m *MaterializeResponseExt) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Flushed == nil { - m.Flushed = &MaterializeResponseExt_Flushed{} + if m.ConnectorCheckpoint == nil { + m.ConnectorCheckpoint = &protocol1.Checkpoint{} } - if err := m.Flushed.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.ConnectorCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -6997,7 +16529,7 @@ func (m *MaterializeResponseExt) Unmarshal(dAtA []byte) error { } return nil } -func (m *MaterializeResponseExt_Flushed) Unmarshal(dAtA []byte) error { +func (m *Materialize_Load) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7020,15 +16552,15 @@ func (m *MaterializeResponseExt_Flushed) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Flushed: wiretype end group for non-group") + return fmt.Errorf("proto: Load: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Flushed: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Load: 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) + return fmt.Errorf("proto: wrong wireType = %d for field Frontier", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -7055,10 +16587,10 @@ func (m *MaterializeResponseExt_Flushed) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Stats == nil { - m.Stats = &ops.Stats{} + if m.Frontier == nil { + m.Frontier = &shuffle.Frontier{} } - if err := m.Stats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Frontier.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -7084,7 +16616,7 @@ func (m *MaterializeResponseExt_Flushed) Unmarshal(dAtA []byte) error { } return nil } -func (m *CombineRequest) Unmarshal(dAtA []byte) error { +func (m *Materialize_Loaded) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7107,15 +16639,15 @@ func (m *CombineRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CombineRequest: wiretype end group for non-group") + return fmt.Errorf("proto: Loaded: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CombineRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Loaded: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Open", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Bindings", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -7142,18 +16674,16 @@ func (m *CombineRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Open == nil { - m.Open = &CombineRequest_Open{} - } - if err := m.Open.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Bindings = append(m.Bindings, &Materialize_Loaded_Binding{}) + if err := m.Bindings[len(m.Bindings)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Add", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CombinerUsageBytes", wireType) } - var msglen int + m.CombinerUsageBytes = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7163,28 +16693,11 @@ func (m *CombineRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.CombinerUsageBytes |= uint64(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.Add == nil { - m.Add = &CombineRequest_Add{} - } - if err := m.Add.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRuntime(dAtA[iNdEx:]) @@ -7207,7 +16720,7 @@ func (m *CombineRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *CombineRequest_Open) Unmarshal(dAtA []byte) error { +func (m *Materialize_Loaded_Binding) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7230,17 +16743,17 @@ func (m *CombineRequest_Open) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Open: wiretype end group for non-group") + return fmt.Errorf("proto: Binding: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Open: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Binding: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Bindings", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) } - var msglen int + m.Index = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7250,82 +16763,36 @@ func (m *CombineRequest_Open) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.Index |= uint32(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 - } - m.Bindings = append(m.Bindings, &CombineRequest_Open_Binding{}) - if err := m.Bindings[len(m.Bindings)-1].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 + case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field MinSourceClock", wireType) } - if (iNdEx + skippy) > l { + m.MinSourceClock = 0 + if (iNdEx + 8) > 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 *CombineRequest_Open_Binding) 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 + m.MinSourceClock = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + case 3: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxSourceClock", wireType) } - if iNdEx >= l { + m.MaxSourceClock = 0 + if (iNdEx + 8) > 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: Binding: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Binding: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + m.MaxSourceClock = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Full", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SourcedDocsTotal", wireType) } - var v int + m.SourcedDocsTotal = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7335,17 +16802,16 @@ func (m *CombineRequest_Open_Binding) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.SourcedDocsTotal |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.Full = bool(v != 0) - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SourcedBytesTotal", wireType) } - var stringLen uint64 + m.SourcedBytesTotal = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7355,29 +16821,16 @@ func (m *CombineRequest_Open_Binding) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.SourcedBytesTotal |= uint64(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthRuntime - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthRuntime - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Key = append(m.Key, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 3: + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Projections", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field MaxKeyDelta", wireType) } - var msglen int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7387,29 +16840,80 @@ func (m *CombineRequest_Open_Binding) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + if byteLen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + msglen + postIndex := iNdEx + byteLen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.Projections = append(m.Projections, flow.Projection{}) - if err := m.Projections[len(m.Projections)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.MaxKeyDelta = append(m.MaxKeyDelta[:0], dAtA[iNdEx:postIndex]...) + if m.MaxKeyDelta == nil { + m.MaxKeyDelta = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { return err } - iNdEx = postIndex - case 4: + 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 *Materialize_Flush) 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: Flush: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Flush: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SchemaJson", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorPatchesJson", wireType) } var byteLen int for shift := uint(0); ; shift += 7 { @@ -7436,52 +16940,67 @@ func (m *CombineRequest_Open_Binding) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.SchemaJson = append(m.SchemaJson[:0], dAtA[iNdEx:postIndex]...) - if m.SchemaJson == nil { - m.SchemaJson = []byte{} + m.ConnectorPatchesJson = append(m.ConnectorPatchesJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorPatchesJson == nil { + m.ConnectorPatchesJson = []byte{} } iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SerPolicy", 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 + default: + iNdEx = preIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + msglen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthRuntime } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - if m.SerPolicy == nil { - m.SerPolicy = &flow.SerPolicy{} + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Materialize_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 err := m.SerPolicy.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + if iNdEx >= l { + return io.ErrUnexpectedEOF } - iNdEx = postIndex - case 6: + 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 UuidPtr", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Bindings", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7491,29 +17010,31 @@ func (m *CombineRequest_Open_Binding) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.UuidPtr = string(dAtA[iNdEx:postIndex]) + m.Bindings = append(m.Bindings, &Materialize_Flushed_Binding{}) + if err := m.Bindings[len(m.Bindings)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex - case 7: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Values", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorPatchesJson", wireType) } - var stringLen uint64 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7523,23 +17044,25 @@ func (m *CombineRequest_Open_Binding) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if byteLen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + byteLen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.Values = append(m.Values, string(dAtA[iNdEx:postIndex])) + m.ConnectorPatchesJson = append(m.ConnectorPatchesJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorPatchesJson == nil { + m.ConnectorPatchesJson = []byte{} + } iNdEx = postIndex default: iNdEx = preIndex @@ -7563,7 +17086,7 @@ func (m *CombineRequest_Open_Binding) Unmarshal(dAtA []byte) error { } return nil } -func (m *CombineRequest_Add) Unmarshal(dAtA []byte) error { +func (m *Materialize_Flushed_Binding) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7586,17 +17109,17 @@ func (m *CombineRequest_Add) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Add: wiretype end group for non-group") + return fmt.Errorf("proto: Binding: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Add: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Binding: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Binding", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) } - m.Binding = 0 + m.Index = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7606,16 +17129,16 @@ func (m *CombineRequest_Add) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Binding |= uint32(b&0x7F) << shift + m.Index |= uint32(b&0x7F) << shift if b < 0x80 { break } } case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DocJson", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LoadedDocsTotal", wireType) } - var byteLen int + m.LoadedDocsTotal = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7625,31 +17148,137 @@ func (m *CombineRequest_Add) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + m.LoadedDocsTotal |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LoadedBytesTotal", wireType) + } + m.LoadedBytesTotal = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LoadedBytesTotal |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + byteLen - if postIndex < 0 { + 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 *Materialize_Store) 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: Store: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Store: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthRuntime } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.DocJson = append(m.DocJson[:0], dAtA[iNdEx:postIndex]...) - if m.DocJson == nil { - m.DocJson = []byte{} + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Materialize_Stored) 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 } - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Front", wireType) + if iNdEx >= l { + return io.ErrUnexpectedEOF } - var v int + 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: Stored: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Stored: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Bindings", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7659,12 +17288,26 @@ func (m *CombineRequest_Add) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.Front = bool(v != 0) + if msglen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Bindings = append(m.Bindings, &Materialize_Stored_Binding{}) + if err := m.Bindings[len(m.Bindings)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRuntime(dAtA[iNdEx:]) @@ -7687,7 +17330,7 @@ func (m *CombineRequest_Add) Unmarshal(dAtA []byte) error { } return nil } -func (m *CombineResponse) Unmarshal(dAtA []byte) error { +func (m *Materialize_Stored_Binding) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7710,17 +17353,17 @@ func (m *CombineResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CombineResponse: wiretype end group for non-group") + return fmt.Errorf("proto: Binding: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CombineResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Binding: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Binding", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) } - m.Binding = 0 + m.Index = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7730,16 +17373,16 @@ func (m *CombineResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Binding |= uint32(b&0x7F) << shift + m.Index |= uint32(b&0x7F) << shift if b < 0x80 { break } } case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Deleted", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StoredDocsTotal", wireType) } - var v int + m.StoredDocsTotal = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7749,17 +17392,16 @@ func (m *CombineResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.StoredDocsTotal |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.Deleted = bool(v != 0) case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DocJson", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StoredBytesTotal", wireType) } - var byteLen int + m.StoredBytesTotal = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7769,49 +17411,65 @@ func (m *CombineResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + m.StoredBytesTotal |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { - return ErrInvalidLengthRuntime + default: + iNdEx = preIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + byteLen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthRuntime } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.DocJson = append(m.DocJson[:0], dAtA[iNdEx:postIndex]...) - if m.DocJson == nil { - m.DocJson = []byte{} + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Materialize_StartCommit) 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 } - iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Front", wireType) + if iNdEx >= l { + return io.ErrUnexpectedEOF } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRuntime - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - m.Front = bool(v != 0) - case 5: + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: StartCommit: wiretype end group for non-group") + } + if fieldNum <= 0 { + 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 KeyPacked", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorPatchesJson", wireType) } var byteLen int for shift := uint(0); ; shift += 7 { @@ -7838,16 +17496,16 @@ func (m *CombineResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.KeyPacked = append(m.KeyPacked[:0], dAtA[iNdEx:postIndex]...) - if m.KeyPacked == nil { - m.KeyPacked = []byte{} + m.ConnectorPatchesJson = append(m.ConnectorPatchesJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorPatchesJson == nil { + m.ConnectorPatchesJson = []byte{} } iNdEx = postIndex - case 6: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ValuesPacked", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorCheckpoint", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7857,24 +17515,26 @@ func (m *CombineResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.ValuesPacked = append(m.ValuesPacked[:0], dAtA[iNdEx:postIndex]...) - if m.ValuesPacked == nil { - m.ValuesPacked = []byte{} + if m.ConnectorCheckpoint == nil { + m.ConnectorCheckpoint = &protocol1.Checkpoint{} + } + if err := m.ConnectorCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } iNdEx = postIndex default: @@ -7899,7 +17559,7 @@ func (m *CombineResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *ConnectorProxyRequest) Unmarshal(dAtA []byte) error { +func (m *Materialize_StartedCommit) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7922,12 +17582,46 @@ func (m *ConnectorProxyRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ConnectorProxyRequest: wiretype end group for non-group") + return fmt.Errorf("proto: StartedCommit: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ConnectorProxyRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: StartedCommit: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorPatchesJson", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuntime + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRuntime + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRuntime + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ConnectorPatchesJson = append(m.ConnectorPatchesJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorPatchesJson == nil { + m.ConnectorPatchesJson = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRuntime(dAtA[iNdEx:]) @@ -7950,7 +17644,7 @@ func (m *ConnectorProxyRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *ConnectorProxyResponse) Unmarshal(dAtA []byte) error { +func (m *Materialize_Acknowledge) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7973,17 +17667,17 @@ func (m *ConnectorProxyResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ConnectorProxyResponse: wiretype end group for non-group") + return fmt.Errorf("proto: Acknowledge: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ConnectorProxyResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Acknowledge: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Address", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorPatchesJson", wireType) } - var stringLen uint64 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -7993,61 +17687,82 @@ func (m *ConnectorProxyResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if byteLen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + byteLen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - m.Address = go_gazette_dev_core_broker_protocol.Endpoint(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ProxyId", wireType) + m.ConnectorPatchesJson = append(m.ConnectorPatchesJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorPatchesJson == nil { + m.ConnectorPatchesJson = []byte{} } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowRuntime - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRuntime(dAtA[iNdEx:]) + if err != nil { + return err } - intStringLen := int(stringLen) - if intStringLen < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthRuntime + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF } - if postIndex > l { + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Materialize_Acknowledged) 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 } - m.ProxyId = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: + 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: Acknowledged: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Acknowledged: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Log", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ConnectorPatchesJson", wireType) } - var msglen int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -8057,26 +17772,24 @@ func (m *ConnectorProxyResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + byteLen |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + if byteLen < 0 { return ErrInvalidLengthRuntime } - postIndex := iNdEx + msglen + postIndex := iNdEx + byteLen if postIndex < 0 { return ErrInvalidLengthRuntime } if postIndex > l { return io.ErrUnexpectedEOF } - if m.Log == nil { - m.Log = &ops.Log{} - } - if err := m.Log.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.ConnectorPatchesJson = append(m.ConnectorPatchesJson[:0], dAtA[iNdEx:postIndex]...) + if m.ConnectorPatchesJson == nil { + m.ConnectorPatchesJson = []byte{} } iNdEx = postIndex default: diff --git a/go/protocols/runtime/runtime.proto b/go/protocols/runtime/runtime.proto index 93e2d41500d..96aac6241a4 100644 --- a/go/protocols/runtime/runtime.proto +++ b/go/protocols/runtime/runtime.proto @@ -5,8 +5,11 @@ option go_package = "github.com/estuary/flow/go/protocols/runtime"; import "broker/protocol/protocol.proto"; import "consumer/protocol/protocol.proto"; +import "go/protocols/derive/derive.proto"; import "go/protocols/flow/flow.proto"; +import "go/protocols/materialize/materialize.proto"; import "go/protocols/ops/ops.proto"; +import "go/protocols/shuffle/shuffle.proto"; import "gogoproto/gogo.proto"; option (gogoproto.marshaler_all) = true; @@ -26,11 +29,11 @@ enum Plane { } message TaskServiceConfig { - int32 log_file_fd = 1; - string task_name = 2; - string uds_path = 3; + int32 log_file_fd = 1; + string task_name = 2; + string uds_path = 3; string container_network = 4; - Plane plane = 6; + Plane plane = 6; } // ShuffleRequest is the request message of a Shuffle RPC. @@ -50,7 +53,7 @@ message TaskServiceConfig { message ShuffleRequest { // Journal to be shuffled. string journal = 1 - [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Journal" ]; + [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Journal" ]; // Is this a reply of the journal's content? We separate ongoing vs replayed // reads of a journal's content into distinct rings. bool replay = 2; @@ -59,10 +62,10 @@ message ShuffleRequest { // Offset to begin reading the journal from. int64 offset = 4 - [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Offset" ]; + [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Offset" ]; // Offset to stop reading the journal at, or zero if unbounded. int64 end_offset = 5 - [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Offset" ]; + [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Offset" ]; // Ranges of responsibility which are unique to this reader, // against which document shuffle outcomes are matched to determine // read eligibility. @@ -70,7 +73,7 @@ message ShuffleRequest { // Coordinator is the Shard ID which is responsible for reads of this journal. string coordinator = 7 - [ (gogoproto.casttype) = "go.gazette.dev/core/consumer/protocol.ShardID" ]; + [ (gogoproto.casttype) = "go.gazette.dev/core/consumer/protocol.ShardID" ]; // Resolution header of the |shuffle.coordinator| shard. protocol.Header resolution = 8; @@ -102,11 +105,11 @@ message ShuffleResponse { string terminal_error = 3; // Offset which was read through to produce this ShuffleResponse. int64 read_through = 4 - [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Offset" ]; + [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Offset" ]; // WriteHead of the journal as reported by the broker, as of the creation of // this ShuffleResponse. int64 write_head = 5 - [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Offset" ]; + [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Offset" ]; // Memory arena of this message. bytes arena = 6 [ (gogoproto.casttype) = "github.com/estuary/flow/go/protocols/flow.Arena" ]; // Shuffled documents, each encoded in the 'application/json' @@ -116,7 +119,7 @@ message ShuffleResponse { // For a document at index i, its offsets are [ offsets[2*i], offsets[2*i+1] // ). repeated int64 offsets = 8 - [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Offset" ]; + [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Offset" ]; // UUIDParts of each document. repeated flow.UUIDParts uuid_parts = 9 [ (gogoproto.nullable) = false ]; // Packed, embedded encoding of the shuffle key into a byte string. @@ -129,10 +132,10 @@ service Shuffler { } message RocksDBDescriptor { - // Instrumented RocksDB environment which should be opened. - fixed64 rocksdb_env_memptr = 1; - // Path to the RocksDB directory to be opened. - string rocksdb_path = 2; + // Instrumented RocksDB environment which should be opened. + fixed64 rocksdb_env_memptr = 1; + // Path to the RocksDB directory to be opened. + string rocksdb_path = 2; } // Container is a description of a running connector container. @@ -166,7 +169,6 @@ message CaptureRequestExt { StartCommit start_commit = 3; } - message CaptureResponseExt { Container container = 1; @@ -176,10 +178,10 @@ message CaptureResponseExt { Opened opened = 2; message Captured { - // Packed key extracted from the captured document. - bytes key_packed = 1; - // Packed partition values extracted from the captured document. - bytes partitions_packed = 2; + // Packed key extracted from the captured document. + bytes key_packed = 1; + // Packed partition values extracted from the captured document. + bytes partitions_packed = 2; } Captured captured = 3; @@ -196,7 +198,7 @@ message CaptureResponseExt { } message Checkpoint { - ops.Stats stats = 1; + ops.Stats stats = 1; PollResult poll_result = 2; } Checkpoint checkpoint = 4; @@ -225,12 +227,12 @@ message DeriveResponseExt { Opened opened = 2; message Published { - // Maximum UUID Clock of sourced document clocks which lead to this published document. - fixed64 max_clock = 1; - // Packed key extracted from the published document. - bytes key_packed = 2; - // Packed partition values extracted from the published document. - bytes partitions_packed = 3; + // Maximum UUID Clock of sourced document clocks which lead to this published document. + fixed64 max_clock = 1; + // Packed key extracted from the published document. + bytes key_packed = 2; + // Packed partition values extracted from the published document. + bytes partitions_packed = 3; } Published published = 3; @@ -320,7 +322,7 @@ message ConnectorProxyResponse { // Address to dial for invoking connector RPCs. // Sent with the first response. string address = 1 - [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Endpoint" ]; + [ (gogoproto.casttype) = "go.gazette.dev/core/broker/protocol.Endpoint" ]; // Proxy ID to be attached as metadata key "proxy-id" to connector RPCs. // Sent with the first response. @@ -334,3 +336,548 @@ message ConnectorProxyResponse { service ConnectorProxy { rpc ProxyConnectors(stream ConnectorProxyRequest) returns (stream ConnectorProxyResponse); } + +// ===================================================================== +// Runtime V2 protocol +// ===================================================================== +// +// Three roles cooperate over two bidirectional gRPC services: +// +// Controller drives a shard's lifecycle. The Go runtime in production, +// `flowctl preview` in dev, or a unit-test harness. The +// Controller talks to the Shard service. +// Shard runtime-next's per-shard service. Each Shard stream +// terminates *both* the Controller-bound protocol and an +// outbound Leader stream, translating between them and +// the connector RPC. +// Leader sidecar service that joins shards into a session and +// drives multi-shard transactions. Hosted on the reactor +// machine to which the task's shard zero is assigned. +// +// Both services carry the same `Materialize` / `Derive` message types, +// and exactly one field is set per message. +// +// Shard zero is special: it hosts the recovery log, forwards `Task` to +// the leader, and is the *only* shard that receives `Apply` and +// `Persist`. Non-zero shards have no recovery log; they spin up against +// a typically-empty RocksDB and MUST send `Recover` and `Opened` +// messages equal to `default()` at session startup. The leader treats +// any deviation as a hard protocol error — this is the migration guard +// against stale per-shard state from before consolidation. +// +// All shards participate in every transaction (idle shards send empty +// deltas). Shard topology is fail-stop: any shard drop aborts the +// session and the next session re-joins from PRIMARY. +// +// Connector vs leader message naming: The runtime messages overlap by +// name with messages of the connector protocols. We use the short-hand +// `C:Foo` for the connector message and `L:Foo` for the runtime / +// leader message. So `C:Acknowledged` is the connector's response on +// its acknowledge phase; `L:Acknowledged` is what a shard relays back +// to the leader after observing `C:Acknowledged`. +// +// Hinted vs committed Frontier (used by Recover, Persist, Open): +// `committed` state is durable in the recovery log. `hinted` state is +// what the leader *intended* to commit but may or may not yet be +// durable. Remote-authoritative connectors may commit a transaction in +// the *endpoint* ahead of the recovery log: at session startup we +// compare the close-clock embedded in `C:Opened`'s checkpoint against +// our recovered `hinted_close_clock` to detect this case and promote +// the hinted Frontier to committed. +// +// Leader joins a task's shards into a session and drives multi-shard +// transactions. Shard instances dial the leader, which joins them and +// leads them through Recover → Apply → Open → ⟨transactions⟩ → Stopped. +// +// The protocol is "inverted" compared to typical client-server: sometimes +// the shard initiates (Join, Recover, Flushed) and sometimes the leader +// initiates (Open, Load, Flush, Persist). +service Leader { + // rpc Derive(stream .runtime.Derive) returns (stream .runtime.Derive); + rpc Materialize(stream .runtime.Materialize) returns (stream .runtime.Materialize); +} + +// Shard is the controller-facing bidirectional service exposed by +// runtime-next. It carries the same message types as Leader on the +// wire, but is a distinct service: each Shard stream terminates one +// shard's controller-bound protocol and (separately) dials the leader +// sidecar's Leader service, translating between the two and the +// connector RPC. +// +// Stream modes are determined by the first message: +// - Session mode: first message is `SessionLoop`. Then any number of +// leader sessions cycle through Join → ... → Stopped, terminated by +// controller EOF. +// - Unary mode: first message is `spec` or `validate`. The leader is +// not involved; the handler dials a transient connector and streams +// the response back. A Validate may follow a Spec on the same stream. +service Shard { + // rpc Derive(stream .runtime.Derive) returns (stream .runtime.Derive); + rpc Materialize(stream .runtime.Materialize) returns (stream .runtime.Materialize); +} + +// Join is sent by each shard to the leader at session start. The leader +// gathers all expected Joins, verifies `shards` are exactly equal, +// and responds with Joined on consensus or disagreement. +// +// Index 0 in `shards` is "shard zero": its reactor is co-located +// with the Shuffle Leader sidecar that owns this session, and it hosts +// the task's recovery log. +message Join { + // Etcd mod revision at which `shards` was observed. + // On disagreement all participants await the highest revision, then re-try. + int64 etcd_mod_revision = 1; + // Shard describes one shard and the reactor it is assigned to. + // + // Shards are ordered on ascending `shard.range.key_begin` and, + // within a key range, on ascending `shard.range.r_clock_begin`. The + // tiling must be complete and non-overlapping in 2D over + // [key_begin, key_end] and [r_clock_begin, r_clock_end]. + message Shard { + string id = 1; + // Labeling that captures all shard metadata. + ops.ShardLabeling labeling = 2; + // Hosting reactor ID. + protocol.ProcessSpec.ID reactor = 3; + // Creation revision of the Etcd key representing the shard's + // reactor assignment. Represents assignment identity: + // each re-assignment has a unique, monotonic revision. + int64 etcd_create_revision = 4; + } + // Current shard topology as observed by the sender. + repeated Shard shards = 2; + // This shard's index within `shards`. + uint32 shard_index = 3; + // Client-owned local directory where this shuffle Log segments should be written. + string shuffle_directory = 4; + // Dial-able gRPC endpoint for reaching this shard's shuffle service. + string shuffle_endpoint = 5; + // Dial-able gRPC endpoint for reaching the shuffle Leader's shuffle service. + string leader_endpoint = 6; +} + +// Joined is the leader's response to Join, fanned out to every joining shard. +// Each shard then forwards Joined to its controller. +message Joined { + // Zero: consensus reached (success); the session proceeds. + // Non-zero: topology disagreement. Await this revision, then retry. + int64 max_etcd_revision = 1; +} + +// Task which is being processed by the runtime. +// Sent from Controller to Shard, and from Shard zero (only) to Leader +// after Joined. Other shards do not forward Task. +message Task { + // Task specification (protobuf-encoded bytes). + bytes spec = 1; + // Collection journal partition to which task states are written. + string ops_stats_journal = 2; + // Collection to which task stats are written. + flow.CollectionSpec ops_stats_spec = 3; + // When true, documents and stats are written to output and not directed to collections. + bool preview = 4; + // Preview / harness control. Zero means unlimited. + uint32 max_transactions = 5; +} + +// Recover is sent by each shard to the leader after Joined, and carries +// state recovered from the shard's RocksDB. +// +// Shard zero populates Recover from its actual RocksDB scan. Non-zero +// shards have no recovery log: they MUST send `Recover{}` (proto default) +// or the leader fails the session. This is the migration guard: a non-zero +// shard reporting state is evidence of stale per-shard data left over +// from before consolidation, and we'd rather fail loudly than silently +// merge it. +// +// Hinted vs committed. `committed_*` is durable in the recovery log. +// `hinted_*` was *intended* to commit; it is durable on its last write +// but the corresponding transaction may or may not have completed. +// Remote-authoritative connectors may commit a transaction in the +// *endpoint* (during StartCommit/StartedCommit) ahead of the recovery +// log; the leader detects this at startup by comparing the close-clock +// embedded in `C:Opened`'s checkpoint against `hinted_close_clock`, +// and promotes the hinted Frontier to committed when they agree. +message Recover { + // Last-persisted ACK intents. + map ack_intents = 1; + // Clock at which the last-committed transaction closed, + // or zero if never committed. + fixed64 committed_close_clock = 2; + // Committed Frontier entries. Not a delta. + shuffle.Frontier committed_frontier = 3; + // Persisted connector state. + bytes connector_state_json = 4 [json_name = "connectorState"]; + // Clock at which the last hinted transaction closed, or zero if never + // hinted. If equal to `committed_close_clock` the hinted transaction + // committed normally; if ahead of `committed_close_clock` the hinted + // transaction was started but did not commit in the recovery log + // (and may or may not have committed in the connector's endpoint). + fixed64 hinted_close_clock = 5; + // Persisted hinted Frontier entries (FH: range). + shuffle.Frontier hinted_frontier = 6; + // Last-applied task specification (protobuf-encoded bytes), or empty. + bytes last_applied = 7; + // Legacy Checkpoint persisted in RocksDB for roll-forward / roll-back + // capability with the V1 runtime. This will be phased out as we fully + // cut over to the `committed_frontier` representation. + // + // Distinct from the checkpoint returned by C:Opened (see the + // `Recover` message comment above). The C:Opened checkpoint stays; + // only this `legacy_checkpoint` is on the path to removal. + consumer.Checkpoint legacy_checkpoint = 8; + // Per-binding max-key entries. + // Key: binding index; Value: packed composite key tuple. + map max_keys = 9; + // Persisted trigger parameters (materialize only), or empty. + bytes trigger_params_json = 10 [json_name = "triggerParams"]; +} + +// Persist is sent by the leader to shard zero when state must be durably +// written. Each field maps to a contractual WriteBatch effect on shard +// zero's RocksDB. Absent fields are inert. +// +// All fields of a Persist land together in a single WriteBatch. +// `nonce` is echoed back by the shard's Persisted response, allowing +// the leader to match a Persisted response to its originating request. +message Persist { + // Request ID picked by the leader and echoed verbatim in the shard's + // `Persisted` response. The leader chooses any value and the shard + // does not interpret it. + uint64 nonce = 1; + // Delete previously-persisted ACK intents. Applies ahead of `ack_intents`. + // Effect: DeleteRange("AI:") + bool delete_ack_intents = 2; + // ACK intent entries. + // Key: journal name; Value: raw journal content to write. + // Effect: Put under "AI:{journal}". + map ack_intents = 3; + // Clock at which the last-committed transaction closed. + // Effect: Put under "committed-close-clock". + fixed64 committed_close_clock = 4; + // Committed Frontier entries. + // Effect: Put under "FC:..." keys. + shuffle.Frontier committed_frontier = 5; + // Connector state patches. State Update Wire Format. + // Effect: Merge each patch under "connector-state". + bytes connector_patches_json = 6 [json_name = "connectorPatches"]; + // Clock at which the hinted transaction closed. + // Effect: Put under "hinted-close-clock". + fixed64 hinted_close_clock = 7; + // Delete a previously-persisted hinted frontier. Applies ahead of `hinted_frontier`. + // Effect: DeleteRange("FH:") + bool delete_hinted_frontier = 8; + // Hinted Frontier entries. + // Effect: Put under "FH:" keys. + shuffle.Frontier hinted_frontier = 9; + // Last-applied task specification (protobuf-encoded bytes), or empty. + // Effect: Put under "last-applied" key. + bytes last_applied = 10; + // Legacy checkpoint, required for rollback to legacy runtime. + // Effect: Put under "checkpoint" key. + consumer.Checkpoint legacy_checkpoint = 11; + // Per-binding max-key updates, reduced to per-binding maximum across shards. + // Key: binding index; Value: packed composite key tuple. + // Effect: Put value under "MK-v2:{state_key}" (state_key resolved by the encoder). + map max_keys = 12; + // Delete previously-persisted trigger parameters. Applies ahead of `trigger_params_json`. + // Effect: Delete the "trigger-params" key. + bool delete_trigger_params = 13; + // Materialization trigger parameters. + // Effect: Put under "trigger-params" key. + bytes trigger_params_json = 14 [ json_name = "triggerParams" ]; +} + +// Persisted is sent by shard zero to the leader after the state is durable +// in the recovery log. +message Persisted { + // Echoed back from the originating `Persist.nonce` request. + uint64 nonce = 1; +} + +// Apply asks shard zero to invoke its connector's Apply action, both for +// the initial application of a new spec and for re-application after a +// connector returned state patches that the leader has now persisted. +// +// The leader's apply loop iterates Apply / Applied / (Persist / +// Persisted) until the connector returns no further patches; on the +// terminal iteration it Persists `last_applied = spec` and returns to +// the rest of session startup. +message Apply { + // Task specification to be applied (protobuf-encoded bytes). + bytes spec = 1; + // Version of the specification being applied. + string version = 2; + // Last specification which was successfully applied, or empty. + bytes last_spec = 4; + // Version of the last applied specification, or empty. + string last_version = 5; + // Reduced connector state observed by this Apply iteration. + bytes connector_state_json = 6 [ json_name = "connectorStateJson" ]; +} + +message Applied { + // Human-readable description of the action that the connector took. + // If empty, this Apply is to be considered a "no-op". + string action_description = 1; + // Applied connector state patches. State Update Wire Format. + bytes connector_patches_json = 2 [json_name = "connectorPatches"]; +} + +message Open { + // Task specification to be opened (protobuf-encoded bytes). + bytes spec = 1; + // Version of the specification being opened. + string version = 2; + // Range of documents to be processed by this session. + flow.RangeSpec range = 3; + // Last-persisted connector state, post-Apply, from `Recover`. + bytes connector_state_json = 4 [ json_name = "state" ]; + // Materializations only: per-binding maximum keys. + // Key: binding index; Value: packed composite key tuple. + map max_keys = 5; +} + +// CloseNow is sent Controller → Shard → Leader, as a request to +// immediately close a transaction being held open by policy. +message CloseNow {} + +// Stop is sent Controller → Shard → Leader to request graceful shutdown. +message Stop {} + +// Stopped confirms the session has shut down. The leader sends Stopped +// to each shard followed by EOF; each shard then forwards Stopped to its +// controller and EOFs. +message Stopped {} + +// SessionLoop is sent as the first message of a session-loop stream on +// the Shard service. It carries process-level configuration that +// outlives the cycle of leader sessions on this stream. The Leader +// service never sees SessionLoop. +message SessionLoop { + // RocksDB context opened for this Shard stream and reused across all + // of its leader sessions. Absent for non-zero materialize/derive + // shards, which don't host a recovery log. + RocksDBDescriptor rocksdb_descriptor = 1; +} + +// Materialize is the bidirectional message type for materialization +// sessions. Exactly one field is set per message. +message Materialize { + + // ----- UNARY REQUESTS ----- + + // Controller → Shard. Unary request outside of a SessionLoop. + materialize.Request.Spec spec = 1; + // Shard → Controller. Connector's reply to `spec`. + materialize.Response.Spec spec_response = 2; + // Controller → Shard. Unary request outside of a SessionLoop. + materialize.Request.Validate validate = 3; + // Shard → Controller. Connector's reply to `validate`. + materialize.Response.Validated validated = 4; + + // ----- STARTUP ----- + + // Controller → Shard. First message of a session-loop stream; + // never sent to the Leader. + SessionLoop session_loop = 20; + + // Controller → Shard → Leader. Session initiation with topology. + Join join = 21; + + // Leader → Shards → Controllers. Consensus or retry directive + // (`max_etcd_revision == 0` means consensus; non-zero means retry). + Joined joined = 22; + + // Controller → Shard, and Shard zero (only) → Leader. + // Defines the task being processed by the now-joined topology. + Task task = 23; + + // Each Shard → Leader. State recovered from RocksDB on startup. + // Non-zero shards MUST send `Recover{}` (default) — see Recover. + Recover recover = 24; + + // Leader → Shard zero. Re-applied until `Applied` returns no patches. + Apply apply = 25; + + // Shard zero → Leader. C:Apply complete. If `Applied` emits state + // patches the leader runs Persist/Persisted and calls Apply again, + // until it emits no further patches. + Applied applied = 26; + + // Leader → Shards. Open connector and prepare for transactions. + Open open = 27; + + // Each Shard → Leader, and each Shard → Controller. Connector is + // running and session startup is complete. + // + // The shard sends *two* Opened messages per session — one to each + // peer — populated differently: + // - To the leader: `container` is empty; `connector_checkpoint` is + // the C:Opened checkpoint (if any). Non-zero shards MUST send + // `Opened{}` (default) here, mirroring the Recover migration + // guard: a non-empty checkpoint from a non-zero shard is treated + // as evidence of stale per-shard state, and startup fails. + // - To the controller: `container` describes the running connector + // container; `connector_checkpoint` is empty. + message Opened { + // Description of the running connector container. + Container container = 1; + // Optional connector consumer.Checkpoint returned by C:Opened. + // + // Remote-authoritative connectors commit to their endpoint during + // StartCommit, and the leader compares the close-clock embedded in + // this checkpoint against the recovered `hinted_close_clock` to + // detect a commit that landed in the endpoint but not in the + // recovery log. See `Recover` for the full hinted/committed model. + // + // Migration: if no close-clock is embedded the checkpoint is + // assumed to be a V1 legacy checkpoint and replaces the recovered + // committed Frontier. The V2 runtime may write additional checkpoint + // keys for rollback compatibility with V1. + consumer.Checkpoint connector_checkpoint = 2; + } + Opened opened = 28; + + // ----- TRANSACTION PROCESSING ----- + + // Leader → Shards. Incremental Frontier to process into transaction. + // Shards add source docs to their combiner and issue C:Load RPCs for + // documents that may already exist in the endpoint. + // Multiple Load messages may be sent per transaction. + message Load { + shuffle.Frontier frontier = 1; + } + Load load = 40; + + // Shard → Leader. All frontier documents have been loaded into the + // combiner and all C:Load requests sent into the connector. + message Loaded { + // Binding which participated in this Loaded frontier. + message Binding { + // Binding index. + uint32 index = 1; + // Minimum source document Clock observed + fixed64 min_source_clock = 2; + // Maximum source document Clock observed + fixed64 max_source_clock = 3; + // Number of source documents processed. + uint64 sourced_docs_total = 4; + // Number of source document bytes processed. + uint64 sourced_bytes_total = 5; + // When non-empty, a new maximum key was observed. + bytes max_key_delta = 6; + } + repeated Binding bindings = 1; + // On-disk size of this shard's combiner, for transaction close policy. + uint64 combiner_usage_bytes = 2; + } + Loaded loaded = 41; + + // Leader → Shards. Signals end of Load phase. + message Flush { + // Prior transaction's aggregated C:Acknowledged state patches. + // State Update Wire Format. + bytes connector_patches_json = 1 [json_name = "connectorPatches"]; + } + Flush flush = 42; + + // Shard → Leader. Flush phase complete. + // Reports connector state patches and max-key deltas from C:Flushed. + message Flushed { + // Binding which participated in this Flushed phase. + message Binding { + // Binding index. + uint32 index = 1; + // Number of endpoint documents loaded by this binding + // between the last L:Loaded and C:Flushed. + uint64 loaded_docs_total = 2; + // Number of endpoint document bytes loaded by this binding + // between the last L:Loaded and C:Flushed. + uint64 loaded_bytes_total = 3; + } + repeated Binding bindings = 1; + // Connector state patches from this shard's C:Flushed. + // State Update Wire Format. + bytes connector_patches_json = 2 [json_name = "connectorPatches"]; + } + Flushed flushed = 43; + + // Leader → Shards. Idempotency Persist now complete; drain combiners + // into C:Store requests. + message Store {} + Store store = 44; + + // Shard → Leader. Store phase complete; shard is ready to commit. + // Reports final per-binding transaction statistics. + message Stored { + // Binding which participated in this Stored phase. + message Binding { + // Binding index. + uint32 index = 1; + // Number of endpoint documents stored by this binding. + uint64 stored_docs_total = 2; + // Number of endpoint document bytes stored by this binding. + uint64 stored_bytes_total = 3; + } + repeated Binding bindings = 1; + } + Stored stored = 45; + + // Leader → Shards. Begin commit phase. + // Distributes aggregated L:Flushed patches from all shards so each connector + // instance can observe its peers' state for cooperative strategies (such as + // parallel file staging). + message StartCommit { + // Aggregated Flushed connector state patches from all shards. + // State Update Wire Format. + bytes connector_patches_json = 1 [json_name = "connectorPatches"]; + // Transaction Checkpoint for remote-authoritative connectors. + consumer.Checkpoint connector_checkpoint = 2; + } + StartCommit start_commit = 46; + + // Shard → Leader. Commit initiated. + // Remote-authoritative connectors will literally commit prior to return. + message StartedCommit { + // Connector state patches from this shard's C:StartedCommit. + // State Update Wire Format. + bytes connector_patches_json = 1 [json_name = "connectorPatches"]; + } + StartedCommit started_commit = 47; + + // Leader → Shards. Transaction complete. + message Acknowledge { + // Aggregated StartedCommit connector state patches from all shards. + // State Update Wire Format. + bytes connector_patches_json = 1 [json_name = "connectorPatches"]; + } + Acknowledge acknowledge = 48; + + // Shard → Leader. Reports C:Acknowledged state from prior transaction. + // Sent by each shard after its connector responds to C:Acknowledge. + message Acknowledged { + // Connector state patches from this shard's C:Acknowledged. + // State Update Wire Format. + bytes connector_patches_json = 1 [json_name = "connectorPatches"]; + } + Acknowledged acknowledged = 49; + + // Leader → Shard zero. Durably persist state. + Persist persist = 50; + + // Shard zero → Leader. State is durable. + Persisted persisted = 51; + + // Controller → Shard → Leader. Request immediate close of the + // currently-open transaction. + CloseNow close_now = 52; + + // ----- TEARDOWN ----- + + // Controller → Shard → Leader. Graceful shutdown request. + Stop stop = 60; + // Leader → Shards → Controllers. Shutdown confirmed; EOF follows on + // each leg. + Stopped stopped = 61; +} diff --git a/go/protocols/shuffle/shuffle.pb.go b/go/protocols/shuffle/shuffle.pb.go new file mode 100644 index 00000000000..742cc83b623 --- /dev/null +++ b/go/protocols/shuffle/shuffle.pb.go @@ -0,0 +1,7212 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: go/protocols/shuffle/shuffle.proto + +package shuffle + +import ( + context "context" + encoding_binary "encoding/binary" + fmt "fmt" + flow "github.com/estuary/flow/go/protocols/flow" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + protocol "go.gazette.dev/core/broker/protocol" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" + io "io" + math "math" + math_bits "math/bits" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +// Shard represents a participant in the shuffle topology (e.x. a task shard). +type Shard struct { + // Key and r-clock document range owned by this shard. + Range *flow.RangeSpec `protobuf:"bytes,1,opt,name=range,proto3" json:"range,omitempty"` + // gRPC endpoint of this shard's shuffle service. + Endpoint string `protobuf:"bytes,2,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + // Filesystem path where the Log actor writes segment files for this shard. + // The consumer joins over shuffle-produced log segments via this directory. + // Multiple shard indices may share a single directory. + Directory string `protobuf:"bytes,3,opt,name=directory,proto3" json:"directory,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Shard) Reset() { *m = Shard{} } +func (m *Shard) String() string { return proto.CompactTextString(m) } +func (*Shard) ProtoMessage() {} +func (*Shard) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{0} +} +func (m *Shard) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Shard) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Shard.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 *Shard) XXX_Merge(src proto.Message) { + xxx_messageInfo_Shard.Merge(m, src) +} +func (m *Shard) XXX_Size() int { + return m.ProtoSize() +} +func (m *Shard) XXX_DiscardUnknown() { + xxx_messageInfo_Shard.DiscardUnknown(m) +} + +var xxx_messageInfo_Shard proto.InternalMessageInfo + +type CollectionPartitions struct { + // Collection specification. + Collection *flow.CollectionSpec `protobuf:"bytes,1,opt,name=collection,proto3" json:"collection,omitempty"` + // Partition selector for filtering source collection journals. + PartitionSelector *protocol.LabelSelector `protobuf:"bytes,2,opt,name=partition_selector,json=partitionSelector,proto3" json:"partition_selector,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CollectionPartitions) Reset() { *m = CollectionPartitions{} } +func (m *CollectionPartitions) String() string { return proto.CompactTextString(m) } +func (*CollectionPartitions) ProtoMessage() {} +func (*CollectionPartitions) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{1} +} +func (m *CollectionPartitions) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CollectionPartitions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_CollectionPartitions.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 *CollectionPartitions) XXX_Merge(src proto.Message) { + xxx_messageInfo_CollectionPartitions.Merge(m, src) +} +func (m *CollectionPartitions) XXX_Size() int { + return m.ProtoSize() +} +func (m *CollectionPartitions) XXX_DiscardUnknown() { + xxx_messageInfo_CollectionPartitions.DiscardUnknown(m) +} + +var xxx_messageInfo_CollectionPartitions proto.InternalMessageInfo + +// Task which we're performing shuffles for. +type Task struct { + // Types that are valid to be assigned to Task: + // *Task_CollectionPartitions + // *Task_Derivation + // *Task_Materialization + Task isTask_Task `protobuf_oneof:"task"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Task) Reset() { *m = Task{} } +func (m *Task) String() string { return proto.CompactTextString(m) } +func (*Task) ProtoMessage() {} +func (*Task) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{2} +} +func (m *Task) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Task) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Task.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 *Task) XXX_Merge(src proto.Message) { + xxx_messageInfo_Task.Merge(m, src) +} +func (m *Task) XXX_Size() int { + return m.ProtoSize() +} +func (m *Task) XXX_DiscardUnknown() { + xxx_messageInfo_Task.DiscardUnknown(m) +} + +var xxx_messageInfo_Task proto.InternalMessageInfo + +type isTask_Task interface { + isTask_Task() + MarshalTo([]byte) (int, error) + ProtoSize() int +} + +type Task_CollectionPartitions struct { + CollectionPartitions *CollectionPartitions `protobuf:"bytes,1,opt,name=collection_partitions,json=collectionPartitions,proto3,oneof" json:"collection_partitions,omitempty"` +} +type Task_Derivation struct { + Derivation *flow.CollectionSpec `protobuf:"bytes,2,opt,name=derivation,proto3,oneof" json:"derivation,omitempty"` +} +type Task_Materialization struct { + Materialization *flow.MaterializationSpec `protobuf:"bytes,3,opt,name=materialization,proto3,oneof" json:"materialization,omitempty"` +} + +func (*Task_CollectionPartitions) isTask_Task() {} +func (*Task_Derivation) isTask_Task() {} +func (*Task_Materialization) isTask_Task() {} + +func (m *Task) GetTask() isTask_Task { + if m != nil { + return m.Task + } + return nil +} + +func (m *Task) GetCollectionPartitions() *CollectionPartitions { + if x, ok := m.GetTask().(*Task_CollectionPartitions); ok { + return x.CollectionPartitions + } + return nil +} + +func (m *Task) GetDerivation() *flow.CollectionSpec { + if x, ok := m.GetTask().(*Task_Derivation); ok { + return x.Derivation + } + return nil +} + +func (m *Task) GetMaterialization() *flow.MaterializationSpec { + if x, ok := m.GetTask().(*Task_Materialization); ok { + return x.Materialization + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Task) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Task_CollectionPartitions)(nil), + (*Task_Derivation)(nil), + (*Task_Materialization)(nil), + } +} + +// ProducerFrontier is the frontier state of a single producer within a journal. +type ProducerFrontier struct { + // Producer ID, extracted from document UUID. Only the low 6 bytes are used. + Producer int64 `protobuf:"varint,1,opt,name=producer,proto3" json:"producer,omitempty"` + // Clock of the last committing ACK_TXN or OUTSIDE_TXN. + LastCommit uint64 `protobuf:"fixed64,2,opt,name=last_commit,json=lastCommit,proto3" json:"last_commit,omitempty"` + // Clock of the last hinted (causal) commit, or zero if no hint. + HintedCommit uint64 `protobuf:"fixed64,3,opt,name=hinted_commit,json=hintedCommit,proto3" json:"hinted_commit,omitempty"` + // Journal byte offset, sign-encoded: + // - Non-negative: begin offset of first pending CONTINUE_TXN. + // - Negative: negation of end offset of last committing ACK_TXN / OUTSIDE_TXN. + Offset int64 `protobuf:"varint,4,opt,name=offset,proto3" json:"offset,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ProducerFrontier) Reset() { *m = ProducerFrontier{} } +func (m *ProducerFrontier) String() string { return proto.CompactTextString(m) } +func (*ProducerFrontier) ProtoMessage() {} +func (*ProducerFrontier) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{3} +} +func (m *ProducerFrontier) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProducerFrontier) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ProducerFrontier.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 *ProducerFrontier) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProducerFrontier.Merge(m, src) +} +func (m *ProducerFrontier) XXX_Size() int { + return m.ProtoSize() +} +func (m *ProducerFrontier) XXX_DiscardUnknown() { + xxx_messageInfo_ProducerFrontier.DiscardUnknown(m) +} + +var xxx_messageInfo_ProducerFrontier proto.InternalMessageInfo + +// JournalFrontier is the frontier state for a single journal under a specific binding. +// It uses delta-encoding for journal names: given a preceding journal name, +// truncate `journal_name_truncate_delta` bytes then append `journal_name_suffix`. +// +// At a given time, the set of distinct producers writing to a journal is small. +// However, Producer IDs come and go over time, and the historical set of producers +// who have EVER written to a journal may be large. +// +// Most JournalFrontier focuses on _deltas_ of frontier state, and have few producers. +// The exception is when a Coordinator client is streaming in a resume checkpoint. +type JournalFrontier struct { + // Number of bytes to truncate from the preceding name. + // Must align with a UTF-8 code point boundary. + JournalNameTruncateDelta int32 `protobuf:"varint,1,opt,name=journal_name_truncate_delta,json=journalNameTruncateDelta,proto3" json:"journal_name_truncate_delta,omitempty"` + // Suffix to append to the preceding, truncated name. + JournalNameSuffix string `protobuf:"bytes,2,opt,name=journal_name_suffix,json=journalNameSuffix,proto3" json:"journal_name_suffix,omitempty"` + // Binding index under which the journal is read. + // When persisting across sessions, this should be mapped via the task binding's + // `journal_read_suffix` to ensure stability across task versions. + Binding uint32 `protobuf:"varint,3,opt,name=binding,proto3" json:"binding,omitempty"` + // Delta of journal bytes read since the last checkpoint. + // Summed during reduction. + BytesReadDelta int64 `protobuf:"varint,4,opt,name=bytes_read_delta,json=bytesReadDelta,proto3" json:"bytes_read_delta,omitempty"` + // Delta of bytes-behind (write_head - read_offset) since last checkpoint. + // Positive when the reader is falling behind, negative when catching up. + // Summed during reduction. + BytesBehindDelta int64 `protobuf:"varint,5,opt,name=bytes_behind_delta,json=bytesBehindDelta,proto3" json:"bytes_behind_delta,omitempty"` + // Producers of this journal. + // Sorted and unique on `producer`. + Producers []*ProducerFrontier `protobuf:"bytes,6,rep,name=producers,proto3" json:"producers,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *JournalFrontier) Reset() { *m = JournalFrontier{} } +func (m *JournalFrontier) String() string { return proto.CompactTextString(m) } +func (*JournalFrontier) ProtoMessage() {} +func (*JournalFrontier) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{4} +} +func (m *JournalFrontier) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *JournalFrontier) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_JournalFrontier.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 *JournalFrontier) XXX_Merge(src proto.Message) { + xxx_messageInfo_JournalFrontier.Merge(m, src) +} +func (m *JournalFrontier) XXX_Size() int { + return m.ProtoSize() +} +func (m *JournalFrontier) XXX_DiscardUnknown() { + xxx_messageInfo_JournalFrontier.DiscardUnknown(m) +} + +var xxx_messageInfo_JournalFrontier proto.InternalMessageInfo + +// Frontier is a complete frontier (or frontier delta) of journal progress. +// Entries are sorted and unique on (journal name, binding). +type Frontier struct { + Journals []*JournalFrontier `protobuf:"bytes,1,rep,name=journals,proto3" json:"journals,omitempty"` + // Per-shard flushed LSN, indexed by shard_index. + FlushedLsn []uint64 `protobuf:"varint,2,rep,packed,name=flushed_lsn,json=flushedLsn,proto3" json:"flushed_lsn,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Frontier) Reset() { *m = Frontier{} } +func (m *Frontier) String() string { return proto.CompactTextString(m) } +func (*Frontier) ProtoMessage() {} +func (*Frontier) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{5} +} +func (m *Frontier) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Frontier) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Frontier.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 *Frontier) XXX_Merge(src proto.Message) { + xxx_messageInfo_Frontier.Merge(m, src) +} +func (m *Frontier) XXX_Size() int { + return m.ProtoSize() +} +func (m *Frontier) XXX_DiscardUnknown() { + xxx_messageInfo_Frontier.DiscardUnknown(m) +} + +var xxx_messageInfo_Frontier proto.InternalMessageInfo + +// SessionRequest is sent by the Coordinator to manage the shuffle session. +type SessionRequest struct { + Open *SessionRequest_Open `protobuf:"bytes,1,opt,name=open,proto3" json:"open,omitempty"` + // The resume checkpoint: the non-delta frontier from which the + // session is to resume. It's sent by the Coordinator client after reading + // SessionResponse.Opened. + // + // This is a comprehensive checkpoint, reflecting all journals and producers + // which have committed transactions. There may be a *lot* of historical producers + // in long-lived tasks, and the Coordinator should employ regular "pruning" to + // remove producers having a `last_commit` clock which is far older than the + // latest `last_commit` of a peer journal producer. Such producers are assumed + // to have been retired and will produce no further transactions. + // + // Resume checkpoint producers use `hinted_commit` to represent read-through state: + // transactions that were prepared but not yet committed during the previous + // session. The Session will read through these and emit the read-through + // frontier as the first NextCheckpoint response. + // + // The Coordinator should use a last unfinished NextCheckpoint of a prior + // Session to initialize `hinted_commit` of its resume checkpoint, by mapping + // an incomplete producer `last_commit` to `hinted_commit`. This ensures that + // the first NextCheckpoint of the current session will match the last + // unfinished NextCheckpoint of its prior session, which enables the transaction + // to be idempotent. + ResumeCheckpoint *Frontier `protobuf:"bytes,2,opt,name=resume_checkpoint,json=resumeCheckpoint,proto3" json:"resume_checkpoint,omitempty"` + NextCheckpoint *SessionRequest_NextCheckpoint `protobuf:"bytes,3,opt,name=next_checkpoint,json=nextCheckpoint,proto3" json:"next_checkpoint,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SessionRequest) Reset() { *m = SessionRequest{} } +func (m *SessionRequest) String() string { return proto.CompactTextString(m) } +func (*SessionRequest) ProtoMessage() {} +func (*SessionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{6} +} +func (m *SessionRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SessionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SessionRequest.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 *SessionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SessionRequest.Merge(m, src) +} +func (m *SessionRequest) XXX_Size() int { + return m.ProtoSize() +} +func (m *SessionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SessionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SessionRequest proto.InternalMessageInfo + +// Open initiates a new shuffle session. +// Sent once at the start of the Session RPC. +type SessionRequest_Open struct { + // Task for which we're performing shuffles. + Task *Task `protobuf:"bytes,2,opt,name=task,proto3" json:"task,omitempty"` + // Shards participating in this shuffle session. + Shards []*Shard `protobuf:"bytes,3,rep,name=shards,proto3" json:"shards,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SessionRequest_Open) Reset() { *m = SessionRequest_Open{} } +func (m *SessionRequest_Open) String() string { return proto.CompactTextString(m) } +func (*SessionRequest_Open) ProtoMessage() {} +func (*SessionRequest_Open) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{6, 0} +} +func (m *SessionRequest_Open) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SessionRequest_Open) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SessionRequest_Open.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 *SessionRequest_Open) XXX_Merge(src proto.Message) { + xxx_messageInfo_SessionRequest_Open.Merge(m, src) +} +func (m *SessionRequest_Open) XXX_Size() int { + return m.ProtoSize() +} +func (m *SessionRequest_Open) XXX_DiscardUnknown() { + xxx_messageInfo_SessionRequest_Open.DiscardUnknown(m) +} + +var xxx_messageInfo_SessionRequest_Open proto.InternalMessageInfo + +// NextCheckpoint requests the next available checkpoint delta. +// This is a blocking request: the Session only responds when progress is +// available. The Coordinator client requests a next checkpoint at times of +// its choosing (e.g., after completing processing of the previous checkpoint). +// +// A NextCheckpoint response may be a fully-resolved checkpoint OR a "peek" +// of an in-flight frontier whose causal hints have not yet fully resolved. +// See SessionResponse.next_checkpoint for the contract. +type SessionRequest_NextCheckpoint struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SessionRequest_NextCheckpoint) Reset() { *m = SessionRequest_NextCheckpoint{} } +func (m *SessionRequest_NextCheckpoint) String() string { return proto.CompactTextString(m) } +func (*SessionRequest_NextCheckpoint) ProtoMessage() {} +func (*SessionRequest_NextCheckpoint) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{6, 1} +} +func (m *SessionRequest_NextCheckpoint) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SessionRequest_NextCheckpoint) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SessionRequest_NextCheckpoint.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 *SessionRequest_NextCheckpoint) XXX_Merge(src proto.Message) { + xxx_messageInfo_SessionRequest_NextCheckpoint.Merge(m, src) +} +func (m *SessionRequest_NextCheckpoint) XXX_Size() int { + return m.ProtoSize() +} +func (m *SessionRequest_NextCheckpoint) XXX_DiscardUnknown() { + xxx_messageInfo_SessionRequest_NextCheckpoint.DiscardUnknown(m) +} + +var xxx_messageInfo_SessionRequest_NextCheckpoint proto.InternalMessageInfo + +// SessionResponse is sent by the Session to the Coordinator. +type SessionResponse struct { + Opened *SessionResponse_Opened `protobuf:"bytes,1,opt,name=opened,proto3" json:"opened,omitempty"` + // The next frontier delta (atop the resume checkpoint), sent in response to + // SessionRequest.NextCheckpoint. Never empty (though the Session may block + // indefinitely until progress is available). + // + // The response may be either (a) a fully-resolved checkpoint, where all + // causal hints in the delta are resolved (every ProducerFrontier has + // `last_commit >= hinted_commit`) — this is a transactional boundary + // suitable for commit; or (b) a "peek" of an in-flight frontier, where + // at least one ProducerFrontier still has `hinted_commit > last_commit`, + // signaling that resolution has made progress but is incomplete. Peeks + // let the client begin processing partial progress (e.g. release log + // segments) without waiting for full resolution. Peeks carry zeroed + // `bytes_*_delta` fields; those byte deltas are surfaced exactly once + // on the eventual fully-resolved checkpoint. + // + // For a transactional boundary the Coordinator MUST keep calling + // NextCheckpoint until it receives a fully-resolved checkpoint (no + // ProducerFrontier with `hinted_commit > last_commit`). + // + // The Coordinator client should retain the checkpoint and use it to + // initialize `hinted_commit` of future session resume checkpoints. Then, + // upon its durable completion of all downstream processing related to a + // fully-resolved NextCheckpoint, it should merge it into its base checkpoint. + NextCheckpoint *Frontier `protobuf:"bytes,2,opt,name=next_checkpoint,json=nextCheckpoint,proto3" json:"next_checkpoint,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SessionResponse) Reset() { *m = SessionResponse{} } +func (m *SessionResponse) String() string { return proto.CompactTextString(m) } +func (*SessionResponse) ProtoMessage() {} +func (*SessionResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{7} +} +func (m *SessionResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SessionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SessionResponse.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 *SessionResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SessionResponse.Merge(m, src) +} +func (m *SessionResponse) XXX_Size() int { + return m.ProtoSize() +} +func (m *SessionResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SessionResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SessionResponse proto.InternalMessageInfo + +// Opened confirms the Session is ready. +// Sent after all Slices have responded Opened. +type SessionResponse_Opened struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SessionResponse_Opened) Reset() { *m = SessionResponse_Opened{} } +func (m *SessionResponse_Opened) String() string { return proto.CompactTextString(m) } +func (*SessionResponse_Opened) ProtoMessage() {} +func (*SessionResponse_Opened) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{7, 0} +} +func (m *SessionResponse_Opened) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SessionResponse_Opened) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SessionResponse_Opened.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 *SessionResponse_Opened) XXX_Merge(src proto.Message) { + xxx_messageInfo_SessionResponse_Opened.Merge(m, src) +} +func (m *SessionResponse_Opened) XXX_Size() int { + return m.ProtoSize() +} +func (m *SessionResponse_Opened) XXX_DiscardUnknown() { + xxx_messageInfo_SessionResponse_Opened.DiscardUnknown(m) +} + +var xxx_messageInfo_SessionResponse_Opened proto.InternalMessageInfo + +// SliceRequest is sent by the Session to each shard's Slice RPC. +type SliceRequest struct { + Open *SliceRequest_Open `protobuf:"bytes,1,opt,name=open,proto3" json:"open,omitempty"` + Start *SliceRequest_Start `protobuf:"bytes,2,opt,name=start,proto3" json:"start,omitempty"` + StartRead *SliceRequest_StartRead `protobuf:"bytes,3,opt,name=start_read,json=startRead,proto3" json:"start_read,omitempty"` + Progress *SliceRequest_Progress `protobuf:"bytes,4,opt,name=progress,proto3" json:"progress,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SliceRequest) Reset() { *m = SliceRequest{} } +func (m *SliceRequest) String() string { return proto.CompactTextString(m) } +func (*SliceRequest) ProtoMessage() {} +func (*SliceRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{8} +} +func (m *SliceRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SliceRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SliceRequest.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 *SliceRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SliceRequest.Merge(m, src) +} +func (m *SliceRequest) XXX_Size() int { + return m.ProtoSize() +} +func (m *SliceRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SliceRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SliceRequest proto.InternalMessageInfo + +// Open initiates the Slice. +// The Slice opens Log RPCs to all shards before responding Opened. +type SliceRequest_Open struct { + // Session ID, generated by the Session actor for tracing correlation. + SessionId uint32 `protobuf:"fixed32,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"` + // Task for which we're performing shuffles. + Task *Task `protobuf:"bytes,2,opt,name=task,proto3" json:"task,omitempty"` + // Shards participating in this shuffle session. + // Must be the same shards and order as Session.Open. + Shards []*Shard `protobuf:"bytes,3,rep,name=shards,proto3" json:"shards,omitempty"` + // Index of this shard within the shards list. + ShardIndex uint32 `protobuf:"varint,4,opt,name=shard_index,json=shardIndex,proto3" json:"shard_index,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SliceRequest_Open) Reset() { *m = SliceRequest_Open{} } +func (m *SliceRequest_Open) String() string { return proto.CompactTextString(m) } +func (*SliceRequest_Open) ProtoMessage() {} +func (*SliceRequest_Open) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{8, 0} +} +func (m *SliceRequest_Open) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SliceRequest_Open) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SliceRequest_Open.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 *SliceRequest_Open) XXX_Merge(src proto.Message) { + xxx_messageInfo_SliceRequest_Open.Merge(m, src) +} +func (m *SliceRequest_Open) XXX_Size() int { + return m.ProtoSize() +} +func (m *SliceRequest_Open) XXX_DiscardUnknown() { + xxx_messageInfo_SliceRequest_Open.DiscardUnknown(m) +} + +var xxx_messageInfo_SliceRequest_Open proto.InternalMessageInfo + +// Start is sent after all Slices have responded Opened and the Session has +// received the resume checkpoint frontier from the Coordinator. +// On receipt, Slices start listing watches and may send SliceResponse.ListingAdded. +type SliceRequest_Start struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SliceRequest_Start) Reset() { *m = SliceRequest_Start{} } +func (m *SliceRequest_Start) String() string { return proto.CompactTextString(m) } +func (*SliceRequest_Start) ProtoMessage() {} +func (*SliceRequest_Start) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{8, 1} +} +func (m *SliceRequest_Start) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SliceRequest_Start) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SliceRequest_Start.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 *SliceRequest_Start) XXX_Merge(src proto.Message) { + xxx_messageInfo_SliceRequest_Start.Merge(m, src) +} +func (m *SliceRequest_Start) XXX_Size() int { + return m.ProtoSize() +} +func (m *SliceRequest_Start) XXX_DiscardUnknown() { + xxx_messageInfo_SliceRequest_Start.DiscardUnknown(m) +} + +var xxx_messageInfo_SliceRequest_Start proto.InternalMessageInfo + +// StartRead instructs the Slice to begin reading a journal. +type SliceRequest_StartRead struct { + // Binding index within the task (derivation transform or + // materialization binding). + Binding uint32 `protobuf:"varint,1,opt,name=binding,proto3" json:"binding,omitempty"` + // Journal to be read. + Spec *protocol.JournalSpec `protobuf:"bytes,2,opt,name=spec,proto3" json:"spec,omitempty"` + // Etcd revision at which the journal was created. + CreateRevision int64 `protobuf:"varint,3,opt,name=create_revision,json=createRevision,proto3" json:"create_revision,omitempty"` + // Etcd revision at which the journal was last modified. + ModRevision int64 `protobuf:"varint,4,opt,name=mod_revision,json=modRevision,proto3" json:"mod_revision,omitempty"` + // Current route of the journal, as an initial advisory hint. + Route *protocol.Route `protobuf:"bytes,5,opt,name=route,proto3" json:"route,omitempty"` + // Producers of this journal as-of the resume checkpoint frontier. + Checkpoint []*ProducerFrontier `protobuf:"bytes,6,rep,name=checkpoint,proto3" json:"checkpoint,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SliceRequest_StartRead) Reset() { *m = SliceRequest_StartRead{} } +func (m *SliceRequest_StartRead) String() string { return proto.CompactTextString(m) } +func (*SliceRequest_StartRead) ProtoMessage() {} +func (*SliceRequest_StartRead) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{8, 2} +} +func (m *SliceRequest_StartRead) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SliceRequest_StartRead) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SliceRequest_StartRead.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 *SliceRequest_StartRead) XXX_Merge(src proto.Message) { + xxx_messageInfo_SliceRequest_StartRead.Merge(m, src) +} +func (m *SliceRequest_StartRead) XXX_Size() int { + return m.ProtoSize() +} +func (m *SliceRequest_StartRead) XXX_DiscardUnknown() { + xxx_messageInfo_SliceRequest_StartRead.DiscardUnknown(m) +} + +var xxx_messageInfo_SliceRequest_StartRead proto.InternalMessageInfo + +// Progress asks the Slice to report its frontier of queued progress. +// The Session maintains exactly one Progress/Progressed cycle in-flight per Slice. +// The Slice responds once at least one Flush/Flushed cycle has completed +// since its last Progressed response. If queued progress is already +// available, the response is immediate. +type SliceRequest_Progress struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SliceRequest_Progress) Reset() { *m = SliceRequest_Progress{} } +func (m *SliceRequest_Progress) String() string { return proto.CompactTextString(m) } +func (*SliceRequest_Progress) ProtoMessage() {} +func (*SliceRequest_Progress) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{8, 3} +} +func (m *SliceRequest_Progress) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SliceRequest_Progress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SliceRequest_Progress.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 *SliceRequest_Progress) XXX_Merge(src proto.Message) { + xxx_messageInfo_SliceRequest_Progress.Merge(m, src) +} +func (m *SliceRequest_Progress) XXX_Size() int { + return m.ProtoSize() +} +func (m *SliceRequest_Progress) XXX_DiscardUnknown() { + xxx_messageInfo_SliceRequest_Progress.DiscardUnknown(m) +} + +var xxx_messageInfo_SliceRequest_Progress proto.InternalMessageInfo + +// SliceResponse is sent by each Slice back to the Session. +type SliceResponse struct { + Opened *SliceResponse_Opened `protobuf:"bytes,1,opt,name=opened,proto3" json:"opened,omitempty"` + ListingAdded *SliceResponse_ListingAdded `protobuf:"bytes,2,opt,name=listing_added,json=listingAdded,proto3" json:"listing_added,omitempty"` + // Progressed reports the frontier of log progress since the last Progressed. + // It's sent in response to a Request.Progress, and only once at least one + // flush cycle has completed since the prior Progressed. + Progressed *Frontier `protobuf:"bytes,3,opt,name=progressed,proto3" json:"progressed,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SliceResponse) Reset() { *m = SliceResponse{} } +func (m *SliceResponse) String() string { return proto.CompactTextString(m) } +func (*SliceResponse) ProtoMessage() {} +func (*SliceResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{9} +} +func (m *SliceResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SliceResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SliceResponse.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 *SliceResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SliceResponse.Merge(m, src) +} +func (m *SliceResponse) XXX_Size() int { + return m.ProtoSize() +} +func (m *SliceResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SliceResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SliceResponse proto.InternalMessageInfo + +// Opened confirms the Slice is ready. +// Sent after all Log RPCs have responded Opened. +type SliceResponse_Opened struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SliceResponse_Opened) Reset() { *m = SliceResponse_Opened{} } +func (m *SliceResponse_Opened) String() string { return proto.CompactTextString(m) } +func (*SliceResponse_Opened) ProtoMessage() {} +func (*SliceResponse_Opened) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{9, 0} +} +func (m *SliceResponse_Opened) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SliceResponse_Opened) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SliceResponse_Opened.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 *SliceResponse_Opened) XXX_Merge(src proto.Message) { + xxx_messageInfo_SliceResponse_Opened.Merge(m, src) +} +func (m *SliceResponse_Opened) XXX_Size() int { + return m.ProtoSize() +} +func (m *SliceResponse_Opened) XXX_DiscardUnknown() { + xxx_messageInfo_SliceResponse_Opened.DiscardUnknown(m) +} + +var xxx_messageInfo_SliceResponse_Opened proto.InternalMessageInfo + +type SliceResponse_ListingAdded struct { + // Binding index of the listing. + Binding uint32 `protobuf:"varint,1,opt,name=binding,proto3" json:"binding,omitempty"` + // Journal which was added to the listing. + Spec *protocol.JournalSpec `protobuf:"bytes,2,opt,name=spec,proto3" json:"spec,omitempty"` + // Etcd revision at which the journal was created. + CreateRevision int64 `protobuf:"varint,3,opt,name=create_revision,json=createRevision,proto3" json:"create_revision,omitempty"` + // Etcd revision at which the journal was last modified. + ModRevision int64 `protobuf:"varint,4,opt,name=mod_revision,json=modRevision,proto3" json:"mod_revision,omitempty"` + // Current route of the journal, as an initial advisory hint. + Route *protocol.Route `protobuf:"bytes,5,opt,name=route,proto3" json:"route,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SliceResponse_ListingAdded) Reset() { *m = SliceResponse_ListingAdded{} } +func (m *SliceResponse_ListingAdded) String() string { return proto.CompactTextString(m) } +func (*SliceResponse_ListingAdded) ProtoMessage() {} +func (*SliceResponse_ListingAdded) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{9, 1} +} +func (m *SliceResponse_ListingAdded) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SliceResponse_ListingAdded) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SliceResponse_ListingAdded.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 *SliceResponse_ListingAdded) XXX_Merge(src proto.Message) { + xxx_messageInfo_SliceResponse_ListingAdded.Merge(m, src) +} +func (m *SliceResponse_ListingAdded) XXX_Size() int { + return m.ProtoSize() +} +func (m *SliceResponse_ListingAdded) XXX_DiscardUnknown() { + xxx_messageInfo_SliceResponse_ListingAdded.DiscardUnknown(m) +} + +var xxx_messageInfo_SliceResponse_ListingAdded proto.InternalMessageInfo + +// LogRequest is sent by Slices to each shard's Log. +type LogRequest struct { + Open *LogRequest_Open `protobuf:"bytes,1,opt,name=open,proto3" json:"open,omitempty"` + Append *LogRequest_Append `protobuf:"bytes,2,opt,name=append,proto3" json:"append,omitempty"` + Flush *LogRequest_Flush `protobuf:"bytes,3,opt,name=flush,proto3" json:"flush,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LogRequest) Reset() { *m = LogRequest{} } +func (m *LogRequest) String() string { return proto.CompactTextString(m) } +func (*LogRequest) ProtoMessage() {} +func (*LogRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{10} +} +func (m *LogRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LogRequest.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 *LogRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogRequest.Merge(m, src) +} +func (m *LogRequest) XXX_Size() int { + return m.ProtoSize() +} +func (m *LogRequest) XXX_DiscardUnknown() { + xxx_messageInfo_LogRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_LogRequest proto.InternalMessageInfo + +// Open initiates the Log RPC. +// Multiple Slices open Log RPCs to the same shard, and the Log +// task joins these streams and processes documents in merged order. +type LogRequest_Open struct { + // Session ID for correlating streams from the same session. + SessionId uint32 `protobuf:"fixed32,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"` + // Shards participating in this shuffle session. + // Must be the same shards and order as Session.Open. + Shards []*Shard `protobuf:"bytes,2,rep,name=shards,proto3" json:"shards,omitempty"` + // Index of the source Slice shard within the session's shard list. + SliceShardIndex uint32 `protobuf:"varint,3,opt,name=slice_shard_index,json=sliceShardIndex,proto3" json:"slice_shard_index,omitempty"` + // Index of the target Log shard within the session's shard list. + LogShardIndex uint32 `protobuf:"varint,4,opt,name=log_shard_index,json=logShardIndex,proto3" json:"log_shard_index,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LogRequest_Open) Reset() { *m = LogRequest_Open{} } +func (m *LogRequest_Open) String() string { return proto.CompactTextString(m) } +func (*LogRequest_Open) ProtoMessage() {} +func (*LogRequest_Open) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{10, 0} +} +func (m *LogRequest_Open) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogRequest_Open) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LogRequest_Open.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 *LogRequest_Open) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogRequest_Open.Merge(m, src) +} +func (m *LogRequest_Open) XXX_Size() int { + return m.ProtoSize() +} +func (m *LogRequest_Open) XXX_DiscardUnknown() { + xxx_messageInfo_LogRequest_Open.DiscardUnknown(m) +} + +var xxx_messageInfo_LogRequest_Open proto.InternalMessageInfo + +// Append sends a document to be written to the log. +// The Log actor merges across Slice streams, ordering by (priority, clock). +type LogRequest_Append struct { + // Number of bytes to truncate from the preceding name. + // Must align with a UTF-8 code point boundary. + JournalNameTruncateDelta int32 `protobuf:"varint,1,opt,name=journal_name_truncate_delta,json=journalNameTruncateDelta,proto3" json:"journal_name_truncate_delta,omitempty"` + // Suffix to append to the preceding, truncated name. + JournalNameSuffix string `protobuf:"bytes,2,opt,name=journal_name_suffix,json=journalNameSuffix,proto3" json:"journal_name_suffix,omitempty"` + // Binding index for this document. + Binding uint32 `protobuf:"varint,3,opt,name=binding,proto3" json:"binding,omitempty"` + // Priority of this binding. + Priority uint32 `protobuf:"varint,4,opt,name=priority,proto3" json:"priority,omitempty"` + // Read delay of the binding, as a uuid::Clock duration. + // The Log actor applies adjusted_clock = clock + read_delay for merge ordering. + // Zero (the common case) means no delay. + ReadDelay uint64 `protobuf:"varint,5,opt,name=read_delay,json=readDelay,proto3" json:"read_delay,omitempty"` + // Producer of the document, extracted from its UUID. + Producer int64 `protobuf:"varint,6,opt,name=producer,proto3" json:"producer,omitempty"` + // Publication clock of the document, extracted from its UUID. + Clock uint64 `protobuf:"fixed64,7,opt,name=clock,proto3" json:"clock,omitempty"` + // Publication flags of the document, extracted from its UUID. + // Bit 15 (FLAGS_SCHEMA_VALID = 0x8000) is set when the document passes schema validation. + // Bits 0-9 carry UUID transaction flags (OUTSIDE_TXN, CONTINUE_TXN, etc.). + Flags uint32 `protobuf:"varint,8,opt,name=flags,proto3" json:"flags,omitempty"` + // Packed shuffle key of the document. + PackedKey []byte `protobuf:"bytes,9,opt,name=packed_key,json=packedKey,proto3" json:"packed_key,omitempty"` + // Document content as transcoded ArchivedNode bytes. + DocArchived []byte `protobuf:"bytes,10,opt,name=doc_archived,json=docArchived,proto3" json:"doc_archived,omitempty"` + // Byte length of this document in its source journal (end_offset - begin_offset). + // Always fits in uint32 because a single document is always < 4 GiB. + SourceByteLength uint32 `protobuf:"varint,11,opt,name=source_byte_length,json=sourceByteLength,proto3" json:"source_byte_length,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LogRequest_Append) Reset() { *m = LogRequest_Append{} } +func (m *LogRequest_Append) String() string { return proto.CompactTextString(m) } +func (*LogRequest_Append) ProtoMessage() {} +func (*LogRequest_Append) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{10, 1} +} +func (m *LogRequest_Append) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogRequest_Append) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LogRequest_Append.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 *LogRequest_Append) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogRequest_Append.Merge(m, src) +} +func (m *LogRequest_Append) XXX_Size() int { + return m.ProtoSize() +} +func (m *LogRequest_Append) XXX_DiscardUnknown() { + xxx_messageInfo_LogRequest_Append.DiscardUnknown(m) +} + +var xxx_messageInfo_LogRequest_Append proto.InternalMessageInfo + +// Flush requests a durability barrier. +type LogRequest_Flush struct { + // Cycle number for correlating Flush with Flushed response. + // Allows pipelining if we later want multiple in-flight flushes. + Cycle uint64 `protobuf:"varint,1,opt,name=cycle,proto3" json:"cycle,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LogRequest_Flush) Reset() { *m = LogRequest_Flush{} } +func (m *LogRequest_Flush) String() string { return proto.CompactTextString(m) } +func (*LogRequest_Flush) ProtoMessage() {} +func (*LogRequest_Flush) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{10, 2} +} +func (m *LogRequest_Flush) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogRequest_Flush) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LogRequest_Flush.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 *LogRequest_Flush) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogRequest_Flush.Merge(m, src) +} +func (m *LogRequest_Flush) XXX_Size() int { + return m.ProtoSize() +} +func (m *LogRequest_Flush) XXX_DiscardUnknown() { + xxx_messageInfo_LogRequest_Flush.DiscardUnknown(m) +} + +var xxx_messageInfo_LogRequest_Flush proto.InternalMessageInfo + +// LogResponse is sent by the Log actor back to each Slice. +type LogResponse struct { + Opened *LogResponse_Opened `protobuf:"bytes,1,opt,name=opened,proto3" json:"opened,omitempty"` + Flushed *LogResponse_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 *LogResponse) Reset() { *m = LogResponse{} } +func (m *LogResponse) String() string { return proto.CompactTextString(m) } +func (*LogResponse) ProtoMessage() {} +func (*LogResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{11} +} +func (m *LogResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LogResponse.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 *LogResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogResponse.Merge(m, src) +} +func (m *LogResponse) XXX_Size() int { + return m.ProtoSize() +} +func (m *LogResponse) XXX_DiscardUnknown() { + xxx_messageInfo_LogResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_LogResponse proto.InternalMessageInfo + +// Opened confirms the Log is ready to receive documents. +// Sent after this Log RPC has joined over the shard's single session Log actor. +type LogResponse_Opened struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LogResponse_Opened) Reset() { *m = LogResponse_Opened{} } +func (m *LogResponse_Opened) String() string { return proto.CompactTextString(m) } +func (*LogResponse_Opened) ProtoMessage() {} +func (*LogResponse_Opened) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{11, 0} +} +func (m *LogResponse_Opened) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogResponse_Opened) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LogResponse_Opened.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 *LogResponse_Opened) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogResponse_Opened.Merge(m, src) +} +func (m *LogResponse_Opened) XXX_Size() int { + return m.ProtoSize() +} +func (m *LogResponse_Opened) XXX_DiscardUnknown() { + xxx_messageInfo_LogResponse_Opened.DiscardUnknown(m) +} + +var xxx_messageInfo_LogResponse_Opened proto.InternalMessageInfo + +// Flushed confirms all preceding documents are visible for dequeue. +// Only after receiving Flushed from ALL Log RPCs can a Slice safely +// report ProgressDelta to the Session. +type LogResponse_Flushed struct { + // Cycle number from the corresponding Flush request. + Cycle uint64 `protobuf:"varint,1,opt,name=cycle,proto3" json:"cycle,omitempty"` + // Log head position after all preceding appends are durable. + // The consumer must read through this LSN to observe all + // documents appended before this flush. + FlushedLsn uint64 `protobuf:"varint,2,opt,name=flushed_lsn,json=flushedLsn,proto3" json:"flushed_lsn,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LogResponse_Flushed) Reset() { *m = LogResponse_Flushed{} } +func (m *LogResponse_Flushed) String() string { return proto.CompactTextString(m) } +func (*LogResponse_Flushed) ProtoMessage() {} +func (*LogResponse_Flushed) Descriptor() ([]byte, []int) { + return fileDescriptor_8851eb1ddb7aa19d, []int{11, 1} +} +func (m *LogResponse_Flushed) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogResponse_Flushed) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_LogResponse_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 *LogResponse_Flushed) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogResponse_Flushed.Merge(m, src) +} +func (m *LogResponse_Flushed) XXX_Size() int { + return m.ProtoSize() +} +func (m *LogResponse_Flushed) XXX_DiscardUnknown() { + xxx_messageInfo_LogResponse_Flushed.DiscardUnknown(m) +} + +var xxx_messageInfo_LogResponse_Flushed proto.InternalMessageInfo + +func init() { + proto.RegisterType((*Shard)(nil), "shuffle.Shard") + proto.RegisterType((*CollectionPartitions)(nil), "shuffle.CollectionPartitions") + proto.RegisterType((*Task)(nil), "shuffle.Task") + proto.RegisterType((*ProducerFrontier)(nil), "shuffle.ProducerFrontier") + proto.RegisterType((*JournalFrontier)(nil), "shuffle.JournalFrontier") + proto.RegisterType((*Frontier)(nil), "shuffle.Frontier") + proto.RegisterType((*SessionRequest)(nil), "shuffle.SessionRequest") + proto.RegisterType((*SessionRequest_Open)(nil), "shuffle.SessionRequest.Open") + proto.RegisterType((*SessionRequest_NextCheckpoint)(nil), "shuffle.SessionRequest.NextCheckpoint") + proto.RegisterType((*SessionResponse)(nil), "shuffle.SessionResponse") + proto.RegisterType((*SessionResponse_Opened)(nil), "shuffle.SessionResponse.Opened") + proto.RegisterType((*SliceRequest)(nil), "shuffle.SliceRequest") + proto.RegisterType((*SliceRequest_Open)(nil), "shuffle.SliceRequest.Open") + proto.RegisterType((*SliceRequest_Start)(nil), "shuffle.SliceRequest.Start") + proto.RegisterType((*SliceRequest_StartRead)(nil), "shuffle.SliceRequest.StartRead") + proto.RegisterType((*SliceRequest_Progress)(nil), "shuffle.SliceRequest.Progress") + proto.RegisterType((*SliceResponse)(nil), "shuffle.SliceResponse") + proto.RegisterType((*SliceResponse_Opened)(nil), "shuffle.SliceResponse.Opened") + proto.RegisterType((*SliceResponse_ListingAdded)(nil), "shuffle.SliceResponse.ListingAdded") + proto.RegisterType((*LogRequest)(nil), "shuffle.LogRequest") + proto.RegisterType((*LogRequest_Open)(nil), "shuffle.LogRequest.Open") + proto.RegisterType((*LogRequest_Append)(nil), "shuffle.LogRequest.Append") + proto.RegisterType((*LogRequest_Flush)(nil), "shuffle.LogRequest.Flush") + proto.RegisterType((*LogResponse)(nil), "shuffle.LogResponse") + proto.RegisterType((*LogResponse_Opened)(nil), "shuffle.LogResponse.Opened") + proto.RegisterType((*LogResponse_Flushed)(nil), "shuffle.LogResponse.Flushed") +} + +func init() { + proto.RegisterFile("go/protocols/shuffle/shuffle.proto", fileDescriptor_8851eb1ddb7aa19d) +} + +var fileDescriptor_8851eb1ddb7aa19d = []byte{ + // 1449 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x57, 0x4b, 0x73, 0x14, 0xd5, + 0x17, 0xa7, 0x33, 0xaf, 0xcc, 0x99, 0x24, 0x93, 0x5c, 0x02, 0xf4, 0xbf, 0x49, 0x42, 0x18, 0xea, + 0x8f, 0xd1, 0xa2, 0x26, 0x10, 0x10, 0x4a, 0xaa, 0xa4, 0x20, 0x68, 0x0a, 0x34, 0x02, 0xde, 0x61, + 0xe5, 0xa6, 0xab, 0xd3, 0x7d, 0xa6, 0xa7, 0x4d, 0x4f, 0xdf, 0xb1, 0x6f, 0x0f, 0x66, 0x5c, 0xbb, + 0x74, 0xa1, 0x55, 0x96, 0x6e, 0x75, 0xed, 0x17, 0x70, 0xe3, 0x96, 0x62, 0xe9, 0x57, 0x10, 0x56, + 0x7e, 0x09, 0xcb, 0xba, 0x8f, 0x7e, 0xcc, 0xa4, 0x83, 0x2e, 0x5c, 0xe8, 0x26, 0xe9, 0x7b, 0xce, + 0xef, 0x3c, 0xee, 0x79, 0xdd, 0x33, 0xd0, 0xf1, 0xd9, 0xf6, 0x28, 0x66, 0x09, 0x73, 0x59, 0xc8, + 0xb7, 0xf9, 0x60, 0xdc, 0xef, 0x87, 0x98, 0xfe, 0xef, 0x4a, 0x0e, 0x69, 0xe8, 0xa3, 0xb5, 0x71, + 0x10, 0xb3, 0x43, 0x8c, 0x33, 0x81, 0xec, 0x43, 0x01, 0xad, 0xb5, 0x29, 0x65, 0xfd, 0x90, 0x7d, + 0x2e, 0xff, 0x68, 0xee, 0xaa, 0xcf, 0x7c, 0x26, 0x3f, 0xb7, 0xc5, 0x97, 0xa2, 0x76, 0x06, 0x50, + 0xeb, 0x0d, 0x9c, 0xd8, 0x23, 0xff, 0x87, 0x5a, 0xec, 0x44, 0x3e, 0x9a, 0xc6, 0xa6, 0xb1, 0xd5, + 0xda, 0x69, 0x77, 0xa5, 0x28, 0x15, 0xa4, 0xde, 0x08, 0x5d, 0xaa, 0xb8, 0xc4, 0x82, 0x79, 0x8c, + 0xbc, 0x11, 0x0b, 0xa2, 0xc4, 0x9c, 0xdb, 0x34, 0xb6, 0x9a, 0x34, 0x3b, 0x93, 0x35, 0x68, 0x7a, + 0x41, 0x8c, 0x6e, 0xc2, 0xe2, 0x89, 0x59, 0x91, 0xcc, 0x9c, 0xd0, 0xf9, 0xd6, 0x80, 0xd5, 0xfb, + 0x2c, 0x0c, 0xd1, 0x4d, 0x02, 0x16, 0x3d, 0x71, 0xe2, 0x24, 0x10, 0x1f, 0x9c, 0xdc, 0x00, 0x70, + 0x33, 0xba, 0x36, 0xbf, 0xaa, 0xcc, 0xe7, 0x78, 0xe9, 0x43, 0x01, 0x47, 0xf6, 0x80, 0x8c, 0x52, + 0x1d, 0x36, 0xc7, 0x50, 0x5a, 0x91, 0x2e, 0xb5, 0x76, 0xce, 0x75, 0xb3, 0xc8, 0xec, 0x3b, 0x07, + 0x18, 0xf6, 0x34, 0x9b, 0xae, 0x64, 0x22, 0x29, 0xa9, 0xf3, 0xca, 0x80, 0xea, 0x53, 0x87, 0x1f, + 0x92, 0xa7, 0x70, 0x26, 0x57, 0x6f, 0x67, 0x40, 0xae, 0x3d, 0x5a, 0xef, 0xa6, 0x59, 0x29, 0xbb, + 0xc4, 0x83, 0x53, 0x74, 0xd5, 0x2d, 0xbb, 0xdc, 0x4d, 0x00, 0x0f, 0xe3, 0xe0, 0x99, 0x23, 0x2f, + 0x37, 0x77, 0xf2, 0xe5, 0x1e, 0x9c, 0xa2, 0x05, 0x24, 0x79, 0x1f, 0xda, 0x43, 0x27, 0xc1, 0x38, + 0x70, 0xc2, 0xe0, 0x0b, 0x25, 0x5c, 0x91, 0xc2, 0xff, 0x53, 0xc2, 0x1f, 0x4d, 0x33, 0xb5, 0x86, + 0x59, 0x99, 0xdd, 0x3a, 0x54, 0x13, 0x87, 0x1f, 0x76, 0xbe, 0x32, 0x60, 0xf9, 0x49, 0xcc, 0xbc, + 0xb1, 0x8b, 0xf1, 0x5e, 0xcc, 0xa2, 0x24, 0xc0, 0x58, 0xe4, 0x72, 0xa4, 0x69, 0xf2, 0x92, 0x15, + 0x9a, 0x9d, 0xc9, 0x05, 0x68, 0x85, 0x0e, 0x4f, 0x6c, 0x97, 0x0d, 0x87, 0x81, 0x4a, 0x75, 0x9d, + 0x82, 0x20, 0xdd, 0x97, 0x14, 0x72, 0x09, 0x16, 0x07, 0x41, 0x94, 0xa0, 0x97, 0x42, 0x2a, 0x12, + 0xb2, 0xa0, 0x88, 0x1a, 0x74, 0x16, 0xea, 0xac, 0xdf, 0xe7, 0x98, 0x98, 0x55, 0xa9, 0x5f, 0x9f, + 0x3a, 0x3f, 0xcd, 0x41, 0xfb, 0x03, 0x36, 0x8e, 0x23, 0x27, 0xcc, 0xbc, 0x79, 0x17, 0xce, 0x7f, + 0xaa, 0x48, 0x76, 0xe4, 0x0c, 0xd1, 0x4e, 0xe2, 0x71, 0xe4, 0x3a, 0x09, 0xda, 0x1e, 0x86, 0x89, + 0x23, 0x1d, 0xac, 0x51, 0x53, 0x43, 0x1e, 0x39, 0x43, 0x7c, 0xaa, 0x01, 0xef, 0x09, 0x3e, 0xe9, + 0xc2, 0xe9, 0x29, 0x71, 0x3e, 0xee, 0xf7, 0x83, 0x23, 0x5d, 0xa3, 0x2b, 0x05, 0xb1, 0x9e, 0x64, + 0x10, 0x13, 0x1a, 0x07, 0x41, 0xe4, 0x05, 0x91, 0x2f, 0x3d, 0x5f, 0xa4, 0xe9, 0x91, 0x6c, 0xc1, + 0xf2, 0xc1, 0x24, 0x41, 0x6e, 0xc7, 0xe8, 0x78, 0xda, 0xba, 0x72, 0x7f, 0x49, 0xd2, 0x29, 0x3a, + 0x9e, 0xb2, 0x79, 0x05, 0x88, 0x42, 0x1e, 0xe0, 0x20, 0x88, 0x52, 0x6c, 0x4d, 0x62, 0x95, 0x8e, + 0x5d, 0xc9, 0x50, 0xe8, 0x5b, 0xd0, 0x4c, 0xc3, 0xcb, 0xcd, 0xfa, 0x66, 0x45, 0x26, 0x33, 0x2d, + 0xaa, 0xd9, 0xe4, 0xd0, 0x1c, 0xdb, 0x71, 0x60, 0x3e, 0x8b, 0xd2, 0x0d, 0x98, 0xd7, 0x77, 0x11, + 0x85, 0x29, 0x74, 0x98, 0x99, 0x8e, 0x99, 0x88, 0xd2, 0x0c, 0x29, 0xb2, 0xd9, 0x0f, 0xc7, 0x7c, + 0x80, 0x9e, 0x1d, 0x72, 0x51, 0x86, 0x95, 0xad, 0x2a, 0x05, 0x4d, 0xda, 0xe7, 0x51, 0xe7, 0xe7, + 0x39, 0x58, 0xea, 0x21, 0xe7, 0x01, 0x8b, 0x28, 0x7e, 0x36, 0x46, 0x9e, 0x90, 0xab, 0x50, 0x65, + 0x23, 0x4c, 0x1b, 0x72, 0x2d, 0xb3, 0x32, 0x0d, 0xeb, 0x3e, 0x1e, 0x61, 0x44, 0x25, 0x92, 0xdc, + 0x81, 0x95, 0x18, 0xf9, 0x78, 0x88, 0xb6, 0x3b, 0x40, 0xf7, 0x30, 0x1f, 0x12, 0xad, 0x9d, 0x95, + 0x4c, 0x3c, 0xf3, 0x6e, 0x59, 0x61, 0xef, 0x67, 0x50, 0xf2, 0x18, 0xda, 0x11, 0x1e, 0x25, 0x45, + 0x69, 0x55, 0xf3, 0x97, 0x4f, 0x32, 0xfe, 0x08, 0x8f, 0x92, 0x5c, 0x01, 0x5d, 0x8a, 0xa6, 0xce, + 0xd6, 0xc7, 0x50, 0x15, 0xee, 0x91, 0x8b, 0xaa, 0x0b, 0xb4, 0x2f, 0x8b, 0x99, 0x36, 0xd1, 0xf7, + 0x54, 0xb2, 0xc8, 0x65, 0xa8, 0x73, 0x31, 0x07, 0xb9, 0x59, 0x91, 0x51, 0x5d, 0xca, 0x4d, 0x0a, + 0x32, 0xd5, 0x5c, 0x6b, 0x19, 0x96, 0xa6, 0x8d, 0x76, 0xbe, 0x36, 0xa0, 0x9d, 0xb9, 0xc5, 0x47, + 0x2c, 0xe2, 0x48, 0x6e, 0x41, 0x5d, 0x44, 0x04, 0x3d, 0x1d, 0xbd, 0x0b, 0xc7, 0x2f, 0xa0, 0x90, + 0x32, 0x7c, 0xe8, 0x51, 0x0d, 0x27, 0xb7, 0x8f, 0x87, 0xe0, 0xc4, 0x00, 0xce, 0xde, 0x76, 0x1e, + 0xea, 0x4a, 0x5b, 0xe7, 0xfb, 0x1a, 0x2c, 0xf4, 0xc2, 0xc0, 0xc5, 0x34, 0x97, 0xdd, 0xa9, 0x5c, + 0x5a, 0xb9, 0x37, 0x05, 0x50, 0x31, 0x93, 0xd7, 0xa0, 0xc6, 0x13, 0x27, 0x4e, 0x8d, 0x9f, 0x2f, + 0x17, 0xe8, 0x09, 0x08, 0x55, 0x48, 0x72, 0x07, 0x40, 0x7e, 0xc8, 0xae, 0xd1, 0x79, 0xbb, 0xf0, + 0x3a, 0x39, 0x74, 0x3c, 0xda, 0xe4, 0xe9, 0x27, 0xb9, 0x2d, 0x87, 0x91, 0x1f, 0x23, 0xe7, 0xb2, + 0xdb, 0x5a, 0x3b, 0x1b, 0xe5, 0xd2, 0x4f, 0x34, 0x8a, 0x66, 0x78, 0xeb, 0x1b, 0x43, 0x27, 0x7a, + 0x1d, 0x80, 0xab, 0x00, 0xdb, 0x81, 0x8a, 0x7d, 0x83, 0x36, 0x35, 0xe5, 0xa1, 0xf7, 0x0f, 0xd6, + 0x81, 0xe8, 0x28, 0xf9, 0x65, 0x07, 0x91, 0x87, 0x47, 0xd2, 0xe3, 0x45, 0x0a, 0x92, 0xf4, 0x50, + 0x50, 0xac, 0x06, 0xd4, 0xe4, 0x3d, 0xad, 0x3f, 0x0c, 0x68, 0x66, 0x37, 0x2e, 0x8e, 0x1d, 0x63, + 0x7a, 0xec, 0xbc, 0x09, 0x55, 0x3e, 0x42, 0x57, 0x3b, 0x77, 0x26, 0x7f, 0xc2, 0x74, 0x5b, 0xcb, + 0x17, 0x50, 0x42, 0xc8, 0x1b, 0xd0, 0x76, 0x63, 0x14, 0xb3, 0x31, 0xc6, 0x67, 0x01, 0x4f, 0x1f, + 0x87, 0x0a, 0x5d, 0x52, 0x64, 0xaa, 0xa9, 0xe4, 0x22, 0x2c, 0x0c, 0x99, 0x97, 0xa3, 0xd4, 0x18, + 0x6b, 0x0d, 0x99, 0x97, 0x41, 0xc4, 0xbb, 0xcf, 0xc6, 0x09, 0xca, 0xb1, 0x25, 0xde, 0xfd, 0xcc, + 0x2e, 0x15, 0x64, 0xaa, 0xb8, 0xe4, 0x1d, 0x80, 0x42, 0x4d, 0xfe, 0xe5, 0xf4, 0x2a, 0x80, 0x2d, + 0x80, 0xf9, 0x34, 0x67, 0x9d, 0xef, 0x2a, 0xb0, 0xa8, 0xb3, 0xa9, 0x5b, 0xe5, 0xed, 0x99, 0x56, + 0x59, 0x9f, 0xcd, 0x7a, 0x79, 0xa3, 0x3c, 0x80, 0xc5, 0x30, 0xe0, 0x49, 0x10, 0xf9, 0xb6, 0xe3, + 0x79, 0xe8, 0xe9, 0xb0, 0x5d, 0x3a, 0x41, 0x7a, 0x5f, 0x61, 0xef, 0x09, 0x28, 0x5d, 0x08, 0x0b, + 0x27, 0x72, 0x0d, 0x20, 0x2d, 0x24, 0x4c, 0x0b, 0xb7, 0xa4, 0xdb, 0x0a, 0xa0, 0xbc, 0xd3, 0xac, + 0xe7, 0x06, 0x2c, 0x14, 0x75, 0xff, 0x57, 0xf3, 0xdb, 0xf9, 0xbd, 0x06, 0xb0, 0xcf, 0xfc, 0x74, + 0x60, 0x5c, 0x99, 0x1a, 0x18, 0xf9, 0x13, 0x93, 0x43, 0x8a, 0xe3, 0x62, 0x07, 0xea, 0xce, 0x68, + 0x84, 0x51, 0x9a, 0x05, 0xab, 0x0c, 0x7f, 0x4f, 0x22, 0xa8, 0x46, 0x92, 0x6d, 0xa8, 0xc9, 0xf7, + 0x27, 0x5b, 0x6b, 0x4a, 0x44, 0xf6, 0x04, 0x80, 0x2a, 0x9c, 0xf5, 0xe3, 0xdf, 0x6c, 0xf2, 0xbc, + 0x83, 0xe7, 0x5e, 0xdb, 0xc1, 0x6f, 0xc1, 0x0a, 0x17, 0x35, 0x62, 0x17, 0xfb, 0x58, 0xad, 0x02, + 0x6d, 0xc9, 0xe8, 0x65, 0xcd, 0x4c, 0x2e, 0x43, 0x3b, 0x64, 0xbe, 0x7d, 0xbc, 0xe3, 0x17, 0x43, + 0xe6, 0xe7, 0x38, 0xeb, 0xcb, 0x0a, 0xd4, 0xd5, 0x3d, 0xff, 0x3d, 0xeb, 0x8c, 0xdc, 0xf2, 0x02, + 0x16, 0x07, 0xc9, 0x44, 0x3b, 0x9d, 0x9d, 0x45, 0x28, 0xd3, 0x25, 0xc7, 0x99, 0xc8, 0x0a, 0xa9, + 0xd2, 0x66, 0xac, 0xf6, 0x1b, 0x67, 0x32, 0xb5, 0x20, 0xd6, 0x67, 0x16, 0xc4, 0x55, 0xa8, 0xb9, + 0x21, 0x73, 0x0f, 0xcd, 0x86, 0xdc, 0xfb, 0xd4, 0x41, 0x50, 0xfb, 0xa1, 0xe3, 0x73, 0x73, 0x5e, + 0x5a, 0x52, 0x07, 0x61, 0x66, 0xe4, 0xb8, 0x87, 0xe8, 0xd9, 0x87, 0x38, 0x31, 0x9b, 0x9b, 0xc6, + 0xd6, 0x02, 0x6d, 0x2a, 0xca, 0x87, 0x38, 0x11, 0x55, 0xec, 0x31, 0xd7, 0x76, 0x62, 0x77, 0x10, + 0x3c, 0x43, 0xcf, 0x04, 0x09, 0x68, 0x79, 0xcc, 0xbd, 0xa7, 0x49, 0x62, 0xd3, 0xe2, 0x6c, 0x1c, + 0xbb, 0x68, 0x8b, 0xb5, 0xca, 0x0e, 0x31, 0xf2, 0x93, 0x81, 0xd9, 0x92, 0x46, 0x96, 0x15, 0x67, + 0x77, 0x92, 0xe0, 0xbe, 0xa4, 0x5b, 0xeb, 0x50, 0x93, 0xa5, 0x23, 0x9d, 0x9c, 0xb8, 0xa1, 0xfa, + 0x51, 0x53, 0xa5, 0xea, 0xd0, 0x79, 0x6e, 0x40, 0x4b, 0x56, 0x99, 0x1e, 0x41, 0xd7, 0x67, 0x46, + 0xd0, 0xf9, 0xe9, 0x5a, 0x2c, 0x1f, 0x40, 0x37, 0xa1, 0xa1, 0xf7, 0x27, 0x5d, 0xf4, 0x6b, 0xa5, + 0x52, 0x7b, 0x0a, 0x43, 0x53, 0x70, 0x61, 0x76, 0xdc, 0x85, 0x86, 0xe6, 0x96, 0xfb, 0x79, 0x7c, + 0x6b, 0x33, 0xa6, 0xb7, 0xb6, 0x9d, 0x5f, 0x0c, 0x68, 0xf4, 0x94, 0x51, 0x72, 0x17, 0x1a, 0x7a, + 0xb7, 0x20, 0xe7, 0x4e, 0x58, 0x97, 0x2c, 0xf3, 0xa4, 0x35, 0x64, 0xcb, 0xb8, 0x6a, 0x90, 0xdb, + 0x50, 0x93, 0x43, 0x93, 0x9c, 0x29, 0x7d, 0x78, 0xad, 0xb3, 0xe5, 0xb3, 0x55, 0xca, 0xde, 0x80, + 0xca, 0x3e, 0xf3, 0xc9, 0xe9, 0x92, 0x2e, 0xb6, 0x56, 0xcb, 0x02, 0x23, 0xa4, 0x76, 0xef, 0xbc, + 0xf8, 0x6d, 0xe3, 0xd4, 0x8b, 0x97, 0x1b, 0xc6, 0xaf, 0x2f, 0x37, 0x8c, 0x1f, 0x5e, 0x6d, 0x18, + 0x9f, 0x5c, 0xf1, 0x83, 0x64, 0x30, 0x3e, 0xe8, 0xba, 0x6c, 0xb8, 0x8d, 0x3c, 0x19, 0x3b, 0xf1, + 0x44, 0xfd, 0x90, 0x2d, 0xfb, 0x9d, 0x7c, 0x50, 0x97, 0xa4, 0xeb, 0x7f, 0x06, 0x00, 0x00, 0xff, + 0xff, 0x6f, 0x72, 0xb3, 0x68, 0x46, 0x0f, 0x00, 0x00, +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// ShuffleClient is the client API for Shuffle service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type ShuffleClient interface { + // Session is the top-level RPC opened by the Coordinator (typically + // shard-000). It manages the shuffle lifecycle, journal watches, + // and checkpoint aggregation. + Session(ctx context.Context, opts ...grpc.CallOption) (Shuffle_SessionClient, error) + // Slice is opened by the Session to each shard. + // Each Slice watches binding journals, reads from its assigned subset of journals, and routes documents to Logs. + Slice(ctx context.Context, opts ...grpc.CallOption) (Shuffle_SliceClient, error) + // Log is opened by each Slice to each shard. + // Documents flow from Slices through Log RPCs to on-disk log files. + Log(ctx context.Context, opts ...grpc.CallOption) (Shuffle_LogClient, error) +} + +type shuffleClient struct { + cc *grpc.ClientConn +} + +func NewShuffleClient(cc *grpc.ClientConn) ShuffleClient { + return &shuffleClient{cc} +} + +func (c *shuffleClient) Session(ctx context.Context, opts ...grpc.CallOption) (Shuffle_SessionClient, error) { + stream, err := c.cc.NewStream(ctx, &_Shuffle_serviceDesc.Streams[0], "/shuffle.Shuffle/Session", opts...) + if err != nil { + return nil, err + } + x := &shuffleSessionClient{stream} + return x, nil +} + +type Shuffle_SessionClient interface { + Send(*SessionRequest) error + Recv() (*SessionResponse, error) + grpc.ClientStream +} + +type shuffleSessionClient struct { + grpc.ClientStream +} + +func (x *shuffleSessionClient) Send(m *SessionRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *shuffleSessionClient) Recv() (*SessionResponse, error) { + m := new(SessionResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *shuffleClient) Slice(ctx context.Context, opts ...grpc.CallOption) (Shuffle_SliceClient, error) { + stream, err := c.cc.NewStream(ctx, &_Shuffle_serviceDesc.Streams[1], "/shuffle.Shuffle/Slice", opts...) + if err != nil { + return nil, err + } + x := &shuffleSliceClient{stream} + return x, nil +} + +type Shuffle_SliceClient interface { + Send(*SliceRequest) error + Recv() (*SliceResponse, error) + grpc.ClientStream +} + +type shuffleSliceClient struct { + grpc.ClientStream +} + +func (x *shuffleSliceClient) Send(m *SliceRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *shuffleSliceClient) Recv() (*SliceResponse, error) { + m := new(SliceResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *shuffleClient) Log(ctx context.Context, opts ...grpc.CallOption) (Shuffle_LogClient, error) { + stream, err := c.cc.NewStream(ctx, &_Shuffle_serviceDesc.Streams[2], "/shuffle.Shuffle/Log", opts...) + if err != nil { + return nil, err + } + x := &shuffleLogClient{stream} + return x, nil +} + +type Shuffle_LogClient interface { + Send(*LogRequest) error + Recv() (*LogResponse, error) + grpc.ClientStream +} + +type shuffleLogClient struct { + grpc.ClientStream +} + +func (x *shuffleLogClient) Send(m *LogRequest) error { + return x.ClientStream.SendMsg(m) +} + +func (x *shuffleLogClient) Recv() (*LogResponse, error) { + m := new(LogResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// ShuffleServer is the server API for Shuffle service. +type ShuffleServer interface { + // Session is the top-level RPC opened by the Coordinator (typically + // shard-000). It manages the shuffle lifecycle, journal watches, + // and checkpoint aggregation. + Session(Shuffle_SessionServer) error + // Slice is opened by the Session to each shard. + // Each Slice watches binding journals, reads from its assigned subset of journals, and routes documents to Logs. + Slice(Shuffle_SliceServer) error + // Log is opened by each Slice to each shard. + // Documents flow from Slices through Log RPCs to on-disk log files. + Log(Shuffle_LogServer) error +} + +// UnimplementedShuffleServer can be embedded to have forward compatible implementations. +type UnimplementedShuffleServer struct { +} + +func (*UnimplementedShuffleServer) Session(srv Shuffle_SessionServer) error { + return status.Errorf(codes.Unimplemented, "method Session not implemented") +} +func (*UnimplementedShuffleServer) Slice(srv Shuffle_SliceServer) error { + return status.Errorf(codes.Unimplemented, "method Slice not implemented") +} +func (*UnimplementedShuffleServer) Log(srv Shuffle_LogServer) error { + return status.Errorf(codes.Unimplemented, "method Log not implemented") +} + +func RegisterShuffleServer(s *grpc.Server, srv ShuffleServer) { + s.RegisterService(&_Shuffle_serviceDesc, srv) +} + +func _Shuffle_Session_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(ShuffleServer).Session(&shuffleSessionServer{stream}) +} + +type Shuffle_SessionServer interface { + Send(*SessionResponse) error + Recv() (*SessionRequest, error) + grpc.ServerStream +} + +type shuffleSessionServer struct { + grpc.ServerStream +} + +func (x *shuffleSessionServer) Send(m *SessionResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *shuffleSessionServer) Recv() (*SessionRequest, error) { + m := new(SessionRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func _Shuffle_Slice_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(ShuffleServer).Slice(&shuffleSliceServer{stream}) +} + +type Shuffle_SliceServer interface { + Send(*SliceResponse) error + Recv() (*SliceRequest, error) + grpc.ServerStream +} + +type shuffleSliceServer struct { + grpc.ServerStream +} + +func (x *shuffleSliceServer) Send(m *SliceResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *shuffleSliceServer) Recv() (*SliceRequest, error) { + m := new(SliceRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func _Shuffle_Log_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(ShuffleServer).Log(&shuffleLogServer{stream}) +} + +type Shuffle_LogServer interface { + Send(*LogResponse) error + Recv() (*LogRequest, error) + grpc.ServerStream +} + +type shuffleLogServer struct { + grpc.ServerStream +} + +func (x *shuffleLogServer) Send(m *LogResponse) error { + return x.ServerStream.SendMsg(m) +} + +func (x *shuffleLogServer) Recv() (*LogRequest, error) { + m := new(LogRequest) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +var _Shuffle_serviceDesc = grpc.ServiceDesc{ + ServiceName: "shuffle.Shuffle", + HandlerType: (*ShuffleServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "Session", + Handler: _Shuffle_Session_Handler, + ServerStreams: true, + ClientStreams: true, + }, + { + StreamName: "Slice", + Handler: _Shuffle_Slice_Handler, + ServerStreams: true, + ClientStreams: true, + }, + { + StreamName: "Log", + Handler: _Shuffle_Log_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "go/protocols/shuffle/shuffle.proto", +} + +func (m *Shard) 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 *Shard) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Shard) 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 len(m.Directory) > 0 { + i -= len(m.Directory) + copy(dAtA[i:], m.Directory) + i = encodeVarintShuffle(dAtA, i, uint64(len(m.Directory))) + i-- + dAtA[i] = 0x1a + } + if len(m.Endpoint) > 0 { + i -= len(m.Endpoint) + copy(dAtA[i:], m.Endpoint) + i = encodeVarintShuffle(dAtA, i, uint64(len(m.Endpoint))) + i-- + dAtA[i] = 0x12 + } + if m.Range != nil { + { + size, err := m.Range.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CollectionPartitions) 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 *CollectionPartitions) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CollectionPartitions) 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.PartitionSelector != nil { + { + size, err := m.PartitionSelector.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.Collection != nil { + { + size, err := m.Collection.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Task) 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 *Task) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Task) 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.Task != nil { + { + size := m.Task.ProtoSize() + i -= size + if _, err := m.Task.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *Task_CollectionPartitions) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Task_CollectionPartitions) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.CollectionPartitions != nil { + { + size, err := m.CollectionPartitions.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} +func (m *Task_Derivation) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Task_Derivation) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Derivation != nil { + { + size, err := m.Derivation.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} +func (m *Task_Materialization) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Task_Materialization) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Materialization != nil { + { + size, err := m.Materialization.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + return len(dAtA) - i, nil +} +func (m *ProducerFrontier) 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 *ProducerFrontier) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ProducerFrontier) 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.Offset != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.Offset)) + i-- + dAtA[i] = 0x20 + } + if m.HintedCommit != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.HintedCommit)) + i-- + dAtA[i] = 0x19 + } + if m.LastCommit != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LastCommit)) + i-- + dAtA[i] = 0x11 + } + if m.Producer != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.Producer)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *JournalFrontier) 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 *JournalFrontier) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *JournalFrontier) 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 len(m.Producers) > 0 { + for iNdEx := len(m.Producers) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Producers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + } + if m.BytesBehindDelta != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.BytesBehindDelta)) + i-- + dAtA[i] = 0x28 + } + if m.BytesReadDelta != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.BytesReadDelta)) + i-- + dAtA[i] = 0x20 + } + if m.Binding != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.Binding)) + i-- + dAtA[i] = 0x18 + } + if len(m.JournalNameSuffix) > 0 { + i -= len(m.JournalNameSuffix) + copy(dAtA[i:], m.JournalNameSuffix) + i = encodeVarintShuffle(dAtA, i, uint64(len(m.JournalNameSuffix))) + i-- + dAtA[i] = 0x12 + } + if m.JournalNameTruncateDelta != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.JournalNameTruncateDelta)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Frontier) 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 *Frontier) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Frontier) 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 len(m.FlushedLsn) > 0 { + dAtA8 := make([]byte, len(m.FlushedLsn)*10) + var j7 int + for _, num := range m.FlushedLsn { + for num >= 1<<7 { + dAtA8[j7] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j7++ + } + dAtA8[j7] = uint8(num) + j7++ + } + i -= j7 + copy(dAtA[i:], dAtA8[:j7]) + i = encodeVarintShuffle(dAtA, i, uint64(j7)) + i-- + dAtA[i] = 0x12 + } + if len(m.Journals) > 0 { + for iNdEx := len(m.Journals) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Journals[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *SessionRequest) 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 *SessionRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SessionRequest) 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.NextCheckpoint != nil { + { + size, err := m.NextCheckpoint.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.ResumeCheckpoint != nil { + { + size, err := m.ResumeCheckpoint.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.Open != nil { + { + size, err := m.Open.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SessionRequest_Open) 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 *SessionRequest_Open) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SessionRequest_Open) 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 len(m.Shards) > 0 { + for iNdEx := len(m.Shards) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Shards[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if m.Task != nil { + { + size, err := m.Task.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *SessionRequest_NextCheckpoint) 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 *SessionRequest_NextCheckpoint) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SessionRequest_NextCheckpoint) 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) + } + return len(dAtA) - i, nil +} + +func (m *SessionResponse) 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 *SessionResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SessionResponse) 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.NextCheckpoint != nil { + { + size, err := m.NextCheckpoint.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.Opened != nil { + { + size, err := m.Opened.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SessionResponse_Opened) 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 *SessionResponse_Opened) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SessionResponse_Opened) 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) + } + return len(dAtA) - i, nil +} + +func (m *SliceRequest) 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 *SliceRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SliceRequest) 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.Progress != nil { + { + size, err := m.Progress.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + if m.StartRead != nil { + { + size, err := m.StartRead.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.Start != nil { + { + size, err := m.Start.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.Open != nil { + { + size, err := m.Open.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SliceRequest_Open) 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 *SliceRequest_Open) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SliceRequest_Open) 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.ShardIndex != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.ShardIndex)) + i-- + dAtA[i] = 0x20 + } + if len(m.Shards) > 0 { + for iNdEx := len(m.Shards) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Shards[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if m.Task != nil { + { + size, err := m.Task.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.SessionId != 0 { + i -= 4 + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(m.SessionId)) + i-- + dAtA[i] = 0xd + } + return len(dAtA) - i, nil +} + +func (m *SliceRequest_Start) 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 *SliceRequest_Start) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SliceRequest_Start) 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) + } + return len(dAtA) - i, nil +} + +func (m *SliceRequest_StartRead) 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 *SliceRequest_StartRead) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SliceRequest_StartRead) 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 len(m.Checkpoint) > 0 { + for iNdEx := len(m.Checkpoint) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Checkpoint[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + } + if m.Route != nil { + { + size, err := m.Route.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + if m.ModRevision != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.ModRevision)) + i-- + dAtA[i] = 0x20 + } + if m.CreateRevision != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.CreateRevision)) + i-- + dAtA[i] = 0x18 + } + if m.Spec != nil { + { + size, err := m.Spec.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.Binding != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.Binding)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *SliceRequest_Progress) 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 *SliceRequest_Progress) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SliceRequest_Progress) 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) + } + return len(dAtA) - i, nil +} + +func (m *SliceResponse) 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 *SliceResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SliceResponse) 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.Progressed != nil { + { + size, err := m.Progressed.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.ListingAdded != nil { + { + size, err := m.ListingAdded.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.Opened != nil { + { + size, err := m.Opened.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SliceResponse_Opened) 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 *SliceResponse_Opened) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SliceResponse_Opened) 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) + } + return len(dAtA) - i, nil +} + +func (m *SliceResponse_ListingAdded) 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 *SliceResponse_ListingAdded) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SliceResponse_ListingAdded) 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.Route != nil { + { + size, err := m.Route.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + if m.ModRevision != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.ModRevision)) + i-- + dAtA[i] = 0x20 + } + if m.CreateRevision != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.CreateRevision)) + i-- + dAtA[i] = 0x18 + } + if m.Spec != nil { + { + size, err := m.Spec.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.Binding != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.Binding)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *LogRequest) 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 *LogRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LogRequest) 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.Flush != nil { + { + size, err := m.Flush.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.Append != nil { + { + size, err := m.Append.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.Open != nil { + { + size, err := m.Open.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *LogRequest_Open) 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 *LogRequest_Open) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LogRequest_Open) 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.LogShardIndex != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.LogShardIndex)) + i-- + dAtA[i] = 0x20 + } + if m.SliceShardIndex != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.SliceShardIndex)) + i-- + dAtA[i] = 0x18 + } + if len(m.Shards) > 0 { + for iNdEx := len(m.Shards) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Shards[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if m.SessionId != 0 { + i -= 4 + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(m.SessionId)) + i-- + dAtA[i] = 0xd + } + return len(dAtA) - i, nil +} + +func (m *LogRequest_Append) 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 *LogRequest_Append) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LogRequest_Append) 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.SourceByteLength != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.SourceByteLength)) + i-- + dAtA[i] = 0x58 + } + if len(m.DocArchived) > 0 { + i -= len(m.DocArchived) + copy(dAtA[i:], m.DocArchived) + i = encodeVarintShuffle(dAtA, i, uint64(len(m.DocArchived))) + i-- + dAtA[i] = 0x52 + } + if len(m.PackedKey) > 0 { + i -= len(m.PackedKey) + copy(dAtA[i:], m.PackedKey) + i = encodeVarintShuffle(dAtA, i, uint64(len(m.PackedKey))) + i-- + dAtA[i] = 0x4a + } + if m.Flags != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.Flags)) + i-- + dAtA[i] = 0x40 + } + if m.Clock != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.Clock)) + i-- + dAtA[i] = 0x39 + } + if m.Producer != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.Producer)) + i-- + dAtA[i] = 0x30 + } + if m.ReadDelay != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.ReadDelay)) + i-- + dAtA[i] = 0x28 + } + if m.Priority != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.Priority)) + i-- + dAtA[i] = 0x20 + } + if m.Binding != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.Binding)) + i-- + dAtA[i] = 0x18 + } + if len(m.JournalNameSuffix) > 0 { + i -= len(m.JournalNameSuffix) + copy(dAtA[i:], m.JournalNameSuffix) + i = encodeVarintShuffle(dAtA, i, uint64(len(m.JournalNameSuffix))) + i-- + dAtA[i] = 0x12 + } + if m.JournalNameTruncateDelta != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.JournalNameTruncateDelta)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *LogRequest_Flush) 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 *LogRequest_Flush) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LogRequest_Flush) 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.Cycle != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.Cycle)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *LogResponse) 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 *LogResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LogResponse) 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.Flushed != nil { + { + size, err := m.Flushed.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if m.Opened != nil { + { + size, err := m.Opened.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintShuffle(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *LogResponse_Opened) 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 *LogResponse_Opened) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LogResponse_Opened) 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) + } + return len(dAtA) - i, nil +} + +func (m *LogResponse_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 *LogResponse_Flushed) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LogResponse_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.FlushedLsn != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.FlushedLsn)) + i-- + dAtA[i] = 0x10 + } + if m.Cycle != 0 { + i = encodeVarintShuffle(dAtA, i, uint64(m.Cycle)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func encodeVarintShuffle(dAtA []byte, offset int, v uint64) int { + offset -= sovShuffle(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Shard) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Range != nil { + l = m.Range.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + l = len(m.Endpoint) + if l > 0 { + n += 1 + l + sovShuffle(uint64(l)) + } + l = len(m.Directory) + if l > 0 { + n += 1 + l + sovShuffle(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *CollectionPartitions) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Collection != nil { + l = m.Collection.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.PartitionSelector != nil { + l = m.PartitionSelector.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Task) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Task != nil { + n += m.Task.ProtoSize() + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Task_CollectionPartitions) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.CollectionPartitions != nil { + l = m.CollectionPartitions.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + return n +} +func (m *Task_Derivation) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Derivation != nil { + l = m.Derivation.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + return n +} +func (m *Task_Materialization) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Materialization != nil { + l = m.Materialization.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + return n +} +func (m *ProducerFrontier) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Producer != 0 { + n += 1 + sovShuffle(uint64(m.Producer)) + } + if m.LastCommit != 0 { + n += 9 + } + if m.HintedCommit != 0 { + n += 9 + } + if m.Offset != 0 { + n += 1 + sovShuffle(uint64(m.Offset)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *JournalFrontier) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.JournalNameTruncateDelta != 0 { + n += 1 + sovShuffle(uint64(m.JournalNameTruncateDelta)) + } + l = len(m.JournalNameSuffix) + if l > 0 { + n += 1 + l + sovShuffle(uint64(l)) + } + if m.Binding != 0 { + n += 1 + sovShuffle(uint64(m.Binding)) + } + if m.BytesReadDelta != 0 { + n += 1 + sovShuffle(uint64(m.BytesReadDelta)) + } + if m.BytesBehindDelta != 0 { + n += 1 + sovShuffle(uint64(m.BytesBehindDelta)) + } + if len(m.Producers) > 0 { + for _, e := range m.Producers { + l = e.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Frontier) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Journals) > 0 { + for _, e := range m.Journals { + l = e.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + } + if len(m.FlushedLsn) > 0 { + l = 0 + for _, e := range m.FlushedLsn { + l += sovShuffle(uint64(e)) + } + n += 1 + sovShuffle(uint64(l)) + l + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SessionRequest) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Open != nil { + l = m.Open.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.ResumeCheckpoint != nil { + l = m.ResumeCheckpoint.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.NextCheckpoint != nil { + l = m.NextCheckpoint.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SessionRequest_Open) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Task != nil { + l = m.Task.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if len(m.Shards) > 0 { + for _, e := range m.Shards { + l = e.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SessionRequest_NextCheckpoint) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SessionResponse) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Opened != nil { + l = m.Opened.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.NextCheckpoint != nil { + l = m.NextCheckpoint.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SessionResponse_Opened) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SliceRequest) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Open != nil { + l = m.Open.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.Start != nil { + l = m.Start.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.StartRead != nil { + l = m.StartRead.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.Progress != nil { + l = m.Progress.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SliceRequest_Open) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SessionId != 0 { + n += 5 + } + if m.Task != nil { + l = m.Task.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if len(m.Shards) > 0 { + for _, e := range m.Shards { + l = e.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + } + if m.ShardIndex != 0 { + n += 1 + sovShuffle(uint64(m.ShardIndex)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SliceRequest_Start) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SliceRequest_StartRead) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Binding != 0 { + n += 1 + sovShuffle(uint64(m.Binding)) + } + if m.Spec != nil { + l = m.Spec.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.CreateRevision != 0 { + n += 1 + sovShuffle(uint64(m.CreateRevision)) + } + if m.ModRevision != 0 { + n += 1 + sovShuffle(uint64(m.ModRevision)) + } + if m.Route != nil { + l = m.Route.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if len(m.Checkpoint) > 0 { + for _, e := range m.Checkpoint { + l = e.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SliceRequest_Progress) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SliceResponse) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Opened != nil { + l = m.Opened.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.ListingAdded != nil { + l = m.ListingAdded.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.Progressed != nil { + l = m.Progressed.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SliceResponse_Opened) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SliceResponse_ListingAdded) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Binding != 0 { + n += 1 + sovShuffle(uint64(m.Binding)) + } + if m.Spec != nil { + l = m.Spec.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.CreateRevision != 0 { + n += 1 + sovShuffle(uint64(m.CreateRevision)) + } + if m.ModRevision != 0 { + n += 1 + sovShuffle(uint64(m.ModRevision)) + } + if m.Route != nil { + l = m.Route.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *LogRequest) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Open != nil { + l = m.Open.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.Append != nil { + l = m.Append.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.Flush != nil { + l = m.Flush.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *LogRequest_Open) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SessionId != 0 { + n += 5 + } + if len(m.Shards) > 0 { + for _, e := range m.Shards { + l = e.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + } + if m.SliceShardIndex != 0 { + n += 1 + sovShuffle(uint64(m.SliceShardIndex)) + } + if m.LogShardIndex != 0 { + n += 1 + sovShuffle(uint64(m.LogShardIndex)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *LogRequest_Append) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.JournalNameTruncateDelta != 0 { + n += 1 + sovShuffle(uint64(m.JournalNameTruncateDelta)) + } + l = len(m.JournalNameSuffix) + if l > 0 { + n += 1 + l + sovShuffle(uint64(l)) + } + if m.Binding != 0 { + n += 1 + sovShuffle(uint64(m.Binding)) + } + if m.Priority != 0 { + n += 1 + sovShuffle(uint64(m.Priority)) + } + if m.ReadDelay != 0 { + n += 1 + sovShuffle(uint64(m.ReadDelay)) + } + if m.Producer != 0 { + n += 1 + sovShuffle(uint64(m.Producer)) + } + if m.Clock != 0 { + n += 9 + } + if m.Flags != 0 { + n += 1 + sovShuffle(uint64(m.Flags)) + } + l = len(m.PackedKey) + if l > 0 { + n += 1 + l + sovShuffle(uint64(l)) + } + l = len(m.DocArchived) + if l > 0 { + n += 1 + l + sovShuffle(uint64(l)) + } + if m.SourceByteLength != 0 { + n += 1 + sovShuffle(uint64(m.SourceByteLength)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *LogRequest_Flush) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Cycle != 0 { + n += 1 + sovShuffle(uint64(m.Cycle)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *LogResponse) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Opened != nil { + l = m.Opened.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.Flushed != nil { + l = m.Flushed.ProtoSize() + n += 1 + l + sovShuffle(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *LogResponse_Opened) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *LogResponse_Flushed) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Cycle != 0 { + n += 1 + sovShuffle(uint64(m.Cycle)) + } + if m.FlushedLsn != 0 { + n += 1 + sovShuffle(uint64(m.FlushedLsn)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func sovShuffle(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozShuffle(x uint64) (n int) { + return sovShuffle(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *Shard) 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 ErrIntOverflowShuffle + } + 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: Shard: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Shard: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Range", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Range == nil { + m.Range = &flow.RangeSpec{} + } + if err := m.Range.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Endpoint", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Endpoint = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Directory", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Directory = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *CollectionPartitions) 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 ErrIntOverflowShuffle + } + 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: CollectionPartitions: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CollectionPartitions: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Collection", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Collection == nil { + m.Collection = &flow.CollectionSpec{} + } + if err := m.Collection.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PartitionSelector", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PartitionSelector == nil { + m.PartitionSelector = &protocol.LabelSelector{} + } + if err := m.PartitionSelector.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *Task) 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 ErrIntOverflowShuffle + } + 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: Task: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Task: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CollectionPartitions", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &CollectionPartitions{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Task = &Task_CollectionPartitions{v} + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Derivation", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &flow.CollectionSpec{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Task = &Task_Derivation{v} + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Materialization", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &flow.MaterializationSpec{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Task = &Task_Materialization{v} + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *ProducerFrontier) 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 ErrIntOverflowShuffle + } + 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: ProducerFrontier: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ProducerFrontier: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Producer", wireType) + } + m.Producer = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Producer |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field LastCommit", wireType) + } + m.LastCommit = 0 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + m.LastCommit = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + case 3: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field HintedCommit", wireType) + } + m.HintedCommit = 0 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + m.HintedCommit = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Offset", wireType) + } + m.Offset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Offset |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *JournalFrontier) 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 ErrIntOverflowShuffle + } + 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: JournalFrontier: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: JournalFrontier: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field JournalNameTruncateDelta", wireType) + } + m.JournalNameTruncateDelta = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.JournalNameTruncateDelta |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field JournalNameSuffix", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.JournalNameSuffix = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Binding", wireType) + } + m.Binding = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Binding |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BytesReadDelta", wireType) + } + m.BytesReadDelta = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BytesReadDelta |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BytesBehindDelta", wireType) + } + m.BytesBehindDelta = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BytesBehindDelta |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Producers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Producers = append(m.Producers, &ProducerFrontier{}) + if err := m.Producers[len(m.Producers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *Frontier) 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 ErrIntOverflowShuffle + } + 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: Frontier: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Frontier: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Journals", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Journals = append(m.Journals, &JournalFrontier{}) + if err := m.Journals[len(m.Journals)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.FlushedLsn = append(m.FlushedLsn, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.FlushedLsn) == 0 { + m.FlushedLsn = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.FlushedLsn = append(m.FlushedLsn, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field FlushedLsn", wireType) + } + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SessionRequest) 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 ErrIntOverflowShuffle + } + 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: SessionRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SessionRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Open", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Open == nil { + m.Open = &SessionRequest_Open{} + } + if err := m.Open.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResumeCheckpoint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ResumeCheckpoint == nil { + m.ResumeCheckpoint = &Frontier{} + } + if err := m.ResumeCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NextCheckpoint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.NextCheckpoint == nil { + m.NextCheckpoint = &SessionRequest_NextCheckpoint{} + } + if err := m.NextCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SessionRequest_Open) 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 ErrIntOverflowShuffle + } + 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: Open: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Open: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Task == nil { + m.Task = &Task{} + } + if err := m.Task.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Shards = append(m.Shards, &Shard{}) + if err := m.Shards[len(m.Shards)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SessionRequest_NextCheckpoint) 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 ErrIntOverflowShuffle + } + 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: NextCheckpoint: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: NextCheckpoint: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SessionResponse) 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 ErrIntOverflowShuffle + } + 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: SessionResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SessionResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Opened", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Opened == nil { + m.Opened = &SessionResponse_Opened{} + } + if err := m.Opened.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NextCheckpoint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.NextCheckpoint == nil { + m.NextCheckpoint = &Frontier{} + } + if err := m.NextCheckpoint.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SessionResponse_Opened) 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 ErrIntOverflowShuffle + } + 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: Opened: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Opened: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SliceRequest) 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 ErrIntOverflowShuffle + } + 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: SliceRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SliceRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Open", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Open == nil { + m.Open = &SliceRequest_Open{} + } + if err := m.Open.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Start", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Start == nil { + m.Start = &SliceRequest_Start{} + } + if err := m.Start.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StartRead", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StartRead == nil { + m.StartRead = &SliceRequest_StartRead{} + } + if err := m.StartRead.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Progress", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Progress == nil { + m.Progress = &SliceRequest_Progress{} + } + if err := m.Progress.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SliceRequest_Open) 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 ErrIntOverflowShuffle + } + 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: Open: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Open: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 5 { + return fmt.Errorf("proto: wrong wireType = %d for field SessionId", wireType) + } + m.SessionId = 0 + if (iNdEx + 4) > l { + return io.ErrUnexpectedEOF + } + m.SessionId = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) + iNdEx += 4 + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Task", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Task == nil { + m.Task = &Task{} + } + if err := m.Task.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Shards = append(m.Shards, &Shard{}) + if err := m.Shards[len(m.Shards)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ShardIndex", wireType) + } + m.ShardIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ShardIndex |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SliceRequest_Start) 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 ErrIntOverflowShuffle + } + 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: Start: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Start: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SliceRequest_StartRead) 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 ErrIntOverflowShuffle + } + 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: StartRead: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StartRead: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Binding", wireType) + } + m.Binding = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Binding |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Spec", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Spec == nil { + m.Spec = &protocol.JournalSpec{} + } + if err := m.Spec.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CreateRevision", wireType) + } + m.CreateRevision = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CreateRevision |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ModRevision", wireType) + } + m.ModRevision = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ModRevision |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Route", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Route == nil { + m.Route = &protocol.Route{} + } + if err := m.Route.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Checkpoint", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Checkpoint = append(m.Checkpoint, &ProducerFrontier{}) + if err := m.Checkpoint[len(m.Checkpoint)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SliceRequest_Progress) 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 ErrIntOverflowShuffle + } + 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: Progress: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Progress: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SliceResponse) 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 ErrIntOverflowShuffle + } + 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: SliceResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SliceResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Opened", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Opened == nil { + m.Opened = &SliceResponse_Opened{} + } + if err := m.Opened.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ListingAdded", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ListingAdded == nil { + m.ListingAdded = &SliceResponse_ListingAdded{} + } + if err := m.ListingAdded.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Progressed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Progressed == nil { + m.Progressed = &Frontier{} + } + if err := m.Progressed.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SliceResponse_Opened) 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 ErrIntOverflowShuffle + } + 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: Opened: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Opened: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *SliceResponse_ListingAdded) 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 ErrIntOverflowShuffle + } + 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: ListingAdded: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ListingAdded: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Binding", wireType) + } + m.Binding = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Binding |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Spec", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Spec == nil { + m.Spec = &protocol.JournalSpec{} + } + if err := m.Spec.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CreateRevision", wireType) + } + m.CreateRevision = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CreateRevision |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ModRevision", wireType) + } + m.ModRevision = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ModRevision |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Route", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Route == nil { + m.Route = &protocol.Route{} + } + if err := m.Route.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *LogRequest) 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 ErrIntOverflowShuffle + } + 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: LogRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LogRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Open", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Open == nil { + m.Open = &LogRequest_Open{} + } + if err := m.Open.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Append", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Append == nil { + m.Append = &LogRequest_Append{} + } + if err := m.Append.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Flush", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Flush == nil { + m.Flush = &LogRequest_Flush{} + } + if err := m.Flush.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *LogRequest_Open) 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 ErrIntOverflowShuffle + } + 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: Open: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Open: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 5 { + return fmt.Errorf("proto: wrong wireType = %d for field SessionId", wireType) + } + m.SessionId = 0 + if (iNdEx + 4) > l { + return io.ErrUnexpectedEOF + } + m.SessionId = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) + iNdEx += 4 + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Shards = append(m.Shards, &Shard{}) + if err := m.Shards[len(m.Shards)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SliceShardIndex", wireType) + } + m.SliceShardIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SliceShardIndex |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LogShardIndex", wireType) + } + m.LogShardIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LogShardIndex |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *LogRequest_Append) 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 ErrIntOverflowShuffle + } + 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: Append: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Append: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field JournalNameTruncateDelta", wireType) + } + m.JournalNameTruncateDelta = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.JournalNameTruncateDelta |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field JournalNameSuffix", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.JournalNameSuffix = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Binding", wireType) + } + m.Binding = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Binding |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Priority", wireType) + } + m.Priority = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Priority |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ReadDelay", wireType) + } + m.ReadDelay = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ReadDelay |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Producer", wireType) + } + m.Producer = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Producer |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Clock", wireType) + } + m.Clock = 0 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + m.Clock = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Flags", wireType) + } + m.Flags = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Flags |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PackedKey", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PackedKey = append(m.PackedKey[:0], dAtA[iNdEx:postIndex]...) + if m.PackedKey == nil { + m.PackedKey = []byte{} + } + iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DocArchived", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DocArchived = append(m.DocArchived[:0], dAtA[iNdEx:postIndex]...) + if m.DocArchived == nil { + m.DocArchived = []byte{} + } + iNdEx = postIndex + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceByteLength", wireType) + } + m.SourceByteLength = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SourceByteLength |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *LogRequest_Flush) 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 ErrIntOverflowShuffle + } + 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: Flush: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Flush: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Cycle", wireType) + } + m.Cycle = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Cycle |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *LogResponse) 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 ErrIntOverflowShuffle + } + 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: LogResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LogResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Opened", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Opened == nil { + m.Opened = &LogResponse_Opened{} + } + if err := m.Opened.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + 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 ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthShuffle + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthShuffle + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Flushed == nil { + m.Flushed = &LogResponse_Flushed{} + } + if err := m.Flushed.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *LogResponse_Opened) 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 ErrIntOverflowShuffle + } + 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: Opened: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Opened: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 *LogResponse_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 ErrIntOverflowShuffle + } + 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 != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Cycle", wireType) + } + m.Cycle = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Cycle |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FlushedLsn", wireType) + } + m.FlushedLsn = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowShuffle + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.FlushedLsn |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipShuffle(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthShuffle + } + 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 skipShuffle(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowShuffle + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowShuffle + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowShuffle + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthShuffle + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupShuffle + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthShuffle + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthShuffle = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowShuffle = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupShuffle = fmt.Errorf("proto: unexpected end of group") +) diff --git a/go/protocols/shuffle/shuffle.proto b/go/protocols/shuffle/shuffle.proto index ba97ba25b49..a79ca873b15 100644 --- a/go/protocols/shuffle/shuffle.proto +++ b/go/protocols/shuffle/shuffle.proto @@ -3,11 +3,16 @@ syntax = "proto3"; package shuffle; import "broker/protocol/protocol.proto"; -import "consumer/protocol/protocol.proto"; import "go/protocols/flow/flow.proto"; +import "gogoproto/gogo.proto"; option go_package = "github.com/estuary/flow/go/protocols/shuffle"; +option (gogoproto.marshaler_all) = true; +option (gogoproto.protosizer_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.goproto_getters_all) = false; + // Shard represents a participant in the shuffle topology (e.x. a task shard). message Shard { // Key and r-clock document range owned by this shard. @@ -25,9 +30,6 @@ message CollectionPartitions { flow.CollectionSpec collection = 1; // Partition selector for filtering source collection journals. protocol.LabelSelector partition_selector = 2; - // Disk backlog threshold in bytes before engaging back-pressure. - // Used for ad-hoc reads where the caller controls the threshold. - uint64 disk_backlog_threshold = 3; } // Task which we're performing shuffles for. @@ -89,14 +91,11 @@ message JournalFrontier { repeated ProducerFrontier producers = 6; } -// FrontierChunk is a portion of a frontier sequence for streaming. +// Frontier is a complete frontier (or frontier delta) of journal progress. // Entries are sorted and unique on (journal name, binding). -// A final empty chunk (empty `journals`) represents end-of-sequence. -message FrontierChunk { +message Frontier { repeated JournalFrontier journals = 1; // Per-shard flushed LSN, indexed by shard_index. - // Populated only on the terminal (empty-journals) chunk of a - // Progressed or NextCheckpoint sequence. Empty otherwise. repeated uint64 flushed_lsn = 2; } @@ -113,7 +112,7 @@ message SessionRequest { Open open = 1; // The resume checkpoint: the non-delta frontier from which the - // session is to resume. It's streamed by the Coordinator client after reading + // session is to resume. It's sent by the Coordinator client after reading // SessionResponse.Opened. // // This is a comprehensive checkpoint, reflecting all journals and producers @@ -134,7 +133,7 @@ message SessionRequest { // the first NextCheckpoint of the current session will match the last // unfinished NextCheckpoint of its prior session, which enables the transaction // to be idempotent. - FrontierChunk resume_checkpoint_chunk = 2; + Frontier resume_checkpoint = 2; // NextCheckpoint requests the next available checkpoint delta. // This is a blocking request: the Session only responds when progress is @@ -143,7 +142,7 @@ message SessionRequest { // // A NextCheckpoint response may be a fully-resolved checkpoint OR a "peek" // of an in-flight frontier whose causal hints have not yet fully resolved. - // See SessionResponse.next_checkpoint_chunk for the contract. + // See SessionResponse.next_checkpoint for the contract. message NextCheckpoint {} NextCheckpoint next_checkpoint = 3; } @@ -178,7 +177,7 @@ message SessionResponse { // initialize `hinted_commit` of future session resume checkpoints. Then, // upon its durable completion of all downstream processing related to a // fully-resolved NextCheckpoint, it should merge it into its base checkpoint. - FrontierChunk next_checkpoint_chunk = 2; + Frontier next_checkpoint = 2; } // SliceRequest is sent by the Session to each shard's Slice RPC. @@ -255,7 +254,7 @@ message SliceResponse { // Progressed reports the frontier of log progress since the last Progressed. // It's sent in response to a Request.Progress, and only once at least one // flush cycle has completed since the prior Progressed. - FrontierChunk progressed = 3; + Frontier progressed = 3; } // LogRequest is sent by Slices to each shard's Log. @@ -273,8 +272,6 @@ message LogRequest { uint32 slice_shard_index = 3; // Index of the target Log shard within the session's shard list. uint32 log_shard_index = 4; - // Disk backlog threshold in bytes before engaging back-pressure. - uint64 disk_backlog_threshold = 5; } Open open = 1; diff --git a/mise/tasks/build/go-protobufs b/mise/tasks/build/go-protobufs index 5b2656de41d..7703c22f246 100755 --- a/mise/tasks/build/go-protobufs +++ b/mise/tasks/build/go-protobufs @@ -10,6 +10,7 @@ PROTO_FILES=( "./go/protocols/materialize/materialize.proto" "./go/protocols/ops/ops.proto" "./go/protocols/runtime/runtime.proto" + "./go/protocols/shuffle/shuffle.proto" ) # Modules needed for protoc include paths diff --git a/mise/tasks/build/rust-protobufs b/mise/tasks/build/rust-protobufs index 0f3947c52d7..0098fd67b9d 100755 --- a/mise/tasks/build/rust-protobufs +++ b/mise/tasks/build/rust-protobufs @@ -2,3 +2,4 @@ #MISE description="Generate Rust protobuf bindings" cargo build -p proto-gazette -p proto-flow -p proto-grpc --all-features +cargo fmt diff --git a/plans/runtime-v2/plan.md b/plans/runtime-v2/plan.md new file mode 100644 index 00000000000..63ef8a52943 --- /dev/null +++ b/plans/runtime-v2/plan.md @@ -0,0 +1,238 @@ +# Runtime V2 + +## Objectives + +Replace the Go-driven transaction loop with a Rust-driven architecture. +For derivations and materializations, a **Shuffle Leader** coordinates +multi-shard transactions, the **shuffle** crate replaces the Go shuffle, +and **journal publishing** moves from Go to Rust. Captures follow a +simpler model with independent per-shard transaction loops. The Go +runtime becomes a thin shim for shard lifecycle and ops logs. + +Why: +- **Multi-shard coordination** (derive/materialize): The Go shuffle has + known scaling limitations (per-shard×journal RPCs, in-memory staging, + no coordinated checkpoints). The Shuffle Leader + Rust shuffle solve + these architecturally. +- **Consolidated state** (derive/materialize): Scale-out shards persist + through shard zero's recovery log, eliminating per-shard recovery logs + and simplifying crash recovery. +- **Fewer moving parts**: Transaction lifecycle, publishing, stats, and + shuffle all move to Rust. Go retains only Etcd lifecycle and ops logs. + +There are no multi-shard tasks in production today. Multi-shard +coordination (Shuffle Leader, shard-zero consolidation) will be +validated first with test workloads and introduced gradually +alongside production single-shard migration. + +## Architecture + +``` +Reactor machine + ├─ reactor process(es) (Go + Rust via CGO) + │ ├─ Go: Etcd watch, shard lifecycle, ops logs (OUTSIDE_TXN) + │ └─ Per-shard TaskService (Rust, via CGO): + │ ├─ Connector driving, combining, publishing CONTINUE_TXN docs + │ ├─ In-memory state (connector state, checkpoints, max-keys) + │ ├─ Derive/materialize: state persisted via Leader's Persist/Persisted + │ │ (RocksDB + Go Recorder on the shard hosting the recovery log) + │ └─ Capture: per-shard RocksDB with Go Recorder + │ + └─ shuffle sidecar process (Rust, one per machine, systemd-supervised) + ├─ Shuffle Leader service (per-task, via join pattern) + ├─ Shuffle service (Session/Slice/Log RPCs) + └─ Listens on the fixed shuffle port (same across the fleet) +``` + +Three layers interact: + +1. **Go runtime** (`go/runtime/`) observes Etcd for shard lifecycle. + On assignment, it spawns a CGO TaskService for runtime-next and + sends a `RocksDBDescriptor`. On term changes (spec updates in + Etcd), it cancels the term context for graceful restart. It writes + ops logs as `OUTSIDE_TXN` documents. Feature flags on shard labels + determine whether to call runtime-next or the existing runtime + crate. + + The Gazette consumer framework's transaction lifecycle is + **bypassed entirely**: `StartReadingMessages` drains without + producing messages, so `BeginTxn`/`ConsumeMessage`/`FinalizeTxn`/ + `StartCommit` are never invoked. The framework still manages + assignment, Etcd state, and recovery log setup; all document + processing and commit sequencing happen in Rust via the Shuffle + Leader protocol (or, for captures, a per-shard transaction loop). + +2. **Per-shard `TaskService`** (`crates/runtime-next/`) runs in the + reactor process via CGO — one per shard, created and torn down by + `taskBase` in response to shard assignments. Hosts the `Shard` + gRPC service over a per-shard UDS. Task-scoped logs cross back to + Go via the existing pipe-FD mechanism. + + - For derive/materialize: all mutable state (connector state, + checkpoints, max-keys) is held in memory; state changes flow + through the Shuffle Leader protocol and land in shard zero's + RocksDB. + - For captures: each shard manages its own state independently, + persisting to its own RocksDB via its own recovery log. + +3. **Shuffle sidecar process** (pure Rust), one per reactor machine, + supervised by systemd with the same lifetime as the reactor + process(es). Hosts two gRPC services for *all* tasks on the machine: + + - **Shuffle Leader** (`crates/runtime-next/src/leader/`): accepts + shard streams for tasks whose shard zero is assigned to a reactor + on this machine. + - **Shuffle** (`crates/shuffle/`, Session/Slice/Log RPCs): accepts + shuffle RPCs from any reactor participating in a task. + + Both listen on the **shuffle port**: a fixed CLI argument, shared + fleet-wide. A reactor reaches any peer sidecar by replacing the + port of that peer's `ProcessSpec.endpoint`. One sidecar serves + multiple co-located reactor processes on the same machine (used + by local stacks). + + Sidecar tracing goes to the sidecar's own stderr as application + logs, not task logs. Captures don't use the Shuffle Leader or + shuffle services, but the sidecar runs uniformly regardless. + + *Why a sidecar rather than in-process via CGO.* The shuffle stack + and per-shard TaskService already communicate solely by gRPC with + no shared memory, so the process boundary aligns with an existing + logical one. Separation buys an independent monitoring domain and + a smaller reactor Rust + CGO surface. A sidecar crash fail-stops + every joined session on the machine. + +The sidecar and per-shard TaskServices communicate only via the +Shuffle Leader protocol and the shuffle Session/Slice/Log RPCs over +gRPC — never via shared memory. + +## Where to start reading + +Leader and shard implementations both live in `crates/runtime-next/`. + +- **Runtime protocol** — `go/protocols/runtime/runtime.proto` defines + the bidirectional `Leader` (sidecar) and `Shard` (per-shard, + controller-facing) RPCs. Both carry the same message types (`Derive`, + `Materialize`); messages and their fields are documented inline. +- **Sidecar Shuffle Leader** — `crates/runtime-next/src/leader/`. + - `leader/service.rs`, `leader/materialize/handler.rs`: gRPC entry + points and per-task Join rendezvous. + - `leader/materialize/startup.rs`: Recover/Open/Apply/Recovered phase. + - `leader/materialize/fsm.rs`, `leader/materialize/actor.rs`: + pipelined HeadFSM/TailFSM driving open / commit / acknowledge / + trigger. + - `shard/recovery.rs`: encode/decode of `Persist` payloads to RocksDB + `WriteBatch` operations and recovery iteration to in-memory state. + Single source of truth for the on-disk key layout (`FH:`, `FC:`, + `AI:`, `MK-v2:`, `connector-state`, `trigger-params`, `last-applied`). + - `patches.rs`: wire format for connector state patch streams shared + by leader-side state reduction and shard-side persistence. + - `leader/materialize/frontier_mapping.rs`: bi-directional mapping between + `consumer.Checkpoint` and `shuffle::Frontier`. + - `triggers.rs`, `publish.rs`: webhook trigger delivery and + leader-side journal publishing of stats / ACK intents. +- **Per-shard TaskService** — `crates/runtime-next/src/`. + - `task_service.rs`, `handler.rs`: CGO entry point and `Shard` gRPC + handler. + - `shard/materialize/`: per-shard transaction loop, connector RPC + bridging, and (on shard zero) RocksDB persistence. + - `shard/rocksdb.rs`: the single Persist application code path; capture + will reuse it by synthesizing Persist messages locally. + +## Key invariants + +These are load-bearing rules the implementation enforces and that any +new code must continue to honor: + +- **Crate dependency direction.** Leader and shard implementations both + live in `runtime-next`. The legacy `runtime` crate does not depend on + `runtime-next`, and `runtime-next` MUST NOT depend on `runtime`. + `runtime` is being minimally changed to trivially ensure we don't break it. + Prefer a little copying to a little dependency. + +- **Shard-local processing is identical for all shards.** Shard zero + is special only in that it hosts the recovery log and receives + leader-directed Apply/Persist operations. Shard-zero conditional + branches are confined to startup (forwarding `Task` to the leader, + which only one shard may do) and to receiving Apply/Persist; the + per-transaction loop has no "am I shard zero?" tests — the leader + decides what each shard does and shards follow uniformly. + +- **All shards participate in every transaction**, even idle ones — + they send empty deltas and respond immediately. Shard topology is + fail-stop: any shard drop aborts the session and tears down all + surviving shards. The Gazette allocator reassigns; the next session + re-joins from PRIMARY. + +- **Non-zero derive/materialize shards have no recovery log.** The + consumer framework supports this via `ShardSpec.recovery_log_prefix + = ""`; non-zero shards spin up instantly and acquire state via the + leader protocol. Shard zero's recovery dominates session startup + latency. + + Non-zero shards still open a (typically empty, tempdir-backed) + RocksDB and run the same `scan` path on session start. This is + intentional: during migration a non-zero shard may inherit + pre-existing on-disk state, and the recovery scan must surface that + rather than silently ignore it. Recovery is expected to error if a + non-zero shard observes unexpected committed state. + + The same migration guard applies to connector-reported runtime + checkpoints at `Opened`: non-zero shards are expected to report no + checkpoint state. A non-zero checkpoint is treated as evidence that + the task still has per-shard committed state, and startup fails + explicitly instead of proceeding with an unsafe consolidation. + +## Capture architecture + +Capture shards operate **independently** — each shard has its own +recovery log, RocksDB, connector instance, and publisher. Capture +connector state represents per-shard cursors (e.g., CDC LSNs) with +no cross-shard coordination, so shards must make independent progress. +A capture shard failure affects only that shard. + +Capture is unimplemented in `runtime-next` at the time of writing. +The intent is to reuse `runtime-next`'s single Persist application +path by synthesizing `Persist` messages locally inside the capture +shard, rather than receiving them from a leader. + +## Migration strategy + +The complete runtime-next — Shuffle Leader, Rust shuffle, Rust +publishing, and per-task-type transaction loops — is built and +**deployed inert**. The new code ships alongside the existing runtime +with no tasks using it. Task migration is then controlled via +per-task feature flags on shard labels, with per-task rollback if +issues arise. + +This avoids partial implementations that would require migrating +tasks twice (to a partial runtime, then again to the final one). +Risk is managed through rollout pacing: tasks activate into the +complete runtime in stages of increasing blast radius. + +### Rollout sequence + +1. **Low-value single-shard captures**: exercises the Rust transaction + loop and publishing with the simplest task type. +2. **Single-shard derive/materialize**: adds the Rust shuffle and + leader protocol at N=1: one shard joins with itself, all RPCs local. + All production derive/materialize tasks are single-shard today. +3. **Multi-shard derive/materialize**: full leader coordination at + N>1. No multi-shard tasks exist in production today; they will + be introduced with test workloads first. +4. **Higher-value tasks and full migration**: as confidence grows, + progressively migrate remaining tasks. Go shuffle deleted after + full migration. + +### Rollout mechanics + +- **Per-task**: feature flags on shard labels select old vs new runtime. + All shards of a task use the same runtime. +- **Coexistence**: old-runtime and new-runtime tasks run on the same + reactor. The shuffle sidecar runs uniformly on every reactor + machine regardless of which tasks are assigned; old-runtime tasks + simply don't talk to it. The only change to the existing `runtime` + crate is Frontier-aware rollback (the migration swap on startup, + using `runtime_next::leader::materialize::frontier_mapping`). +- **Rollback**: switching a task's feature flag back to the old runtime + is a per-task operation. No global rollback needed. diff --git a/plans/runtime-v2/preview-harness.md b/plans/runtime-v2/preview-harness.md new file mode 100644 index 00000000000..c3d219ec594 --- /dev/null +++ b/plans/runtime-v2/preview-harness.md @@ -0,0 +1,203 @@ +# `flowctl raw preview-next` as a runtime-next E2E harness + +This is a hands-on guide for using `flowctl raw preview-next` as a +repeatable end-to-end test of the `runtime-next` + `leader` + `shuffle` +stack against a local Postgres database. It assumes the runtime-v2 +branch is checked out and built. + +`flowctl preview` is the legacy harness against the existing runtime +crate; the runtime-next harness lives under `raw` while the new stack +is in development. They share most flags and the same test spec format. + +## One-time setup + +Done once per workstation. Skip the steps you've already completed. + +### 1. Local Postgres + +A Postgres reachable at `localhost:5432` with `postgres / postgres` +credentials (this matches the dev `supabase` instance the repo already +ships with). Quick verify: + +```bash +psql postgresql://postgres:postgres@localhost:5432/postgres -c 'SELECT 1;' +``` + +### 2. Build a native materialize-postgres binary + +The published `ghcr.io/estuary/materialize-postgres:dev` image only +ships `linux/amd64`. On ARM hosts you can build the connector natively +from the sibling `connectors` repo. We use `local:` mode in the spec to +drive the binary directly, avoiding container plumbing entirely: + +```bash +cd /home/johnny/estuary/connectors/materialize-postgres +go build -o /tmp/materialize-postgres . +``` + +Re-run when the connector source changes. Any other `materialize-*` +connector under `connectors/` works the same way. + +### 3. Build flowctl + +From this repo: + +```bash +cd /home/johnny/estuary/flow +cargo build -p flowctl --bin flowctl +``` + +The resulting binary is at `/home/johnny/cargo-target/debug/flowctl`. + +## Repeatable E2E run + +### 1. The harness spec + +Live at `/tmp/preview-test/local.flow.yaml` (or wherever you keep it). +The `local:` endpoint plus `protobuf: true` skips Docker, runs the +connector as a child process. + +```yaml +materializations: + test/preview/wiki: + endpoint: + local: + command: + - /tmp/materialize-postgres + config: + address: localhost:5432 + user: postgres + credentials: + auth_type: UserPassword + password: postgres + database: postgres + schema: public + protobuf: true + shards: + logLevel: info + bindings: + - source: demo/wikipedia/recentchange-sampled + resource: + table: preview_wiki + schema: public +``` + +Notes: +- The `--name` you'll pass to `flowctl raw preview-next` is the + materialization name — `test/preview/wiki` here. +- Source is a real production collection. flowctl auths reads via your + flowctl token (`~/.flowctl/config-default.yaml`). +- Pick a `resource.table` name that's unique per scenario you're + exercising — leftover state from prior runs (the checkpoint table + `flow_checkpoints_v1` and the per-binding table) will block + re-validation. See **Reset Postgres state** below. + +### 2. Reset Postgres state + +Each fresh run requires the bindings table absent (the connector +refuses to bind a new materialization onto a pre-existing table) and +the materialization checkpoint table clean (otherwise `Apply` is a +re-attach against stale state): + +```bash +psql postgresql://postgres:postgres@localhost:5432/postgres -c ' + DROP TABLE IF EXISTS public.preview_wiki, + public.flow_checkpoints_v1 + CASCADE;' +``` + +If you change the binding's `table` in the spec, drop the *old* table +too — the connector will create the new one but refuses to overwrite +either. + +### 3. Run the harness + +The minimal invocation: + +```bash +cd /tmp/preview-test +RUST_BACKTRACE=1 RUST_LOG=h2=info,info /home/johnny/cargo-target/debug/flowctl raw preview-next \ + --source ./local.flow.yaml \ + --name test/preview/wiki \ + --sessions=-1 \ + --timeout 60s \ + 2> preview.stderr +``` + +Flags worth knowing: +- `--sessions=-1` — one unbounded session (default is also one + unbounded session). Use `--sessions 2,2,2` to exercise cross-session + recovery: three sessions of two transactions each, against a single + persistent shard-zero RocksDB tempdir. +- `--shards N` — synthetic N-shard topology. N=1 (default) hits the + fast-path Join consensus; N≥2 exercises full multi-shard rendezvous. + The `materialize-postgres` spec above is not a valid N>1 materialization + workload: each shard drives an independent connector transaction against + the same table. Use a connector/spec designed for + multi-shard materialization before treating N>1 results as runtime signal. +- `--timeout 60s` — graceful stop trigger. Set high enough that the + close-policy can fire on whatever your source produces. +- `--log-json` — JSON ops logs to stderr. Off by default; useful when + feeding the run into log tooling. + +Per-transaction observability is via `tracing` to stderr (see the +`Publisher::Preview` arm in `crates/runtime-next/src/publish.rs`): + +- `connector applied` — emitted by the leader's apply loop with the + connector's `action_description`, the iteration number, and any + applied connector-state patches (one per loop iteration). +- `transaction stats` — emitted once per committed transaction with the + full `ops::Stats` document (per-binding docs/bytes counts, etc). + +These events are at info level. Filter further with +`RUST_LOG=runtime_next::publish=info` if you want only these and nothing +else. + +### 4. Inspect what landed in Postgres + +Standard psql against the dev DB. The connector creates a checkpoint +metadata table alongside the binding table: + +```bash +# All flow tables +psql postgresql://postgres:postgres@localhost:5432/postgres \ + -c '\dt public.flow_*' \ + -c '\dt public.preview_wiki' + +# Row count + sample rows +psql postgresql://postgres:postgres@localhost:5432/postgres \ + -c 'SELECT count(*) FROM public.preview_wiki;' + +psql postgresql://postgres:postgres@localhost:5432/postgres \ + -c 'SELECT title, "user", wiki, type, timestamp + FROM public.preview_wiki + ORDER BY timestamp DESC + LIMIT 10;' + +# Per-binding committed checkpoint position (one row per shard) +psql postgresql://postgres:postgres@localhost:5432/postgres \ + -c 'SELECT * FROM public.flow_checkpoints_v1;' +``` + +A passing run leaves you with: +- `preview_wiki` populated with N rows where N = shuffled documents + combined per transaction × number of committed transactions. +- `flow_checkpoints_v1` with one row containing this shard's last + committed Frontier. + +## What's exercised by each scenario + +| Scenario | Validates | +|--------------------------------------------|---------------------------------------------------------------------------| +| `--sessions=-1 --timeout 60s` | Single open session, transactions close on `maxTxnDuration` / data volume | +| `--shards 4 --sessions=-1 --timeout 60s` | Multi-shard Join consensus, fan-out shuffle, leader cross-shard reduce with a multi-shard-safe connector/spec | +| `--sessions 2,2,2` | Cross-session recovery — sessions 2 and 3 see non-empty `L:Recover` | +| (Ctrl-C mid-session) | Clean tonic-server shutdown, tempdirs removed, no port left bound | +| `--name ` against a capture spec | Error path: "runtime-next preview supports materializations only…" | + +## Known issues / current state (as of branch `johnny/runtime-v2`) + +- A single connector log line at startup renders as nested ANSI — it + comes from the legacy `runtime` crate's build-time validation path, + which doesn't set `LOG_FORMAT=json`. All runtime-next per-shard + connector logs render cleanly.