From c9c7f2f0507f6981d0e73244cebe5d803b6b825a Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Thu, 7 May 2026 15:33:29 +0000 Subject: [PATCH 01/22] runtime-next: add Rust-driven runtime crate MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Introduce the runtime-next crate housing both the Shuffle Leader and per-shard TaskService implementations behind the bidirectional Leader and Shard RPCs defined in runtime.proto. Materialization is implemented end-to-end (open / commit / acknowledge / trigger plus recovery and Frontier↔Checkpoint mapping); derivations and captures land in follow-on work. Small supporting bits land in the runtime, doc, and ops crates: TaskServiceConfig destructuring tolerates new fields, combine::Accumulator exposes spill-segment ranges, and ops re-exports proto_flow::ops as ops::proto. See plans/runtime-v2/plan.md for the architecture and rollout plan. --- crates/runtime-next/src/leader/materialize/fsm.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/runtime-next/src/leader/materialize/fsm.rs b/crates/runtime-next/src/leader/materialize/fsm.rs index 41391a96a82..fede0530fc0 100644 --- a/crates/runtime-next/src/leader/materialize/fsm.rs +++ b/crates/runtime-next/src/leader/materialize/fsm.rs @@ -644,7 +644,7 @@ impl HeadStore { } // Compose the trigger payload now that we have a complete txn-wide view. - if task.triggers.is_some() { + if task.triggers.is_some() && !extents.bindings.is_empty() { let collection_names: Vec = extents .bindings .keys() From a211243122df9e0acf5dc9e5a688a3a35f5e46d4 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 21:49:05 +0000 Subject: [PATCH 02/22] labels: fix RCLOCK_BEGIN_MIN typo RCLOCK_BEGIN_MIN aliased KEY_BEGIN rather than KEY_BEGIN_MIN. --- crates/labels/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/labels/src/lib.rs b/crates/labels/src/lib.rs index 535f421fb67..32362fd7438 100644 --- a/crates/labels/src/lib.rs +++ b/crates/labels/src/lib.rs @@ -23,7 +23,7 @@ pub const TASK_TYPE_CAPTURE: &str = "capture"; pub const TASK_TYPE_DERIVATION: &str = "derivation"; pub const TASK_TYPE_MATERIALIZATION: &str = "materialization"; pub const RCLOCK_BEGIN: &str = "estuary.dev/rclock-begin"; -pub const RCLOCK_BEGIN_MIN: &str = KEY_BEGIN; +pub const RCLOCK_BEGIN_MIN: &str = KEY_BEGIN_MIN; pub const RCLOCK_END: &str = "estuary.dev/rclock-end"; pub const RCLOCK_END_MAX: &str = KEY_END_MAX; pub const SPLIT_TARGET: &str = "estuary.dev/split-target"; From a91175323ab8a4d100e5e0a723928c383ecab663 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 21:49:47 +0000 Subject: [PATCH 03/22] proto-flow: serialize bytesBehind as a native JSON integer `bytesBehind` is typed as u64 to tally large values, but like `bytesTotal` and `docsTotal` it should serialize as a native JSON integer rather than a quoted string. Extend the build.rs codegen rewrite to cover `bytesBehind`. --- crates/proto-flow/build.rs | 6 +++--- crates/proto-flow/src/ops.serde.rs | 4 ++-- .../proto-flow/tests/snapshots/regression__stats_json.snap | 6 +++--- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/crates/proto-flow/build.rs b/crates/proto-flow/build.rs index e22c5f8c3b0..added534998 100644 --- a/crates/proto-flow/build.rs +++ b/crates/proto-flow/build.rs @@ -40,7 +40,7 @@ fn main() { // to BTreeMap<&str, &RawValue>, and deserialize in reverse. // * Fields ending in "_json_vec" are mapped from Vec to Vec<&RawValue>, // and deserialize in reverse as well. - // * Our stats documents' bytesTotal and docsTotal fields are typed as u64 to allow for + // * Our stats documents' bytesTotal, docsTotal, and bytesBehind fields are typed as u64 to allow for // tallying relatively large values in a single document but we do not want // this value serialized as a string, so we remove the string conversion. @@ -53,7 +53,7 @@ fn main() { regex::Regex::new(r#"struct_ser\.serialize_field\((".+"), &(self\..*_json_vec)\.iter\(\).map\(pbjson::private::base64::encode\).collect::>\(\)\)\?"#) .unwrap(); let ser_int64_re = - regex::Regex::new(r#"struct_ser\.serialize_field\("(bytesTotal|docsTotal)", ToString::to_string\(&self\.(bytes_total|docs_total)\).as_str\(\)\)\?;"#) + regex::Regex::new(r#"struct_ser\.serialize_field\("(bytesTotal|docsTotal|bytesBehind)", ToString::to_string\(&self\.(bytes_total|docs_total|bytes_behind)\).as_str\(\)\)\?;"#) .unwrap(); let de_json_re = @@ -121,7 +121,7 @@ fn main() { buf.replace_range(range, &format!("crate::RawJSONDeserialize")); } - // Handle serializing "bytesTotal"/"docsTotal" as an integer rather than a quoted integer. + // Handle serializing stats counters as integers rather than quoted integers. while let Some(capture) = ser_int64_re.captures(&buf) { let range = capture.get(0).unwrap().range(); buf.replace_range( diff --git a/crates/proto-flow/src/ops.serde.rs b/crates/proto-flow/src/ops.serde.rs index b18f3cffd54..1d9777f6cdc 100644 --- a/crates/proto-flow/src/ops.serde.rs +++ b/crates/proto-flow/src/ops.serde.rs @@ -1353,7 +1353,7 @@ impl serde::Serialize for stats::derive::Transform { if self.bytes_behind != 0 { #[allow(clippy::needless_borrow)] #[allow(clippy::needless_borrows_for_generic_args)] - struct_ser.serialize_field("bytesBehind", ToString::to_string(&self.bytes_behind).as_str())?; + struct_ser.serialize_field("bytesBehind", &self.bytes_behind)?; } struct_ser.end() } @@ -1739,7 +1739,7 @@ impl serde::Serialize for stats::MaterializeBinding { if self.bytes_behind != 0 { #[allow(clippy::needless_borrow)] #[allow(clippy::needless_borrows_for_generic_args)] - struct_ser.serialize_field("bytesBehind", ToString::to_string(&self.bytes_behind).as_str())?; + struct_ser.serialize_field("bytesBehind", &self.bytes_behind)?; } struct_ser.end() } diff --git a/crates/proto-flow/tests/snapshots/regression__stats_json.snap b/crates/proto-flow/tests/snapshots/regression__stats_json.snap index e2de5a6c91d..dce700cb0f7 100644 --- a/crates/proto-flow/tests/snapshots/regression__stats_json.snap +++ b/crates/proto-flow/tests/snapshots/regression__stats_json.snap @@ -39,7 +39,7 @@ expression: json_test(msg) "bytesTotal": 369 }, "lastSourcePublishedAt": "1970-01-01T00:00:06.000000007+00:00", - "bytesBehind": "1000" + "bytesBehind": 1000 }, "otherTransform": { "source": "other/collection", @@ -48,7 +48,7 @@ expression: json_test(msg) "bytesTotal": 2389 }, "lastSourcePublishedAt": "1970-01-01T00:00:06.000000007+00:00", - "bytesBehind": "2000" + "bytesBehind": 2000 } }, "published": { @@ -76,7 +76,7 @@ expression: json_test(msg) "bytesTotal": 300 }, "lastSourcePublishedAt": "1970-01-01T00:00:06.000000007+00:00", - "bytesBehind": "5000" + "bytesBehind": 5000 } }, "interval": { From 8aa75996e87574935226b150855a98ae6535c038 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 21:50:44 +0000 Subject: [PATCH 04/22] models: drop redact annotations from relaxed schemas Relaxed schemas strip validation keywords; `redact` belongs in that set. Pass it through to `RelaxedSchemaObj` with `skip_serializing` so it is dropped, and cover the behavior in both the models unit test and a validation scenario exercising a redacted key with a connector and relaxed write schema. --- crates/models/src/schemas.rs | 29 +++++++++++++++ crates/validation/tests/scenario_tests.rs | 43 +++++++++++++++++++++++ 2 files changed, 72 insertions(+) diff --git a/crates/models/src/schemas.rs b/crates/models/src/schemas.rs index 3e09c32e61e..5c1f4a914f4 100644 --- a/crates/models/src/schemas.rs +++ b/crates/models/src/schemas.rs @@ -215,6 +215,8 @@ struct RelaxedSchemaObj { _min_length: Option, #[serde(rename = "maxLength", default, skip_serializing)] _max_length: Option, + #[serde(rename = "redact", default, skip_serializing)] + _redact: Option, // Other keywords are passed-through. #[serde(flatten)] @@ -416,6 +418,33 @@ mod test { ); } + #[test] + fn test_relaxation_drops_redact() { + let schema = schema!({ + "type": "object", + "properties": { + "email": { + "type": "string", + "redact": { "strategy": "sha256" }, + "description": "Sensitive key component" + } + } + }); + + let relaxed = schema.to_relaxed_schema().unwrap().to_value(); + + assert_eq!( + relaxed, + json!({ + "properties": { + "email": { + "description": "Sensitive key component" + } + } + }) + ); + } + #[test] fn test_relaxation_drops_additional_properties_false() { let schema = schema!({ diff --git a/crates/validation/tests/scenario_tests.rs b/crates/validation/tests/scenario_tests.rs index 7cf4cf0a2cb..0ac15926cab 100644 --- a/crates/validation/tests/scenario_tests.rs +++ b/crates/validation/tests/scenario_tests.rs @@ -1792,6 +1792,49 @@ test://example/catalog.yaml: insta::assert_debug_snapshot!(outcome.errors); } +#[test] +fn test_redacted_key_with_connector_and_relaxed_write_schema() { + let outcome = common::run( + include_str!("schema_inference.yaml"), + r##" +driver: + liveInferredSchemas: + testing/foobar: + properties: + key: + type: string + minimum: null + maximum: null + required: [key] + +test://example/catalog.yaml: + collections: + testing/foobar: + writeSchema: + $defs: + "flow://connector-schema": + $id: "flow://connector-schema" + type: object + properties: + key: + type: string + required: [key] + additionalProperties: true + x-infer-schema: true + allOf: + - $ref: "flow://connector-schema" + properties: + key: + type: string + minimum: null + maximum: null + redact: { strategy: sha256 } +"##, + ); + + insta::assert_debug_snapshot!(outcome.errors, @"[]"); +} + #[test] fn test_collection_inferred_schema_add_blocking() { // A redact annotation on a required write schema property raises an error. From 6b3dfac1b40a5f94e3081d064178e1a08bad8f4b Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 21:51:05 +0000 Subject: [PATCH 05/22] gazette: disable do_not_proxy for file:// fragment reads When a journal read skips the direct-fragment path and the broker returns a `file://` fragment URL, the fragment lives on the broker's local filesystem and the client has no transport to read it. With `do_not_proxy=true` and no open spool file, the broker's `serveRead` short-circuits after sending only fragment metadata, EOFs the stream, and the client loop spins. Clear `do_not_proxy` for `file://` fragments so the broker proxies the content instead. --- crates/gazette/src/journal/read/mod.rs | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/crates/gazette/src/journal/read/mod.rs b/crates/gazette/src/journal/read/mod.rs index e64bf88d998..ee5e4db8347 100644 --- a/crates/gazette/src/journal/read/mod.rs +++ b/crates/gazette/src/journal/read/mod.rs @@ -122,6 +122,16 @@ impl Client { return read_fragment_url(co, fragment, &self.fragment_client, fragment_url, req).await; } + // We skipped the direct-fragment path. If the broker returned a + // `file://` URL, the fragment is persisted but lives on the broker's + // local filesystem — we have no transport to read it ourselves, so + // we must ask the broker to proxy. With `do_not_proxy=true` and no + // open spool file, `serveRead` short-circuits after sending only the + // fragment metadata, EOFs the stream, and the outer loop spins. + if metadata.fragment_url.starts_with("file://") { + req.do_not_proxy = false; + } + tracing::trace!(req.offset, write_head, "started direct journal read"); // Restart as a regular (non-metadata) read, re-picking a routed subclient. From ada16c75761da3f9292eb47fc0fa39a9f10d896f Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 21:54:12 +0000 Subject: [PATCH 06/22] runtime: rename Persist/Persisted nonce field to seq_no The "nonce" name is unrelated to cryptography but trips GitHub's secret scanner. Renaming to "seq_no" sidesteps the false positive without changing protocol semantics. --- crates/proto-flow/src/runtime.rs | 8 +- crates/proto-flow/src/runtime.serde.rs | 50 +- .../src/leader/materialize/fsm.rs | 76 +-- .../src/leader/materialize/startup.rs | 34 +- .../src/shard/materialize/actor.rs | 10 +- .../src/shard/materialize/startup.rs | 2 +- crates/runtime-next/src/shard/rocksdb.rs | 6 +- go/protocols/runtime/runtime.pb.go | 510 +++++++++--------- go/protocols/runtime/runtime.proto | 8 +- 9 files changed, 353 insertions(+), 351 deletions(-) diff --git a/crates/proto-flow/src/runtime.rs b/crates/proto-flow/src/runtime.rs index b4abfa0eebf..f9d128883ff 100644 --- a/crates/proto-flow/src/runtime.rs +++ b/crates/proto-flow/src/runtime.rs @@ -574,7 +574,7 @@ pub struct Recover { /// 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 +/// `seq_no` 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 { @@ -582,7 +582,7 @@ pub struct Persist { /// `Persisted` response. The leader chooses any value and the shard /// does not interpret it. #[prost(uint64, tag = "1")] - pub nonce: u64, + pub seq_no: u64, /// Delete previously-persisted ACK intents. Applies ahead of `ack_intents`. /// Effect: DeleteRange("AI:") #[prost(bool, tag = "2")] @@ -645,9 +645,9 @@ pub struct Persist { /// in the recovery log. #[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] pub struct Persisted { - /// Echoed back from the originating `Persist.nonce` request. + /// Echoed back from the originating `Persist.seq_no` request. #[prost(uint64, tag = "1")] - pub nonce: u64, + pub seq_no: 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 diff --git a/crates/proto-flow/src/runtime.serde.rs b/crates/proto-flow/src/runtime.serde.rs index 89c06951649..a9288fe234a 100644 --- a/crates/proto-flow/src/runtime.serde.rs +++ b/crates/proto-flow/src/runtime.serde.rs @@ -5921,7 +5921,7 @@ impl serde::Serialize for Persist { { use serde::ser::SerializeStruct; let mut len = 0; - if self.nonce != 0 { + if self.seq_no != 0 { len += 1; } if self.delete_ack_intents { @@ -5964,10 +5964,10 @@ impl serde::Serialize for Persist { len += 1; } let mut struct_ser = serializer.serialize_struct("runtime.Persist", len)?; - if self.nonce != 0 { + if self.seq_no != 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.serialize_field("seqNo", ToString::to_string(&self.seq_no).as_str())?; } if self.delete_ack_intents { struct_ser.serialize_field("deleteAckIntents", &self.delete_ack_intents)?; @@ -6032,7 +6032,8 @@ impl<'de> serde::Deserialize<'de> for Persist { D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "nonce", + "seq_no", + "seqNo", "delete_ack_intents", "deleteAckIntents", "ack_intents", @@ -6063,7 +6064,7 @@ impl<'de> serde::Deserialize<'de> for Persist { #[allow(clippy::enum_variant_names)] enum GeneratedField { - Nonce, + SeqNo, DeleteAckIntents, AckIntents, CommittedCloseClock, @@ -6098,7 +6099,7 @@ impl<'de> serde::Deserialize<'de> for Persist { E: serde::de::Error, { match value { - "nonce" => Ok(GeneratedField::Nonce), + "seqNo" | "seq_no" => Ok(GeneratedField::SeqNo), "deleteAckIntents" | "delete_ack_intents" => Ok(GeneratedField::DeleteAckIntents), "ackIntents" | "ack_intents" => Ok(GeneratedField::AckIntents), "committedCloseClock" | "committed_close_clock" => Ok(GeneratedField::CommittedCloseClock), @@ -6131,7 +6132,7 @@ impl<'de> serde::Deserialize<'de> for Persist { where V: serde::de::MapAccess<'de>, { - let mut nonce__ = None; + let mut seq_no__ = None; let mut delete_ack_intents__ = None; let mut ack_intents__ = None; let mut committed_close_clock__ = None; @@ -6147,11 +6148,11 @@ impl<'de> serde::Deserialize<'de> for Persist { 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")); + GeneratedField::SeqNo => { + if seq_no__.is_some() { + return Err(serde::de::Error::duplicate_field("seqNo")); } - nonce__ = + seq_no__ = Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } @@ -6252,7 +6253,7 @@ impl<'de> serde::Deserialize<'de> for Persist { } } Ok(Persist { - nonce: nonce__.unwrap_or_default(), + seq_no: seq_no__.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(), @@ -6280,14 +6281,14 @@ impl serde::Serialize for Persisted { { use serde::ser::SerializeStruct; let mut len = 0; - if self.nonce != 0 { + if self.seq_no != 0 { len += 1; } let mut struct_ser = serializer.serialize_struct("runtime.Persisted", len)?; - if self.nonce != 0 { + if self.seq_no != 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.serialize_field("seqNo", ToString::to_string(&self.seq_no).as_str())?; } struct_ser.end() } @@ -6299,12 +6300,13 @@ impl<'de> serde::Deserialize<'de> for Persisted { D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "nonce", + "seq_no", + "seqNo", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - Nonce, + SeqNo, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -6326,7 +6328,7 @@ impl<'de> serde::Deserialize<'de> for Persisted { E: serde::de::Error, { match value { - "nonce" => Ok(GeneratedField::Nonce), + "seqNo" | "seq_no" => Ok(GeneratedField::SeqNo), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -6346,21 +6348,21 @@ impl<'de> serde::Deserialize<'de> for Persisted { where V: serde::de::MapAccess<'de>, { - let mut nonce__ = None; + let mut seq_no__ = None; while let Some(k) = map_.next_key()? { match k { - GeneratedField::Nonce => { - if nonce__.is_some() { - return Err(serde::de::Error::duplicate_field("nonce")); + GeneratedField::SeqNo => { + if seq_no__.is_some() { + return Err(serde::de::Error::duplicate_field("seqNo")); } - nonce__ = + seq_no__ = Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } } Ok(Persisted { - nonce: nonce__.unwrap_or_default(), + seq_no: seq_no__.unwrap_or_default(), }) } } diff --git a/crates/runtime-next/src/leader/materialize/fsm.rs b/crates/runtime-next/src/leader/materialize/fsm.rs index fede0530fc0..04d9f59bb30 100644 --- a/crates/runtime-next/src/leader/materialize/fsm.rs +++ b/crates/runtime-next/src/leader/materialize/fsm.rs @@ -515,7 +515,7 @@ impl HeadFlush { // Persist extents for idempotent transaction replay. let persist = proto::Persist { - nonce: now.as_u64(), + seq_no: now.as_u64(), connector_patches_json: take_patches(&mut pending.persist_patches), delete_hinted_frontier: true, hinted_close_clock: extents.close.as_u64(), @@ -531,7 +531,7 @@ impl HeadFlush { shard_stored: vec![false; task.n_shards], }; let persist_state = HeadPersist { - nonce: persist.nonce, + seq_no: persist.seq_no, next_action: Action::Store, next_state: Box::new(Head::Store(store_state)), }; @@ -544,7 +544,7 @@ impl HeadFlush { /// and chains its contained action and state. #[derive(Debug)] pub struct HeadPersist { - pub nonce: u64, + pub seq_no: u64, pub next_action: Action, pub next_state: Box, } @@ -554,16 +554,16 @@ impl HeadPersist { if let Some(( 0, proto::Materialize { - persisted: Some(proto::Persisted { nonce }), + persisted: Some(proto::Persisted { seq_no }), .. }, )) = shard_rx - && *nonce == self.nonce + && *seq_no == self.seq_no { shard_rx.take(); let Self { - nonce: _, + seq_no: _, next_action, next_state, } = self; @@ -826,7 +826,7 @@ impl HeadStartCommit { .map(|(_full_frontier, full_checkpoint)| full_checkpoint.clone()); let persist = proto::Persist { - nonce: now.as_u64(), + seq_no: now.as_u64(), ack_intents: pending.ack_intents.clone(), committed_close_clock: close.as_u64(), committed_frontier: Some(shuffle::JournalFrontier::encode(&frontier.journals)), @@ -860,7 +860,7 @@ impl HeadStartCommit { }; let state = HeadPersist { - nonce: persist.nonce, + seq_no: persist.seq_no, next_action, next_state: Box::new(next_state), }; @@ -987,16 +987,16 @@ impl TailAcknowledge { // If Acknowledged returned patches, wrap with a Persist that runs first. if !persist_patches.is_empty() { - let nonce = now.as_u64(); + let seq_no = now.as_u64(); state = Tail::Persist(TailPersist { - nonce, + seq_no, next_action: action, next_state: Box::new(state), }); action = Action::Persist { persist: proto::Persist { - nonce, + seq_no, connector_patches_json: persist_patches, ..Default::default() }, @@ -1043,16 +1043,16 @@ impl TailTrigger { let Self { shard_patches } = self; - let nonce = now.as_u64(); + let seq_no = now.as_u64(); let action = Action::Persist { persist: proto::Persist { - nonce, + seq_no, delete_trigger_params: true, ..Default::default() }, }; let state = TailPersist { - nonce, + seq_no, next_action: Action::Idle, next_state: Box::new(Tail::Done(TailDone { shard_patches })), }; @@ -1065,7 +1065,7 @@ impl TailTrigger { /// and chains its contained action and state. #[derive(Debug)] pub struct TailPersist { - pub nonce: u64, + pub seq_no: u64, pub next_action: Action, pub next_state: Box, } @@ -1075,16 +1075,16 @@ impl TailPersist { if let Some(( 0, proto::Materialize { - persisted: Some(proto::Persisted { nonce }), + persisted: Some(proto::Persisted { seq_no }), .. }, )) = shard_rx - && *nonce == self.nonce + && *seq_no == self.seq_no { shard_rx.take(); let Self { - nonce: _, + seq_no: _, next_action, next_state, } = self; @@ -1393,28 +1393,28 @@ mod tests { } fn mk_head_persisted(head: &Head) -> (usize, proto::Materialize) { - let nonce = match head { - Head::Persist(p) => p.nonce, + let seq_no = match head { + Head::Persist(p) => p.seq_no, other => panic!("expected Head::Persist, got {other:?}"), }; ( 0, proto::Materialize { - persisted: Some(proto::Persisted { nonce }), + persisted: Some(proto::Persisted { seq_no }), ..Default::default() }, ) } fn mk_tail_persisted(tail: &Tail) -> (usize, proto::Materialize) { - let nonce = match tail { - Tail::Persist(p) => p.nonce, + let seq_no = match tail { + Tail::Persist(p) => p.seq_no, other => panic!("expected Tail::Persist, got {other:?}"), }; ( 0, proto::Materialize { - persisted: Some(proto::Persisted { nonce }), + persisted: Some(proto::Persisted { seq_no }), ..Default::default() }, ) @@ -1952,12 +1952,12 @@ mod tests { 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 + // `expected_seq_no` is plumbed through so Persisted occasionally matches + // the in-progress seq_no and lets HeadPersist / TailPersist actually // chain forward — without it, fuzz traces would rarely leave Persist. fn random_message( shard: usize, - expected_nonce: u64, + expected_seq_no: u64, rng: &mut SmallRng, ) -> (usize, proto::Materialize) { let mut msg = proto::Materialize::default(); @@ -2007,28 +2007,28 @@ mod tests { }); } _ => { - // Most of the time, target the in-progress Persist's nonce so + // Most of the time, target the in-progress Persist's seq_no so // the FSM can actually chain forward; otherwise emit garbage. - let nonce = if rng.random_bool(0.9) { - expected_nonce + let seq_no = if rng.random_bool(0.9) { + expected_seq_no } else { rng.random() }; - msg.persisted = Some(proto::Persisted { nonce }); + msg.persisted = Some(proto::Persisted { seq_no }); } } (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 + // Pick a "best-guess" seq_no to hand to `random_message`. When Head or + // Tail is awaiting Persisted we surface its seq_no so the message is // sometimes accepted; otherwise return random noise. - fn pick_nonce(head: &Head, tail: &Tail, rng: &mut SmallRng) -> u64 { + fn pick_seq_no(head: &Head, tail: &Tail, rng: &mut SmallRng) -> u64 { if let Head::Persist(p) = head { - return p.nonce; + return p.seq_no; } if let Tail::Persist(p) = tail { - return p.nonce; + return p.seq_no; } rng.random() } @@ -2075,8 +2075,8 @@ mod tests { // (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)); + let seq_no = pick_seq_no(head, tail, rng); + ctx.shard_rx = Some(random_message(shard, seq_no, rng)); } // Add an ACK intent occasionally; HeadWriteStats drains them. diff --git a/crates/runtime-next/src/leader/materialize/startup.rs b/crates/runtime-next/src/leader/materialize/startup.rs index 90760f81cca..6d99328088c 100644 --- a/crates/runtime-next/src/leader/materialize/startup.rs +++ b/crates/runtime-next/src/leader/materialize/startup.rs @@ -401,7 +401,7 @@ async fn apply_loop( let _ = tx.send(Ok(proto::Materialize { persist: Some(proto::Persist { - nonce: iteration, + seq_no: iteration, last_applied: next_applied.clone(), ..Default::default() }), @@ -410,9 +410,9 @@ async fn apply_loop( match verify_persisted.not_eof(rx.next().await)? { proto::Materialize { - persisted: Some(proto::Persisted { nonce }), + persisted: Some(proto::Persisted { seq_no }), .. - } if nonce == iteration => {} + } if seq_no == iteration => {} other => return Err(verify_persisted.fail_msg(other)), } @@ -427,7 +427,7 @@ async fn apply_loop( // Persist the iteration's patches to shard zero. let _ = tx.send(Ok(proto::Materialize { persist: Some(proto::Persist { - nonce: iteration, // End-of-sequence. + seq_no: iteration, // End-of-sequence. connector_patches_json: applied_patches_json, ..Default::default() }), @@ -437,9 +437,9 @@ async fn apply_loop( // Receive Persisted. match verify_persisted.not_eof(rx.next().await)? { proto::Materialize { - persisted: Some(proto::Persisted { nonce }), + persisted: Some(proto::Persisted { seq_no }), .. - } if nonce == iteration => {} + } if seq_no == iteration => {} other => return Err(verify_persisted.fail_msg(other)), } } @@ -516,9 +516,9 @@ mod tests { } } - fn persisted(nonce: u64) -> proto::Materialize { + fn persisted(seq_no: u64) -> proto::Materialize { proto::Materialize { - persisted: Some(proto::Persisted { nonce }), + persisted: Some(proto::Persisted { seq_no }), ..Default::default() } } @@ -550,7 +550,7 @@ mod tests { #[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. + // marking next_applied as the new last_applied with matching seq_no. 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(); @@ -570,7 +570,7 @@ mod tests { let m2 = leader_rx.try_recv().unwrap().unwrap(); let p = m2.persist.unwrap(); - assert_eq!(p.nonce, 1); + assert_eq!(p.seq_no, 1); assert_eq!(p.last_applied, next); assert!(p.connector_patches_json.is_empty()); @@ -610,7 +610,7 @@ mod tests { ); // 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_eq!(p1.seq_no, 1); assert!(p1.last_applied.is_empty()); assert_eq!(p1.connector_patches_json.as_ref(), patch1); @@ -622,7 +622,7 @@ mod tests { 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_eq!(p2.seq_no, 2); assert!(p2.last_applied.is_empty()); assert_eq!(p2.connector_patches_json.as_ref(), patch2); @@ -635,7 +635,7 @@ mod tests { ); // 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.seq_no, 3); assert_eq!(p3.last_applied, next); assert!(p3.connector_patches_json.is_empty()); @@ -660,16 +660,16 @@ mod tests { // Connector returns patches forever; we cap at MAX_APPLY_ITERATIONS. name: "no_convergence", seed: |tx| { - for nonce in 1..=4 { + for seq_no in 1..=4 { tx.send(Ok(applied(b"[{\"x\":1}\n]"))).unwrap(); - tx.send(Ok(persisted(nonce))).unwrap(); + tx.send(Ok(persisted(seq_no))).unwrap(); } }, expect: "did not converge", }, Case { - // Peer returns Persisted with a wrong nonce — protocol error. - name: "persisted_nonce_mismatch", + // Peer returns Persisted with a wrong seq_no — protocol error. + name: "persisted_seq_no_mismatch", seed: |tx| { tx.send(Ok(applied(b"[{\"x\":1}\n]"))).unwrap(); tx.send(Ok(persisted(99))).unwrap(); diff --git a/crates/runtime-next/src/shard/materialize/actor.rs b/crates/runtime-next/src/shard/materialize/actor.rs index 8ef18bd9c60..38bacea875c 100644 --- a/crates/runtime-next/src/shard/materialize/actor.rs +++ b/crates/runtime-next/src/shard/materialize/actor.rs @@ -348,7 +348,7 @@ impl Actor { ..Default::default() }); } else if let Some(persist) = msg.persist { - let nonce = persist.nonce; + let seq_no = persist.seq_no; let (db, binding_state_keys) = self .db @@ -358,7 +358,7 @@ impl Actor { self.db_persist_fut = Some( async move { let db = db.persist(&persist, &binding_state_keys).await?; - Ok(((db, binding_state_keys), proto::Persisted { nonce })) + Ok(((db, binding_state_keys), proto::Persisted { seq_no })) } .boxed(), ); @@ -755,11 +755,11 @@ mod tests { let resp = actor_to_leader_rx.recv().await.unwrap(); assert!(resp.started_commit.is_some()); - // 5) L:Persist → RocksDB write → L:Persisted echoes nonce. + // 5) L:Persist → RocksDB write → L:Persisted echoes seq_no. leader_to_actor_tx .send(Ok(proto::Materialize { persist: Some(proto::Persist { - nonce: 42, + seq_no: 42, last_applied: Bytes::from_static(b"persisted-spec-bytes"), ..Default::default() }), @@ -768,7 +768,7 @@ mod tests { .unwrap(); let resp = actor_to_leader_rx.recv().await.unwrap(); - assert_eq!(resp.persisted.unwrap().nonce, 42); + assert_eq!(resp.persisted.unwrap().seq_no, 42); // 6) Controller Stop + CloseNow → forwarded to the leader. controller_to_actor_tx diff --git a/crates/runtime-next/src/shard/materialize/startup.rs b/crates/runtime-next/src/shard/materialize/startup.rs index efbca991ecb..cc7b3b23568 100644 --- a/crates/runtime-next/src/shard/materialize/startup.rs +++ b/crates/runtime-next/src/shard/materialize/startup.rs @@ -219,7 +219,7 @@ where _ = leader_tx.send(proto::Materialize { persisted: Some(proto::Persisted { - nonce: persist.nonce, + seq_no: persist.seq_no, }), ..Default::default() }); diff --git a/crates/runtime-next/src/shard/rocksdb.rs b/crates/runtime-next/src/shard/rocksdb.rs index 1a7d166c5f4..b4502c64117 100644 --- a/crates/runtime-next/src/shard/rocksdb.rs +++ b/crates/runtime-next/src/shard/rocksdb.rs @@ -828,21 +828,21 @@ mod test { for persist in [ crate::proto::Persist { - nonce: 1, + seq_no: 1, ack_intents: [("j/A".to_string(), bytes::Bytes::from_static(b"INTENT-A"))] .into_iter() .collect(), ..Default::default() }, crate::proto::Persist { - nonce: 2, + seq_no: 2, ack_intents: [("j/B".to_string(), bytes::Bytes::from_static(b"INTENT-B"))] .into_iter() .collect(), ..Default::default() }, crate::proto::Persist { - nonce: 99, + seq_no: 99, last_applied: bytes::Bytes::from_static(b"v9"), ..Default::default() }, diff --git a/go/protocols/runtime/runtime.pb.go b/go/protocols/runtime/runtime.pb.go index 4db9a9027ea..9271a001780 100644 --- a/go/protocols/runtime/runtime.pb.go +++ b/go/protocols/runtime/runtime.pb.go @@ -1654,13 +1654,13 @@ var xxx_messageInfo_Recover proto.InternalMessageInfo // 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 +// `seq_no` 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"` + SeqNo uint64 `protobuf:"varint,1,opt,name=seq_no,json=seqNo,proto3" json:"seq_no,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"` @@ -1743,8 +1743,8 @@ 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"` + // Echoed back from the originating `Persist.seq_no` request. + SeqNo uint64 `protobuf:"varint,1,opt,name=seq_no,json=seqNo,proto3" json:"seq_no,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2920,244 +2920,244 @@ func init() { } var fileDescriptor_73af6e0737ce390c = []byte{ - // 3784 bytes of a gzipped FileDescriptorProto + // 3790 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, + 0x76, 0x9e, 0xff, 0xf4, 0x9b, 0xe1, 0x70, 0x58, 0xa2, 0xe4, 0xf6, 0xd8, 0x16, 0xe9, 0xb1, 0x1d, + 0x73, 0x25, 0x6a, 0xc8, 0xa5, 0xbd, 0xbb, 0xb6, 0xb0, 0x5a, 0x8b, 0x1c, 0x92, 0x36, 0xb5, 0x94, + 0xc4, 0x14, 0x29, 0x21, 0xc9, 0xa5, 0xd1, 0xec, 0x2a, 0x0e, 0x5b, 0xec, 0xe9, 0x6a, 0x77, 0xf5, + 0x90, 0x9a, 0xbd, 0xe6, 0x12, 0x20, 0x87, 0x5c, 0x72, 0xcb, 0x25, 0x39, 0x05, 0x08, 0x92, 0x43, + 0xae, 0x7b, 0xc8, 0x29, 0x40, 0x7c, 0x0c, 0x82, 0x20, 0xc8, 0x49, 0x40, 0x36, 0xd7, 0xdc, 0x36, + 0x01, 0x12, 0x21, 0x87, 0x45, 0x7d, 0xfa, 0x33, 0x3f, 0x5a, 0xa2, 0x7c, 0xf0, 0x41, 0x62, 0xd7, + 0xfb, 0xf5, 0xab, 0x57, 0xef, 0x57, 0xaf, 0x07, 0xda, 0x3d, 0xb6, 0x16, 0x84, 0x2c, 0x62, 0x0e, + 0xf3, 0xf8, 0x5a, 0x38, 0xf0, 0x23, 0xb7, 0x4f, 0xe3, 0xbf, 0x1d, 0x89, 0x41, 0x15, 0xbd, 0x6c, + 0xdd, 0x3c, 0x0e, 0xd9, 0x19, 0x0d, 0x13, 0x86, 0xe4, 0x41, 0x11, 0xb6, 0x96, 0x1d, 0xe6, 0xf3, + 0x41, 0xff, 0x32, 0x8a, 0x91, 0xd7, 0x11, 0x1a, 0xba, 0xe7, 0x54, 0xff, 0xd1, 0x14, 0xef, 0x8d, + 0x50, 0x9c, 0x78, 0xec, 0x42, 0xfe, 0xa7, 0xb1, 0xb7, 0x46, 0xb0, 0x7d, 0x3b, 0xa2, 0xa1, 0x6b, + 0x7b, 0xee, 0xaf, 0x68, 0xf6, 0x59, 0xd3, 0xb6, 0x46, 0x68, 0x59, 0x20, 0xff, 0x69, 0xdc, 0xe8, + 0xb6, 0xf9, 0xe9, 0xe0, 0xe4, 0xc4, 0xa3, 0xf1, 0x5f, 0x4d, 0xb3, 0xd8, 0x63, 0x3d, 0x26, 0x1f, + 0xd7, 0xc4, 0x93, 0x82, 0xb6, 0xff, 0x21, 0x07, 0x0b, 0x47, 0x36, 0x3f, 0x3b, 0xa4, 0xe1, 0xb9, + 0xeb, 0xd0, 0x2e, 0xf3, 0x4f, 0xdc, 0x1e, 0xba, 0x09, 0x35, 0x8f, 0xf5, 0xac, 0x13, 0xd7, 0xa3, + 0xd6, 0x09, 0x31, 0x73, 0xcb, 0xb9, 0x95, 0x12, 0x36, 0x3c, 0xd6, 0xdb, 0x75, 0x3d, 0xba, 0x4b, + 0xd0, 0xbb, 0x60, 0x44, 0x36, 0x3f, 0xb3, 0x7c, 0xbb, 0x4f, 0xcd, 0xfc, 0x72, 0x6e, 0xc5, 0xc0, + 0x55, 0x01, 0x78, 0x64, 0xf7, 0x29, 0x7a, 0x07, 0xaa, 0x03, 0xc2, 0xad, 0xc0, 0x8e, 0x4e, 0xcd, + 0x82, 0xc4, 0x55, 0x06, 0x84, 0x1f, 0xd8, 0xd1, 0x29, 0xba, 0x0d, 0x0b, 0x0e, 0xf3, 0x23, 0xdb, + 0xf5, 0x69, 0x68, 0xf9, 0x34, 0xba, 0x60, 0xe1, 0x99, 0x59, 0x94, 0x34, 0xcd, 0x04, 0xf1, 0x48, + 0xc1, 0xd1, 0x47, 0x50, 0x0a, 0x3c, 0xdb, 0xa7, 0x66, 0x79, 0x39, 0xb7, 0xd2, 0xd8, 0x68, 0x74, + 0xe2, 0x63, 0x3c, 0x10, 0x50, 0xac, 0x90, 0xed, 0xff, 0x2b, 0x42, 0xe3, 0x50, 0x6d, 0x14, 0xd3, + 0x6f, 0x06, 0x94, 0x47, 0x68, 0x0f, 0x2a, 0xcf, 0xd8, 0x20, 0xf4, 0x6d, 0x4f, 0x6a, 0x6e, 0x6c, + 0xad, 0xbd, 0x7c, 0xb1, 0x74, 0xbb, 0xc7, 0x3a, 0x3d, 0xfb, 0x57, 0x34, 0x8a, 0x68, 0x87, 0xd0, + 0xf3, 0x35, 0x87, 0x85, 0x74, 0x6d, 0xcc, 0x01, 0x3a, 0x0f, 0x14, 0x1b, 0x8e, 0xf9, 0xd1, 0x0d, + 0x28, 0x87, 0x34, 0xf0, 0xec, 0xa1, 0xdc, 0x65, 0x15, 0xeb, 0x95, 0xd8, 0xe3, 0xf1, 0xc0, 0xf5, + 0x88, 0xe5, 0x92, 0x78, 0x8f, 0x72, 0xbd, 0x47, 0xd0, 0x2e, 0x94, 0xd9, 0xc9, 0x09, 0xa7, 0x91, + 0xdc, 0x58, 0x61, 0xab, 0xf3, 0xf2, 0xc5, 0xd2, 0xad, 0x57, 0x79, 0xf9, 0x63, 0xc9, 0x85, 0x35, + 0x37, 0x7a, 0x08, 0x40, 0x7d, 0x62, 0x69, 0x59, 0xa5, 0x2b, 0xc9, 0x32, 0xa8, 0x4f, 0xd4, 0x23, + 0xba, 0x0d, 0xa5, 0xd0, 0xf6, 0x7b, 0xca, 0x9a, 0xb5, 0x8d, 0xf9, 0x8e, 0x74, 0x43, 0x2c, 0x40, + 0x87, 0x01, 0x75, 0xb6, 0x8a, 0xdf, 0xbe, 0x58, 0x7a, 0x0b, 0x2b, 0x1a, 0x74, 0x08, 0x35, 0x87, + 0xb1, 0x90, 0xb8, 0xbe, 0x1d, 0xb1, 0xd0, 0xac, 0x48, 0x2b, 0xfe, 0xf8, 0xe5, 0x8b, 0xa5, 0x3b, + 0xd3, 0x5e, 0x3e, 0x11, 0x26, 0x9d, 0xc3, 0x53, 0x3b, 0x24, 0x7b, 0xdb, 0x38, 0x2b, 0x05, 0xad, + 0x03, 0x84, 0x94, 0x33, 0x6f, 0x10, 0xb9, 0xcc, 0x37, 0xab, 0x52, 0x8d, 0x66, 0x27, 0xe1, 0xf9, + 0x9a, 0xda, 0x84, 0x86, 0x38, 0x43, 0x83, 0x3e, 0x84, 0x39, 0xed, 0xc3, 0x96, 0xeb, 0x13, 0xfa, + 0xdc, 0x34, 0x96, 0x73, 0x2b, 0x73, 0xb8, 0xae, 0x81, 0x7b, 0x02, 0x86, 0x3e, 0x03, 0x90, 0x11, + 0x67, 0x4b, 0xb1, 0x20, 0xc5, 0x2e, 0xaa, 0xdd, 0x75, 0x99, 0xe7, 0x51, 0x47, 0xc0, 0xc5, 0x16, + 0x71, 0x86, 0x0e, 0x75, 0x61, 0x3e, 0x0d, 0x31, 0xc5, 0x5a, 0x93, 0xac, 0xef, 0x28, 0xd6, 0x87, + 0xa3, 0x48, 0xc9, 0x3f, 0xce, 0xd1, 0xfe, 0x97, 0x22, 0xcc, 0x27, 0xbe, 0xc7, 0x03, 0xe6, 0x73, + 0x8a, 0x56, 0xa0, 0xcc, 0x23, 0x3b, 0x1a, 0x70, 0xe9, 0x7b, 0x8d, 0x8d, 0x66, 0x27, 0x36, 0x4f, + 0xe7, 0x50, 0xc2, 0xb1, 0xc6, 0x0b, 0xca, 0x53, 0xb9, 0x67, 0xe9, 0x5b, 0xd3, 0x6c, 0xa1, 0xf1, + 0xe8, 0x63, 0x68, 0x44, 0x34, 0xec, 0xbb, 0xbe, 0xed, 0x59, 0x34, 0x0c, 0x59, 0xa8, 0x7d, 0x6e, + 0x2e, 0x86, 0xee, 0x08, 0x20, 0xfa, 0x7d, 0xa8, 0x87, 0xd4, 0x26, 0x56, 0x74, 0x1a, 0xb2, 0x41, + 0xef, 0xf4, 0x8a, 0xfe, 0x57, 0x13, 0x32, 0x8e, 0x94, 0x08, 0xe1, 0x84, 0x17, 0xa1, 0x1b, 0x51, + 0x4b, 0x68, 0x72, 0x55, 0x27, 0x94, 0x12, 0xc4, 0x96, 0xd0, 0x1e, 0x94, 0xec, 0x90, 0xfa, 0xb6, + 0x74, 0xc2, 0xfa, 0xd6, 0xa7, 0x2f, 0x5f, 0x2c, 0xad, 0xf5, 0xdc, 0xe8, 0x74, 0x70, 0xdc, 0x71, + 0x58, 0x7f, 0x8d, 0xf2, 0x68, 0x60, 0x87, 0x43, 0x95, 0x26, 0x27, 0x12, 0x67, 0x67, 0x53, 0xb0, + 0x62, 0x25, 0x01, 0x7d, 0x0c, 0x45, 0xc2, 0x1c, 0x6e, 0x56, 0x96, 0x0b, 0x2b, 0xb5, 0x8d, 0x9a, + 0x3a, 0xb5, 0x43, 0xcf, 0x75, 0xa8, 0x76, 0x65, 0x89, 0x46, 0x5f, 0x43, 0x45, 0x45, 0x10, 0x37, + 0xab, 0xcb, 0x85, 0x2b, 0x68, 0x1f, 0xb3, 0x0b, 0x3f, 0x1b, 0x0c, 0x5c, 0x62, 0x05, 0x76, 0x18, + 0x71, 0xd3, 0x90, 0xaf, 0xd5, 0x51, 0xf4, 0xe4, 0xc9, 0xde, 0xf6, 0x81, 0x00, 0xeb, 0x57, 0x1b, + 0x82, 0x50, 0x02, 0x84, 0xd3, 0x07, 0xb6, 0x73, 0x46, 0x89, 0x75, 0x46, 0x87, 0x26, 0xcc, 0x52, + 0xd6, 0x50, 0x44, 0xbf, 0xa4, 0xc3, 0x36, 0x81, 0x05, 0xcc, 0x9c, 0x33, 0xbe, 0xbd, 0xb5, 0x4d, + 0xb9, 0x13, 0xba, 0x81, 0x88, 0x9d, 0x55, 0x40, 0xa1, 0x00, 0x92, 0x63, 0x8b, 0xfa, 0xe7, 0x56, + 0x9f, 0xf6, 0x83, 0x28, 0x94, 0x1e, 0x56, 0xc6, 0x4d, 0x8d, 0xd9, 0xf1, 0xcf, 0x1f, 0x4a, 0x38, + 0xfa, 0x00, 0xea, 0x31, 0xb5, 0xcc, 0xc2, 0x2a, 0x43, 0xd7, 0x34, 0x4c, 0x64, 0xe2, 0xf6, 0x9f, + 0xe7, 0xc1, 0xe8, 0xc6, 0x19, 0x17, 0xbd, 0x0d, 0x15, 0x37, 0xb0, 0x6c, 0x42, 0x94, 0x4c, 0x03, + 0x97, 0xdd, 0x60, 0x93, 0x90, 0x10, 0xfd, 0x14, 0xe6, 0x74, 0x9a, 0xb6, 0x02, 0x26, 0xf6, 0x9d, + 0x97, 0x3b, 0x58, 0x50, 0x3b, 0xd0, 0x99, 0xfa, 0x80, 0x85, 0x11, 0xae, 0xfb, 0xe9, 0x82, 0xa3, + 0x43, 0x58, 0xe8, 0xdb, 0x41, 0x40, 0x89, 0x75, 0xca, 0x78, 0xa4, 0x79, 0x0b, 0x92, 0xf7, 0x93, + 0x24, 0x8f, 0x27, 0xef, 0xef, 0x3c, 0x94, 0xb4, 0x5f, 0x33, 0x1e, 0x49, 0xf6, 0x1d, 0x3f, 0x0a, + 0x87, 0x22, 0xdc, 0x46, 0xa0, 0xe8, 0x7d, 0x80, 0x01, 0xb7, 0x7b, 0xd4, 0x0a, 0xed, 0x88, 0x4a, + 0xef, 0xce, 0x63, 0x43, 0x42, 0xb0, 0x1d, 0xd1, 0xd6, 0x16, 0x2c, 0x4e, 0x93, 0x83, 0x9a, 0x50, + 0x10, 0xb6, 0xcf, 0xc9, 0xdc, 0x21, 0x1e, 0xd1, 0x22, 0x94, 0xce, 0x6d, 0x6f, 0x10, 0x97, 0x2e, + 0xb5, 0xb8, 0x9b, 0xff, 0x3c, 0xd7, 0xfe, 0x9b, 0x3c, 0x2c, 0x74, 0xed, 0x20, 0x1a, 0x84, 0x71, + 0x35, 0xd9, 0x79, 0x2e, 0x72, 0xa7, 0xa8, 0x7d, 0x96, 0x47, 0xcf, 0xa9, 0xa7, 0xc3, 0xba, 0xd1, + 0x11, 0xd5, 0x77, 0x9f, 0xf5, 0x3a, 0xfb, 0x02, 0x8a, 0xab, 0x1e, 0xeb, 0xc9, 0x27, 0xb4, 0x97, + 0x1e, 0x15, 0x49, 0x0e, 0x50, 0x87, 0x78, 0x2b, 0xd9, 0xfb, 0xc4, 0x11, 0xe3, 0x05, 0xcd, 0x95, + 0x39, 0xf5, 0x3d, 0xa8, 0xf3, 0xc8, 0x0e, 0x23, 0xcb, 0x61, 0xfd, 0xbe, 0x1b, 0xc9, 0xa8, 0xaf, + 0x6d, 0xfc, 0x5e, 0x6a, 0xc0, 0x71, 0x4d, 0x45, 0x8a, 0x09, 0xa3, 0xae, 0xa4, 0xc6, 0x35, 0x9e, + 0x2e, 0x5a, 0x18, 0x6a, 0x19, 0x1c, 0xea, 0x02, 0xd2, 0x42, 0x2c, 0xe7, 0x94, 0x3a, 0x67, 0x01, + 0x73, 0xfd, 0x48, 0x6e, 0x4d, 0x24, 0xcf, 0x24, 0x63, 0x75, 0x13, 0x1c, 0x5e, 0xd0, 0xf4, 0x29, + 0xa8, 0xfd, 0xff, 0x45, 0x40, 0x89, 0x0a, 0x2a, 0xfd, 0x09, 0x6b, 0xad, 0x83, 0x91, 0xd4, 0x72, + 0x2d, 0x12, 0x4d, 0x9e, 0x39, 0x4e, 0x89, 0xd0, 0x5d, 0x28, 0xb3, 0x80, 0xfa, 0x94, 0x68, 0x33, + 0xb5, 0x27, 0x77, 0x98, 0x88, 0xef, 0x3c, 0x96, 0x94, 0x58, 0x73, 0xa0, 0xfb, 0x50, 0x75, 0x14, + 0x11, 0xd1, 0xf6, 0xf9, 0xe8, 0x32, 0x6e, 0x0d, 0x22, 0x38, 0xe1, 0x42, 0xbb, 0x00, 0x19, 0x1b, + 0x14, 0x67, 0xd9, 0x38, 0x23, 0x23, 0xb5, 0x4a, 0x86, 0xb3, 0xf5, 0x10, 0xca, 0x4a, 0xb7, 0xef, + 0xc5, 0xba, 0xad, 0xa7, 0x50, 0x8d, 0x95, 0x15, 0x9e, 0x7f, 0x46, 0x87, 0x96, 0x4a, 0x12, 0x52, + 0x50, 0x1d, 0x1b, 0x67, 0x74, 0x78, 0x20, 0x01, 0xa2, 0xad, 0x12, 0x59, 0xc9, 0x15, 0x45, 0x89, + 0xc7, 0x54, 0x79, 0x49, 0xd5, 0x4c, 0x11, 0x8a, 0xb8, 0x75, 0x01, 0x90, 0xbe, 0x05, 0x2d, 0x43, + 0x49, 0x94, 0x23, 0xae, 0xb5, 0x03, 0xe9, 0xd6, 0xa2, 0x50, 0x71, 0xac, 0x10, 0xe8, 0x2b, 0xa8, + 0x05, 0xcc, 0xf3, 0xac, 0x90, 0xf2, 0x81, 0x17, 0x49, 0xb1, 0x8d, 0xcb, 0xed, 0x73, 0xc0, 0x3c, + 0x0f, 0x4b, 0x6a, 0x0c, 0x41, 0xf2, 0xdc, 0x7e, 0x04, 0x90, 0x62, 0x50, 0x0d, 0x2a, 0x7b, 0x8f, + 0x9e, 0x6e, 0xee, 0xef, 0x6d, 0x37, 0xdf, 0x42, 0x06, 0x94, 0xf0, 0xce, 0xe6, 0xf6, 0x1f, 0x36, + 0x73, 0x68, 0x0e, 0x8c, 0x47, 0x8f, 0x8f, 0x2c, 0xb5, 0xcc, 0xa3, 0x3a, 0x54, 0xbb, 0x8f, 0x1f, + 0xef, 0x5b, 0x8f, 0x77, 0x77, 0x9b, 0x05, 0xc1, 0x84, 0x77, 0x0e, 0x8f, 0x36, 0xf1, 0x51, 0xb3, + 0xd8, 0xfe, 0xaf, 0x1c, 0x34, 0xb7, 0x65, 0xaf, 0xfd, 0x03, 0x08, 0xd5, 0x0d, 0x28, 0x0a, 0x87, + 0xd4, 0x2e, 0x78, 0x33, 0x61, 0x1e, 0x57, 0x50, 0xba, 0x2f, 0x96, 0xb4, 0xad, 0x55, 0x28, 0x8a, + 0x15, 0xfa, 0x08, 0x1a, 0xfc, 0x1b, 0x4f, 0x54, 0xd9, 0xf3, 0x13, 0x6e, 0x0d, 0x42, 0x57, 0x27, + 0xe1, 0xba, 0x82, 0x3e, 0x3d, 0xe1, 0x4f, 0x42, 0xb7, 0xfd, 0xdf, 0x05, 0x58, 0x88, 0xa5, 0xbd, + 0x49, 0xb0, 0x7d, 0x31, 0x16, 0x6c, 0x1f, 0x4c, 0xe8, 0x3a, 0x33, 0xd6, 0xb6, 0xc0, 0x08, 0x06, + 0xc7, 0x9e, 0xcb, 0x4f, 0xa7, 0x04, 0xdb, 0x24, 0xf7, 0x41, 0x4c, 0x8b, 0x53, 0x36, 0xf4, 0x73, + 0xa8, 0x9c, 0x78, 0x03, 0x29, 0xa1, 0x38, 0x16, 0xec, 0x93, 0x12, 0x76, 0x15, 0x25, 0x8e, 0x59, + 0xbe, 0xef, 0x18, 0x8b, 0xc0, 0x48, 0x94, 0x14, 0x97, 0x9a, 0xbe, 0xfd, 0xdc, 0x72, 0x3c, 0xe6, + 0x9c, 0xe9, 0xd2, 0x5a, 0xed, 0xdb, 0xcf, 0xbb, 0x62, 0x3d, 0x16, 0x81, 0xf9, 0x57, 0x8a, 0xc0, + 0xc2, 0x8c, 0x08, 0xbc, 0x0d, 0x15, 0xbd, 0xb1, 0xef, 0x0e, 0xbf, 0xf6, 0x9f, 0xe5, 0xe0, 0x7a, + 0xda, 0x8c, 0xfe, 0x00, 0x5c, 0xbd, 0xfd, 0xeb, 0x1c, 0xdc, 0x18, 0xd1, 0xe8, 0x4d, 0xbc, 0x71, + 0x33, 0x75, 0x07, 0xa5, 0x4c, 0xda, 0x1e, 0x4c, 0x7f, 0xc7, 0xa4, 0x4f, 0xbc, 0x96, 0x39, 0x7f, + 0x5d, 0x84, 0x46, 0x97, 0xf5, 0x8f, 0x5d, 0x3f, 0xb9, 0x2e, 0xae, 0xeb, 0xd0, 0x55, 0x3c, 0xef, + 0x65, 0xf4, 0xcd, 0x92, 0x65, 0x02, 0x17, 0xdd, 0x81, 0x82, 0x4d, 0x62, 0x85, 0xdf, 0x9d, 0xc5, + 0xb0, 0x49, 0x08, 0x16, 0x74, 0xad, 0x7f, 0xcb, 0xeb, 0x40, 0xbf, 0x0f, 0xd5, 0x63, 0xd7, 0x27, + 0xae, 0xdf, 0x13, 0x1a, 0x16, 0x46, 0x6b, 0xd5, 0xe4, 0xdb, 0x3a, 0x5b, 0x8a, 0x18, 0x27, 0x5c, + 0xad, 0x3f, 0xcd, 0x43, 0x45, 0x43, 0x11, 0x82, 0xe2, 0xc9, 0xc0, 0x53, 0x47, 0x5f, 0xc5, 0xf2, + 0x39, 0xee, 0x75, 0x44, 0x97, 0x66, 0xa8, 0x5e, 0xe7, 0x73, 0xa8, 0x05, 0x21, 0x7b, 0xa6, 0xae, + 0x41, 0x71, 0x0f, 0xd6, 0x54, 0xfd, 0xdb, 0x41, 0x82, 0xd0, 0x6d, 0x68, 0x96, 0x14, 0xdd, 0x83, + 0x1a, 0x77, 0x4e, 0x69, 0xdf, 0xb6, 0x9e, 0x71, 0xe6, 0xcb, 0x68, 0xad, 0x6f, 0xbd, 0xf7, 0xf2, + 0xc5, 0x92, 0x49, 0x7d, 0x87, 0x09, 0x15, 0xd6, 0x04, 0xa2, 0x83, 0xed, 0x8b, 0x87, 0x94, 0xcb, + 0x36, 0x0c, 0x14, 0xc3, 0x03, 0xce, 0x7c, 0xd4, 0x01, 0xe0, 0x34, 0xb4, 0x02, 0xe6, 0xb9, 0xce, + 0x50, 0x5e, 0x1d, 0x92, 0x7e, 0xf9, 0x90, 0x86, 0x07, 0x12, 0x8c, 0x0d, 0x1e, 0x3f, 0xca, 0xb1, + 0x81, 0xec, 0xaf, 0xa3, 0x50, 0x5e, 0x0f, 0x0c, 0x5c, 0x91, 0x6d, 0x74, 0x14, 0x8a, 0x5b, 0xb8, + 0x6c, 0xd1, 0x54, 0xb7, 0x6f, 0x60, 0xbd, 0x6a, 0xf9, 0x50, 0xd8, 0x24, 0x04, 0x99, 0x50, 0xd1, + 0x06, 0xd2, 0x4d, 0x5e, 0xbc, 0x44, 0x3f, 0x83, 0x2a, 0x61, 0x8e, 0xd2, 0x3f, 0xff, 0x0a, 0xfa, + 0x57, 0x08, 0x73, 0xa4, 0xf2, 0x8b, 0x50, 0x3a, 0x09, 0x99, 0xaf, 0x5a, 0xae, 0x2a, 0x56, 0x8b, + 0xf6, 0xbf, 0xe7, 0x60, 0x3e, 0x39, 0x27, 0x7d, 0xdf, 0x9b, 0xfd, 0x72, 0x13, 0x2a, 0x84, 0x7a, + 0x34, 0xd2, 0xae, 0x5d, 0xc5, 0xf1, 0x72, 0x44, 0xad, 0xc2, 0x95, 0xd4, 0x2a, 0x66, 0xd4, 0x1a, + 0xcb, 0x4d, 0xa5, 0xf1, 0xdc, 0xf4, 0x21, 0xcc, 0x29, 0x7b, 0xc5, 0x14, 0xf2, 0xf2, 0x85, 0xeb, + 0x0a, 0xa8, 0x88, 0xda, 0x6f, 0xc3, 0xf5, 0x2e, 0xf3, 0x7d, 0xea, 0x44, 0x2c, 0x3c, 0x08, 0xd9, + 0xf3, 0xa1, 0x76, 0xc4, 0xf6, 0x5f, 0xe4, 0xe0, 0xc6, 0x38, 0x46, 0x6f, 0xfd, 0x01, 0x54, 0xc4, + 0x95, 0x81, 0x72, 0xae, 0xe7, 0x2c, 0xeb, 0x2f, 0x5f, 0x2c, 0xad, 0xbe, 0xca, 0xdd, 0x6a, 0xc7, + 0x27, 0x2a, 0x27, 0xc7, 0x02, 0xc4, 0xe9, 0x07, 0x42, 0xb8, 0xe5, 0x12, 0xdd, 0x95, 0x57, 0xe4, + 0x7a, 0x8f, 0xa0, 0x16, 0x14, 0x3c, 0xd6, 0xd3, 0xf5, 0xa6, 0x1a, 0x67, 0x38, 0x2c, 0x80, 0xed, + 0xbf, 0x2f, 0x40, 0xf1, 0x01, 0x73, 0x7d, 0x74, 0x0b, 0x16, 0x68, 0xe4, 0x10, 0xab, 0xcf, 0x88, + 0x15, 0xd2, 0x73, 0x97, 0x8b, 0x1b, 0xbd, 0xd0, 0xaa, 0x80, 0xe7, 0x05, 0xe2, 0x21, 0x23, 0x58, + 0x83, 0xd1, 0x6d, 0x28, 0xf3, 0x53, 0x3b, 0x24, 0xf1, 0x6d, 0xe6, 0x5a, 0x12, 0x84, 0x42, 0x94, + 0x1a, 0x5e, 0x60, 0x4d, 0x82, 0x96, 0xa0, 0x26, 0x9f, 0xf4, 0x04, 0xa2, 0x20, 0xcf, 0x18, 0x24, + 0x48, 0xcd, 0x1f, 0x6e, 0xc3, 0x42, 0x3c, 0xa4, 0x20, 0x6e, 0x28, 0xcd, 0x34, 0x8c, 0x67, 0x5a, + 0x1a, 0xb1, 0x1d, 0xc3, 0xd1, 0x8f, 0x20, 0x86, 0x59, 0x54, 0xdb, 0x40, 0x1e, 0x98, 0x81, 0xe7, + 0x35, 0x3c, 0x36, 0x0d, 0xfa, 0x04, 0xe6, 0x3d, 0x79, 0xfd, 0x4f, 0x29, 0x55, 0x58, 0x34, 0x14, + 0x38, 0x26, 0x6c, 0xfd, 0x5d, 0x0e, 0x4a, 0x52, 0x67, 0xd4, 0x80, 0xbc, 0x4b, 0x74, 0xf3, 0x90, + 0x77, 0x09, 0xea, 0x40, 0xd5, 0xb3, 0x8f, 0xa9, 0x27, 0x9c, 0x33, 0xaf, 0xb3, 0xb1, 0xcc, 0x88, + 0x82, 0x7a, 0x5f, 0x63, 0x70, 0x42, 0x83, 0x36, 0xa0, 0x12, 0x52, 0x5b, 0x68, 0xaa, 0xad, 0x6d, + 0xa6, 0x23, 0x89, 0x83, 0x90, 0x39, 0x94, 0xf3, 0xc3, 0x80, 0x3a, 0x9d, 0xbd, 0x6d, 0x1c, 0x13, + 0xa2, 0x75, 0x58, 0x94, 0x86, 0x77, 0x42, 0x6a, 0x47, 0x34, 0xb5, 0xbd, 0x1c, 0x3e, 0x60, 0x24, + 0x70, 0x5d, 0x89, 0x8a, 0xcd, 0xdf, 0xfe, 0x0c, 0xca, 0xc2, 0xce, 0x94, 0x88, 0x43, 0x13, 0x15, + 0x57, 0xf2, 0x8f, 0x1f, 0x5a, 0xdf, 0x7e, 0xbe, 0x13, 0x39, 0xc9, 0xa1, 0xb5, 0xff, 0x29, 0x07, + 0xc5, 0x23, 0x9b, 0x9f, 0x89, 0xb4, 0xc7, 0x03, 0xea, 0xe8, 0x2e, 0x58, 0x3e, 0x0b, 0x41, 0x2c, + 0xe0, 0x96, 0x4c, 0xf1, 0x56, 0x3c, 0xfb, 0x53, 0x6e, 0x34, 0xcf, 0x02, 0x2e, 0x2b, 0x80, 0x9e, + 0xed, 0xa1, 0xbb, 0xd0, 0x48, 0x69, 0xa5, 0xa4, 0xc2, 0x25, 0x33, 0xa3, 0x7a, 0xcc, 0x2e, 0x56, + 0x22, 0xa4, 0x03, 0xa1, 0x28, 0xbd, 0xd0, 0x11, 0x18, 0x2f, 0xc5, 0xc1, 0x8a, 0xad, 0x44, 0xa1, + 0xed, 0x73, 0x5b, 0xe7, 0xda, 0x92, 0xf4, 0x15, 0xb1, 0x93, 0xa3, 0x0c, 0xb8, 0xfd, 0x57, 0x25, + 0xa8, 0x60, 0xea, 0xb0, 0x73, 0x59, 0xfe, 0x6a, 0xb6, 0x73, 0x66, 0xb9, 0x7e, 0x44, 0xfd, 0x28, + 0x2e, 0x0a, 0xcb, 0x69, 0x3d, 0x56, 0x64, 0x9d, 0x4d, 0xe7, 0x6c, 0x4f, 0x91, 0xa8, 0xab, 0x31, + 0xd8, 0x09, 0x00, 0x6d, 0xc0, 0x75, 0x75, 0x3d, 0x8c, 0x28, 0x11, 0xcd, 0x0b, 0xa7, 0xba, 0x85, + 0xc9, 0xcb, 0x16, 0xe6, 0x5a, 0x82, 0xec, 0x0a, 0x9c, 0xea, 0x66, 0xee, 0x03, 0x4a, 0x79, 0x64, + 0x12, 0x71, 0x69, 0x7c, 0xe6, 0x0b, 0x9d, 0x78, 0x6e, 0xbc, 0xab, 0x11, 0x78, 0x21, 0x21, 0x8e, + 0x41, 0x68, 0x15, 0x16, 0x9d, 0x38, 0x2b, 0x48, 0x5b, 0xd2, 0x4c, 0x95, 0xc0, 0x8d, 0x04, 0x27, + 0x6c, 0x47, 0xd1, 0x2a, 0xa0, 0x53, 0xb1, 0xc7, 0x51, 0x05, 0x4b, 0x6a, 0x7c, 0xa1, 0x30, 0x19, + 0xed, 0xee, 0xc2, 0xbc, 0xa6, 0x4e, 0x54, 0x2b, 0xcf, 0x52, 0xad, 0xa1, 0x28, 0x13, 0xbd, 0x3e, + 0x80, 0xba, 0x67, 0xf3, 0xc8, 0xb2, 0x83, 0xc0, 0x73, 0x29, 0x91, 0xa3, 0xcb, 0x3a, 0xae, 0x09, + 0xd8, 0xa6, 0x02, 0xa1, 0x4d, 0x58, 0xf0, 0x68, 0xcf, 0x76, 0x86, 0xd9, 0xc6, 0xb1, 0x7a, 0x49, + 0xe3, 0xd8, 0x54, 0xe4, 0x99, 0x5b, 0xd3, 0xe7, 0x20, 0x3a, 0x43, 0xeb, 0x8c, 0x0e, 0xe3, 0x49, + 0xd0, 0xfb, 0x13, 0x67, 0xf6, 0xd0, 0x7e, 0xfe, 0x4b, 0x3a, 0xd4, 0x07, 0x56, 0xe9, 0xab, 0x15, + 0xba, 0x05, 0xd7, 0xa2, 0xd0, 0xed, 0xf5, 0x44, 0x65, 0xb4, 0x43, 0xbb, 0xcf, 0x95, 0xd9, 0x40, + 0xaa, 0x39, 0xa7, 0x51, 0x07, 0x12, 0xd3, 0xba, 0x07, 0xf3, 0x63, 0x07, 0x9f, 0x9d, 0x65, 0x18, + 0x53, 0x66, 0x19, 0xf5, 0xcc, 0x2c, 0xa3, 0x75, 0x17, 0xea, 0x59, 0x1d, 0xbe, 0x6b, 0x0e, 0x92, + 0xe5, 0x6d, 0xff, 0x6b, 0x19, 0x2a, 0x07, 0x34, 0xe4, 0x2e, 0x8f, 0xd0, 0x75, 0x28, 0x73, 0xfa, + 0x8d, 0xe5, 0x33, 0xc9, 0x5a, 0xc4, 0x25, 0x4e, 0xbf, 0x79, 0xc4, 0xc4, 0x99, 0xaa, 0x7a, 0x66, + 0x65, 0x3d, 0x58, 0x55, 0xba, 0xa6, 0xc2, 0xa4, 0xda, 0x8f, 0x3b, 0x7a, 0x61, 0xcc, 0xd1, 0xf5, + 0xbb, 0xae, 0xe6, 0xe8, 0xc5, 0xd7, 0x75, 0xf4, 0xd2, 0x6b, 0x38, 0xfa, 0x3a, 0xdc, 0x48, 0x1d, + 0x3d, 0xb0, 0x23, 0xe7, 0x94, 0xea, 0x33, 0x53, 0x65, 0xb4, 0x99, 0x60, 0x0f, 0x14, 0x72, 0x86, + 0xb3, 0x57, 0x66, 0x38, 0xfb, 0x67, 0x70, 0x43, 0x9b, 0x71, 0xdc, 0xe7, 0xab, 0xd2, 0x94, 0x8b, + 0x0a, 0xfb, 0xf5, 0xa8, 0x9b, 0x4f, 0x09, 0x11, 0xe3, 0xaa, 0x21, 0x02, 0xaf, 0x18, 0x22, 0xb5, + 0x2b, 0x87, 0x48, 0x7d, 0x2c, 0x44, 0xe2, 0xd3, 0x9e, 0x1e, 0x22, 0x1b, 0x70, 0x5d, 0x5b, 0x64, + 0x34, 0x52, 0xcc, 0x39, 0x69, 0x90, 0x6b, 0x0a, 0x79, 0x94, 0x0d, 0x95, 0x59, 0x61, 0xd5, 0xf8, + 0x81, 0x85, 0x55, 0x1b, 0x0c, 0xbd, 0x77, 0x4a, 0x66, 0xc4, 0x55, 0xfb, 0xaf, 0x73, 0x50, 0x12, + 0xe7, 0x30, 0x9c, 0x5a, 0xe9, 0x4c, 0xa8, 0x9c, 0x0b, 0x09, 0xba, 0xa1, 0x35, 0x70, 0xbc, 0x14, + 0xd7, 0x57, 0x79, 0xac, 0x92, 0x45, 0xa5, 0xe1, 0xaa, 0x00, 0xc8, 0xc2, 0x15, 0x9f, 0x79, 0xcc, + 0xab, 0x7a, 0x0e, 0x79, 0xe6, 0x4f, 0x35, 0xff, 0xfa, 0x8c, 0x8c, 0xae, 0xdc, 0x1c, 0x8d, 0x66, + 0x74, 0xd1, 0x8d, 0xb6, 0x9f, 0x41, 0x25, 0x76, 0x98, 0x3b, 0x80, 0x54, 0x79, 0x4b, 0x6e, 0x97, + 0x71, 0x29, 0x37, 0xf0, 0x82, 0xc2, 0x6c, 0xa7, 0x88, 0x4b, 0x82, 0x2a, 0x3f, 0x3d, 0xa8, 0xda, + 0xbf, 0xcd, 0xe9, 0x3b, 0xd4, 0xeb, 0x19, 0xe5, 0xe3, 0xf8, 0xab, 0x57, 0x61, 0xea, 0x57, 0xaf, + 0xf8, 0x7b, 0xd7, 0x87, 0x97, 0x56, 0x33, 0x79, 0x75, 0xa4, 0xe8, 0x27, 0x19, 0x8f, 0x2e, 0x49, + 0x8f, 0x4e, 0x2f, 0xce, 0xf2, 0xba, 0x36, 0xd5, 0x9d, 0xdf, 0xc8, 0x5f, 0x00, 0xaa, 0x32, 0x55, + 0x3c, 0x62, 0x17, 0xed, 0x32, 0x14, 0x0f, 0x23, 0x16, 0xb4, 0x0d, 0xa8, 0x88, 0xbf, 0x01, 0x25, + 0xed, 0x3f, 0x80, 0xda, 0x21, 0xe5, 0x62, 0xa3, 0xfb, 0x8c, 0x05, 0x33, 0xee, 0xf8, 0xb9, 0xab, + 0xdc, 0xf1, 0xff, 0xf6, 0x06, 0xd4, 0x32, 0xf7, 0x6f, 0x74, 0x27, 0x63, 0xf5, 0xda, 0xc6, 0x3b, + 0x9d, 0xec, 0xe7, 0xe9, 0xf8, 0xca, 0x2a, 0x0d, 0xaa, 0x0e, 0xe4, 0x4b, 0x98, 0x13, 0x7f, 0xad, + 0x50, 0x5f, 0x15, 0x92, 0x41, 0xc3, 0x28, 0x9f, 0x42, 0x2a, 0xc6, 0xba, 0x60, 0x48, 0xae, 0x16, + 0x5f, 0x40, 0xf5, 0xdc, 0xf6, 0x5c, 0x62, 0x47, 0xf1, 0xd1, 0xbd, 0x3f, 0xf5, 0x9d, 0x4f, 0x35, + 0x11, 0x4e, 0xc8, 0xd1, 0x3d, 0x30, 0xe2, 0xe7, 0x78, 0xc4, 0xb4, 0x34, 0xfd, 0xbd, 0x31, 0x33, + 0xc1, 0x29, 0x07, 0xfa, 0x19, 0xd4, 0xb9, 0xb2, 0xa9, 0xe5, 0x31, 0x16, 0x98, 0x8b, 0x3a, 0xeb, + 0xc5, 0xe6, 0xcb, 0x18, 0x1c, 0xd7, 0x78, 0xc6, 0xfa, 0x1f, 0x40, 0xf1, 0x19, 0x73, 0x7d, 0xf3, + 0xba, 0x64, 0x98, 0x1b, 0xb9, 0x53, 0x60, 0x89, 0x42, 0x9f, 0x40, 0xf9, 0x99, 0xec, 0x7c, 0xcd, + 0x1b, 0xda, 0x1d, 0xb3, 0x44, 0x94, 0x60, 0x8d, 0x16, 0xb2, 0x22, 0x9b, 0x9f, 0x99, 0x6f, 0x8f, + 0xc9, 0x12, 0x0d, 0x30, 0x96, 0x28, 0x74, 0x4b, 0xf4, 0xea, 0xb2, 0xd3, 0x30, 0x4d, 0xfd, 0xf9, + 0x70, 0xac, 0x03, 0xc1, 0x31, 0x01, 0xfa, 0x08, 0x4a, 0x22, 0xd9, 0x0f, 0xcd, 0x77, 0x24, 0x65, + 0xfa, 0x25, 0x5d, 0xe6, 0x19, 0xac, 0x90, 0x42, 0x62, 0x5c, 0x12, 0x5a, 0x63, 0x12, 0x75, 0x98, + 0xe3, 0x98, 0x40, 0x28, 0x28, 0x67, 0x26, 0xef, 0x8e, 0x29, 0x98, 0x19, 0x92, 0x7c, 0x9a, 0xcc, + 0x19, 0xdf, 0x1b, 0x9b, 0x93, 0x64, 0x1c, 0x6b, 0x7c, 0xc2, 0x78, 0x07, 0x8a, 0x1e, 0xb3, 0x89, + 0xb9, 0xa2, 0xfd, 0x6c, 0x1a, 0xcb, 0x3e, 0xb3, 0x09, 0x96, 0x64, 0xe2, 0x1d, 0xe2, 0x2f, 0x25, + 0xe6, 0x8f, 0x2e, 0x79, 0xc7, 0xbe, 0x24, 0xc1, 0x9a, 0x14, 0xad, 0x43, 0x49, 0x8e, 0x8e, 0xcc, + 0x5b, 0x63, 0x91, 0x91, 0xe5, 0x91, 0x13, 0x25, 0xac, 0x08, 0xd1, 0x4f, 0xd3, 0x21, 0xd5, 0xed, + 0xb1, 0x21, 0xd1, 0x04, 0x4f, 0x66, 0x32, 0x25, 0xde, 0xc4, 0x23, 0x16, 0x52, 0x73, 0xf5, 0x92, + 0x37, 0x1d, 0x0a, 0x0a, 0xac, 0x08, 0xc5, 0x86, 0xe4, 0x03, 0x31, 0xef, 0x5c, 0xb2, 0x21, 0xc9, + 0x22, 0xae, 0xa8, 0xf2, 0x2f, 0xea, 0x8e, 0x7d, 0x26, 0xea, 0x48, 0xd6, 0xe5, 0x19, 0xac, 0xd3, + 0x3f, 0x10, 0xa1, 0x3d, 0x68, 0xc8, 0xa5, 0x68, 0x5b, 0x94, 0x98, 0xb5, 0xb1, 0xf1, 0xec, 0x84, + 0x18, 0x4a, 0xb4, 0xa0, 0x39, 0x9e, 0x5d, 0xa2, 0x2d, 0xd9, 0xeb, 0xf9, 0xec, 0xc2, 0xa3, 0xa4, + 0x47, 0xcd, 0xf5, 0x4b, 0xd4, 0xd9, 0x4c, 0xe9, 0x70, 0x96, 0x09, 0xed, 0x40, 0x3d, 0xb3, 0x24, + 0xe6, 0x8f, 0xc7, 0x66, 0xd5, 0x33, 0x84, 0x10, 0x3c, 0xc2, 0x26, 0x7c, 0x3a, 0x50, 0x05, 0xd7, + 0xdc, 0x18, 0xf3, 0x69, 0x5d, 0x88, 0x71, 0x4c, 0x80, 0xd6, 0xc1, 0x08, 0xe2, 0xe2, 0x6c, 0x7e, + 0x3a, 0x36, 0xbc, 0x4c, 0xca, 0x36, 0x4e, 0x89, 0x50, 0x07, 0x0c, 0xd5, 0xe2, 0xf9, 0xec, 0xc2, + 0xfc, 0x4c, 0xf7, 0x5f, 0xc9, 0x40, 0x4f, 0x27, 0x6e, 0x5c, 0x75, 0xf4, 0x93, 0x88, 0x1a, 0x1e, + 0xb1, 0xc0, 0xfc, 0xf9, 0x58, 0xd4, 0x88, 0x7c, 0x8e, 0x25, 0x4a, 0x28, 0xcc, 0x55, 0x76, 0x37, + 0xef, 0x8d, 0x29, 0xac, 0xb3, 0x3e, 0x8e, 0x09, 0x5a, 0x7f, 0x9c, 0x4b, 0xa6, 0xe1, 0xaf, 0x3f, + 0x78, 0xfd, 0x2a, 0x5b, 0xf2, 0x32, 0x5d, 0x5e, 0xfe, 0x92, 0x2e, 0xef, 0x5a, 0xc2, 0x91, 0x99, + 0xa1, 0xff, 0x04, 0x8a, 0x22, 0xc0, 0xd0, 0x1d, 0xa8, 0x26, 0xbd, 0x68, 0x6e, 0x56, 0x2f, 0x9a, + 0x90, 0xb4, 0x7e, 0x9b, 0x87, 0xb2, 0x0a, 0x4c, 0xf4, 0xe5, 0xc4, 0x58, 0xf4, 0xc3, 0x4b, 0xe2, + 0x78, 0x72, 0x2a, 0xaa, 0x5a, 0x17, 0x39, 0x96, 0x0b, 0x2d, 0xf5, 0x85, 0xf8, 0x78, 0x18, 0x51, + 0x75, 0x19, 0x29, 0x8a, 0xd6, 0x45, 0xe1, 0x9e, 0x08, 0xd4, 0x96, 0xc0, 0xb4, 0xfe, 0x27, 0x97, + 0xce, 0x51, 0x17, 0xa1, 0xa4, 0x66, 0x3b, 0xaa, 0x24, 0xab, 0x05, 0x5a, 0x81, 0x66, 0xdf, 0xf5, + 0x2d, 0xce, 0x06, 0xa1, 0x33, 0x7a, 0xa3, 0x6e, 0xf4, 0x5d, 0xff, 0x50, 0x82, 0x55, 0x07, 0xbf, + 0xa2, 0xae, 0xfe, 0x23, 0x94, 0x05, 0x4d, 0x69, 0x3f, 0xcf, 0x52, 0xae, 0x02, 0x52, 0x54, 0xc4, + 0x22, 0xcc, 0xe1, 0x56, 0xc4, 0x22, 0xdb, 0x93, 0x35, 0xaa, 0x88, 0x9b, 0x1a, 0xb3, 0xcd, 0x1c, + 0x7e, 0x24, 0xe0, 0xa8, 0x03, 0xd7, 0x62, 0x6a, 0xb9, 0x1d, 0x4d, 0x5e, 0x92, 0xe4, 0x0b, 0x1a, + 0x25, 0xb7, 0xa3, 0xe8, 0xdb, 0x30, 0xa7, 0xfb, 0x13, 0x8b, 0x50, 0x2f, 0xd2, 0x3f, 0xb2, 0xc0, + 0x35, 0xd5, 0x88, 0x6c, 0x0b, 0x50, 0xeb, 0x0b, 0x28, 0xc9, 0x2c, 0x75, 0x49, 0x07, 0x96, 0x9b, + 0xde, 0x81, 0xb5, 0xfe, 0x37, 0x97, 0xce, 0xd9, 0x2f, 0x1b, 0x64, 0x4f, 0xc9, 0x88, 0x53, 0x8f, + 0xec, 0x35, 0x3b, 0xc0, 0xd6, 0xf0, 0xbb, 0x4e, 0xec, 0x16, 0x2c, 0xa8, 0x0c, 0x9f, 0x35, 0xae, + 0x72, 0x81, 0x79, 0x85, 0x48, 0x6d, 0xbb, 0x0a, 0x48, 0xd3, 0x66, 0x4d, 0x5b, 0x50, 0x27, 0xa1, + 0x30, 0xa9, 0x65, 0x5b, 0x15, 0x28, 0xc9, 0x94, 0xdb, 0xfa, 0xc7, 0x1c, 0x94, 0x55, 0xf2, 0x7d, + 0x65, 0xa7, 0x55, 0xe4, 0x53, 0x46, 0xf9, 0xaf, 0xb2, 0x1f, 0x95, 0xe0, 0xa7, 0xec, 0x47, 0x21, + 0x46, 0xf6, 0xa3, 0x69, 0xa7, 0xec, 0x47, 0x61, 0x32, 0xfb, 0xf9, 0x93, 0xdc, 0xe8, 0xaf, 0x01, + 0x5e, 0xdb, 0x19, 0xbe, 0xbf, 0xec, 0xb1, 0x09, 0x73, 0x23, 0xb5, 0xe4, 0x0a, 0x8e, 0xf9, 0x25, + 0xd4, 0x32, 0x15, 0xe0, 0x0a, 0x02, 0xee, 0x43, 0x3d, 0x5b, 0x42, 0x5e, 0x5f, 0xc2, 0xad, 0xdb, + 0x50, 0x92, 0x3f, 0x4a, 0x44, 0x00, 0xe5, 0x83, 0x27, 0x5b, 0xfb, 0x7b, 0xdd, 0xe6, 0x5b, 0xa8, + 0x06, 0x95, 0x03, 0xbc, 0xf7, 0x74, 0xf3, 0x68, 0xa7, 0x99, 0x43, 0x06, 0x94, 0xf6, 0x1f, 0x77, + 0x37, 0xf7, 0x9b, 0xf9, 0x8d, 0x07, 0x50, 0xd5, 0x3f, 0x1a, 0x0b, 0xd1, 0x2f, 0xa0, 0xa2, 0x9f, + 0xd1, 0xdb, 0x69, 0xa2, 0x1f, 0xf9, 0x39, 0x63, 0xcb, 0x9c, 0x44, 0xa8, 0x06, 0x76, 0x3d, 0xb7, + 0xb1, 0x0f, 0x55, 0xfd, 0x41, 0x22, 0x44, 0xf7, 0xa1, 0xa2, 0x9f, 0x33, 0xb2, 0x46, 0x3f, 0x2b, + 0x65, 0x64, 0x8d, 0x7d, 0xc7, 0x58, 0xc9, 0xad, 0xe7, 0x36, 0x4e, 0xa1, 0x31, 0x3a, 0xea, 0x47, + 0x4f, 0x61, 0x5e, 0x3e, 0x24, 0x60, 0x8e, 0x6e, 0x66, 0xeb, 0xca, 0xe4, 0x07, 0x83, 0xd6, 0xd2, + 0x4c, 0x7c, 0xe6, 0x4d, 0x5f, 0x41, 0x79, 0x5f, 0xfd, 0xb6, 0xed, 0xde, 0xe8, 0x45, 0x63, 0x71, + 0x5a, 0x10, 0xb5, 0xa6, 0x42, 0xa5, 0xa0, 0xdd, 0x78, 0xf6, 0xfd, 0x66, 0x72, 0xb6, 0x7e, 0xf1, + 0xed, 0x7f, 0xdc, 0x7c, 0xeb, 0xdb, 0xdf, 0xdc, 0xcc, 0xfd, 0xf3, 0x6f, 0x6e, 0xe6, 0xfe, 0xf2, + 0x3f, 0x6f, 0xe6, 0xfe, 0x68, 0xf5, 0x95, 0x7e, 0x98, 0xa6, 0xe5, 0x1d, 0x97, 0x25, 0xe8, 0xd3, + 0xdf, 0x05, 0x00, 0x00, 0xff, 0xff, 0x8b, 0xc4, 0xc3, 0xd1, 0x81, 0x2c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -5745,8 +5745,8 @@ func (m *Persist) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x10 } - if m.Nonce != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.Nonce)) + if m.SeqNo != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.SeqNo)) i-- dAtA[i] = 0x8 } @@ -5777,8 +5777,8 @@ func (m *Persisted) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } - if m.Nonce != 0 { - i = encodeVarintRuntime(dAtA, i, uint64(m.Nonce)) + if m.SeqNo != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.SeqNo)) i-- dAtA[i] = 0x8 } @@ -7931,8 +7931,8 @@ func (m *Persist) ProtoSize() (n int) { } var l int _ = l - if m.Nonce != 0 { - n += 1 + sovRuntime(uint64(m.Nonce)) + if m.SeqNo != 0 { + n += 1 + sovRuntime(uint64(m.SeqNo)) } if m.DeleteAckIntents { n += 2 @@ -8009,8 +8009,8 @@ func (m *Persisted) ProtoSize() (n int) { } var l int _ = l - if m.Nonce != 0 { - n += 1 + sovRuntime(uint64(m.Nonce)) + if m.SeqNo != 0 { + n += 1 + sovRuntime(uint64(m.SeqNo)) } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) @@ -13831,9 +13831,9 @@ func (m *Persist) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Nonce", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SeqNo", wireType) } - m.Nonce = 0 + m.SeqNo = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -13843,7 +13843,7 @@ func (m *Persist) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Nonce |= uint64(b&0x7F) << shift + m.SeqNo |= uint64(b&0x7F) << shift if b < 0x80 { break } @@ -14433,9 +14433,9 @@ func (m *Persisted) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Nonce", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SeqNo", wireType) } - m.Nonce = 0 + m.SeqNo = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowRuntime @@ -14445,7 +14445,7 @@ func (m *Persisted) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Nonce |= uint64(b&0x7F) << shift + m.SeqNo |= uint64(b&0x7F) << shift if b < 0x80 { break } diff --git a/go/protocols/runtime/runtime.proto b/go/protocols/runtime/runtime.proto index 96aac6241a4..1a008f6eade 100644 --- a/go/protocols/runtime/runtime.proto +++ b/go/protocols/runtime/runtime.proto @@ -538,13 +538,13 @@ message Recover { // 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 +// `seq_no` 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; + uint64 seq_no = 1; // Delete previously-persisted ACK intents. Applies ahead of `ack_intents`. // Effect: DeleteRange("AI:") bool delete_ack_intents = 2; @@ -591,8 +591,8 @@ message Persist { // 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; + // Echoed back from the originating `Persist.seq_no` request. + uint64 seq_no = 1; } // Apply asks shard zero to invoke its connector's Apply action, both for From 482d12975812aab1776ad003327ccc6e99e9fe00 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 21:54:49 +0000 Subject: [PATCH 07/22] dekaf-e2e: provision test tenant via onboarding API; unify on FLOW_AUTH_TOKEN Replace the direct storage_mappings/grants inserts in `local:test-tenant` with a betaOnboard directive, mint a multi-use refresh token, and emit `~/flow-local/test-tenant.env`. Raise the new tenant's task/collection quotas so concurrent integration suites don't trip the default ceiling. flowctl: collapse FLOW_ACCESS_TOKEN into FLOW_AUTH_TOKEN, which now accepts either a JWT access token or a base64 refresh-token JSON; drop the now-unused base64 dependency. ci:dekaf-e2e and the dekaf e2e harness take FLOW_AUTH_TOKEN / FLOW_TEST_TENANT from that env file instead of a hard-coded system-user token. Also symlink CLAUDE.md -> AGENTS.md and add local/README.md documenting the local-stack systemd topology. --- AGENTS.md | 32 +++-- CLAUDE.md | 134 +------------------ Cargo.lock | 1 - crates/dekaf/tests/e2e/harness.rs | 18 +-- crates/flowctl/Cargo.toml | 1 - crates/flowctl/src/config.rs | 30 ++--- local/README.md | 207 ++++++++++++++++++++++++++++++ mise/tasks/ci/dekaf-e2e | 32 +---- mise/tasks/local/test-tenant | 166 ++++++++++++++++++++---- 9 files changed, 397 insertions(+), 224 deletions(-) mode change 100644 => 120000 CLAUDE.md create mode 100644 local/README.md diff --git a/AGENTS.md b/AGENTS.md index 9a4daaf374c..d67e4df548b 100644 --- a/AGENTS.md +++ b/AGENTS.md @@ -1,5 +1,3 @@ -# CLAUDE.md - Estuary is a real-time data platform with: - Control plane: user-facing catalog management APIs - Data planes: distributed runtime execution @@ -38,17 +36,18 @@ mise run ci:sql-tap # E2E tests over derivation examples (SLOW) mise run ci:catalog-test -``` -### Development - -A development Supabase instance is available: -```bash +# Start (just) local Supabase. +mise run local:supabase # Reset with current migrations as needed supabase db reset - # Interact directly with dev DB psql postgresql://postgres:postgres@localhost:5432/postgres -c 'SELECT 1;' + +# Start a complete local stack (see local/README.md) +mise run local:stack +# CLI for interacting with the platform. +cargo run -p flowctl -- --profile local --help ``` ## Architecture Overview @@ -69,6 +68,13 @@ 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. +Collections and tasks live in a unified, hierarchical namespace. +`/`-delimited prefixes act as "roles" and are the unit of AuthZ. +Users are granted capabilities to roles (`user_grants` table), +and roles are granted capabilities to other roles (`role_grants`). +A top-level prefix like `acmeCo/` homes an organization and +is called a "tenant". + ### Control-plane components - **Supabase**: catalog and platform config DB - **Agent**: APIs and background automation @@ -83,9 +89,9 @@ and activates specs into their associated data plane. ### 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 +- `go/protocols/capture/capture.proto` +- `go/protocols/derive/derive.proto` +- `go/protocols/materialize/materialize.proto` ## README.md @@ -115,8 +121,8 @@ Keep READMEs current - update with code changes. 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. +- Avoid routines with trivial bodies that could be inlined into the caller. + Indirection has cost (hard to read): each routine 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. diff --git a/CLAUDE.md b/CLAUDE.md deleted file mode 100644 index 9a4daaf374c..00000000000 --- a/CLAUDE.md +++ /dev/null @@ -1,133 +0,0 @@ -# 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 new file mode 120000 index 00000000000..47dc3e3d863 --- /dev/null +++ b/CLAUDE.md @@ -0,0 +1 @@ +AGENTS.md \ No newline at end of file diff --git a/Cargo.lock b/Cargo.lock index f93a95f221c..e6d83da5775 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3330,7 +3330,6 @@ dependencies = [ "assert_cmd", "async-process", "axum", - "base64 0.22.1", "build", "bytes", "chrono", diff --git a/crates/dekaf/tests/e2e/harness.rs b/crates/dekaf/tests/e2e/harness.rs index bca972aa946..457ec1daa25 100644 --- a/crates/dekaf/tests/e2e/harness.rs +++ b/crates/dekaf/tests/e2e/harness.rs @@ -3,9 +3,12 @@ use std::collections::BTreeMap; use std::sync::OnceLock; use std::time::Duration; -/// Prefix for test namespaces. Requires storage mappings and user grants -/// provisioned by `mise run local:test-tenant`. -const TEST_NAMESPACE_PREFIX: &str = "test/dekaf"; +/// Prefix for test namespaces, provisioned by `mise run local:test-tenant`, +/// which must match FLOW_AUTH_TOKEN credential. +fn test_namespace_prefix() -> String { + let tenant = std::env::var("FLOW_TEST_TENANT").expect("FLOW_TEST_TENANT must be set"); + format!("{tenant}/dekaf") +} pub fn init_tracing() { let _ = tracing_subscriber::fmt() @@ -19,7 +22,7 @@ pub fn init_tracing() { } /// Create a flowctl command configured for local stack. -/// Requires FLOW_ACCESS_TOKEN environment variable to be set. +/// Requires FLOW_AUTH_TOKEN environment variable to be set. fn flowctl_command() -> anyhow::Result { // Try to find flowctl in cargo-target/debug first (where `cargo build` puts it), // falling back to locate_bin (which checks alongside the test binary and PATH). @@ -36,11 +39,10 @@ fn flowctl_command() -> anyhow::Result { let home = std::env::var("HOME").unwrap(); let ca_cert = std::env::var("SSL_CERT_FILE").unwrap_or_else(|_| format!("{}/flow-local/ca.crt", home)); - let access_token = std::env::var("FLOW_ACCESS_TOKEN") - .context("FLOW_ACCESS_TOKEN environment variable must be set for e2e tests")?; + let auth_token = std::env::var("FLOW_AUTH_TOKEN").context("FLOW_AUTH_TOKEN must be set")?; let mut cmd = async_process::Command::new(flowctl); - cmd.env("FLOW_ACCESS_TOKEN", access_token); + cmd.env("FLOW_AUTH_TOKEN", auth_token); cmd.env("SSL_CERT_FILE", ca_cert); cmd.arg("--profile").arg("local"); Ok(cmd) @@ -61,7 +63,7 @@ impl DekafTestEnv { /// rewritten to include a unique test namespace. pub async fn setup(test_name: &str, fixture_yaml: &str) -> anyhow::Result { let suffix = format!("{:04x}", rand::random::()); - let namespace = format!("{TEST_NAMESPACE_PREFIX}/{test_name}/{suffix}"); + let namespace = format!("{}/{test_name}/{suffix}", test_namespace_prefix()); tracing::info!(%namespace, "Setting up test environment"); diff --git a/crates/flowctl/Cargo.toml b/crates/flowctl/Cargo.toml index 6b997abb0c3..30872ed1d1e 100644 --- a/crates/flowctl/Cargo.toml +++ b/crates/flowctl/Cargo.toml @@ -45,7 +45,6 @@ validation = { path = "../validation" } anyhow = { workspace = true } axum = { workspace = true } -base64 = { workspace = true } bytes = { workspace = true } chrono = { workspace = true } clap = { workspace = true } diff --git a/crates/flowctl/src/config.rs b/crates/flowctl/src/config.rs index 68865e02193..29968810cbf 100644 --- a/crates/flowctl/src/config.rs +++ b/crates/flowctl/src/config.rs @@ -1,5 +1,4 @@ use anyhow::Context; -use base64::Engine; use std::path::PathBuf; use flow_client::{ @@ -158,26 +157,26 @@ impl Config { config.user_refresh_token = refresh_token; } - // If a refresh token is not defined, attempt to parse one from the environment. - if config.user_refresh_token.is_none() { - if let Ok(env_token) = std::env::var(FLOW_AUTH_TOKEN) { - let decoded = base64::engine::general_purpose::STANDARD - .decode(env_token) + // FLOW_AUTH_TOKEN, if present, overrides a refresh or access token + // loaded from disk. A value with three dot-delimited segments is a JWT + // access token; anything else is a base64-encoded refresh token JSON. + if let Ok(env_token) = std::env::var(FLOW_AUTH_TOKEN) { + if env_token.split('.').count() == 3 { + tracing::info!("using FLOW_AUTH_TOKEN environment access token"); + config.user_refresh_token = None; + config.user_access_token = Some(env_token); + } else { + let decoded = tokens::jwt::parse_base64(&env_token) .context("FLOW_AUTH_TOKEN is not base64")?; let token: RefreshToken = serde_json::from_slice(&decoded).context("FLOW_AUTH_TOKEN is invalid JSON")?; - tracing::info!("using refresh token from environment variable {FLOW_AUTH_TOKEN}"); + tracing::info!("using FLOW_AUTH_TOKEN environment refresh token"); config.user_refresh_token = Some(token); + config.user_access_token = None; } } - // Allow overriding access token via environment variable for CI/automation. - if let Ok(token) = std::env::var(FLOW_ACCESS_TOKEN) { - tracing::info!("using access token from environment variable {FLOW_ACCESS_TOKEN}"); - config.user_access_token = Some(token); - } - config.is_local = profile == "local"; Ok(config) @@ -231,7 +230,6 @@ impl Config { } } -// Environment variable which is inspected for a base64-encoded refresh token. +// Environment variable inspected for an auth credential: either a JWT access +// token, or a base64-encoded refresh token JSON. const FLOW_AUTH_TOKEN: &str = "FLOW_AUTH_TOKEN"; -// Environment variable which is inspected for an access token (for CI/automation). -const FLOW_ACCESS_TOKEN: &str = "FLOW_ACCESS_TOKEN"; diff --git a/local/README.md b/local/README.md new file mode 100644 index 00000000000..7d70649bb4f --- /dev/null +++ b/local/README.md @@ -0,0 +1,207 @@ +# Local Stack + +This directory holds the systemd units and helper scripts that `mise run local:*` +glues together into a working Estuary stack. The interesting machinery is in +`local/systemd/` (unit files) and `mise/tasks/local/` (drivers that emit env +files and dropins, then `systemctl --user start`). + +## Topology at a glance + +``` +flow-supabase.service ──┐ +flow-config-encryption ─┤ flow-control-plane.target +flow-control-agent.service ─┘ (8675 agent, 8765 config-encryption) + +flow-etcd.service (single-node, http://etcd.flow.localhost:2379) + +flow-plane@.target ─────────── one per data plane + ├─ flow-gazette@- broker(s), base_port + 0..9 + ├─ flow-reactor@- reactor(s), base_port + 99..90 + ├─ flow-runtime-sidecar@ Rust sidecar, base_port + 60 + ├─ flow-dekaf@ Kafka shim, base_port + 50/51/52 + └─ flow-plane-link@ oneshot: POSTs to agent to register + +flow-dekaf-kafka.service Confluent Kafka in Docker, fixed :29092 +``` + +`local:stack` boots the control plane plus a `local-cluster` data plane at +port-block 8000 with 4 brokers, 1 reactor, dekaf, and link, and then publishes +`ops-catalog/local-view.bundle.json` into it. + +## The port scheme is load-bearing + +Each data plane gets a 100-wide block starting on a multiple of 100. Within the +block: + +| Offset | Service | +| --- | --- | +| +0..+9 | brokers (count up from block start) | +| +50, +51, +52 | dekaf kafka / registry / metrics | +| +60 | runtime-sidecar | +| +90..+99 | reactors (count down from block end) | + +So a "single reactor" lands on `block+99` — `local-cluster` puts its sole +reactor at **8099**, not 8004. The data-plane script enforces `% 100 == 0` on +the base port. `mise/tasks/vm/port-forward` hardcodes three reserved blocks: +**8000–8099**, **10000–10099**, **10100–10199** — those are your three +forwardable data planes from a VM. + +The dockerized Kafka that backs Dekaf consumer groups lives outside any block, +on **:29092**, and is shared across all data planes. + +## Topology is built from drop-ins, not static units + +The template units in `local/systemd/` are deliberately minimal — they don't +know about any specific data plane. mise tasks write out dropins under +`~/.config/systemd/user/`: + +- `flow-plane@.target.d/wants-*.conf` — `Wants=` to pull each instance in +- `flow-@.service.d/plane.conf` — `PartOf=`/`After=` to bind to the target + +`PartOf=` is what makes `systemctl --user stop flow-plane@local-cluster.target` +cascade-stop every broker, reactor, sidecar, dekaf, and link. `Wants=` does +**not** cascade in reverse — starting the target pulls children up, but +starting a child doesn't pull the target up. + +Useful incantations: + +```bash +systemctl --user list-dependencies flow-plane@local-cluster.target +systemctl --user list-units 'flow-*' +journalctl --user -u flow-gazette@local-cluster-8000 -f +journalctl --user -u flow-runtime-sidecar@local-cluster -f +``` + +## You don't have to run the whole stack + +`local:stack` is a convenience that chains `local:control-plane` → +`local:data-plane local-cluster 8000 ...` → bundled-catalog publish. Each +piece is its own task and pulls only what it needs via `#MISE depends=`: + +| Task | Pulls in | When you want it | +| --- | --- | --- | +| `local:supabase` | (just supabase) | Hacking migrations / pgTAP; running Rust integration tests | +| `local:control-plane` | supabase, config-encryption, agent | Agent / controller work without any data plane | +| `local:data-plane ` | etcd, brokers, reactor, runtime-sidecar | A second data plane alongside `local-cluster` (use port-block 10000 or 10100), or a data plane with no control plane at all if you skip `--link` | +| `local:data-plane-controller` | supabase | Dry-run controller convergence loop | + +There's no "down" for an individual task — use `systemctl --user stop` on the +specific unit (or `flow-plane@.target` for a whole data plane). `local:stop` +is the "burn it all" button. + +## `local:stop` is a guillotine + +It does much more than `systemctl stop`: + +``` +systemctl --user stop 'flow-*' +rm -r ~/.config/systemd/user # ALL unit symlinks + dropins +rm -r ~/flow-local/{builds,env,etcd,fragments} +``` + +So journal fragments, Etcd state, generated env files, and the topology dropins +all vanish. If you want to bounce a service without nuking state, stop it +directly. If you want to keep state across a planned restart, stop the target +but leave `~/flow-local/` and `~/.config/systemd/user/` alone. + +Also note: `flow-plane-link@.service` has an `ExecStop=` that deletes +`live_specs` + the data plane row from Postgres. That fires whenever the link +service stops — even on a tidy `systemctl stop flow-plane@.target`. + +## Supabase: first start vs every subsequent start + +`flow-supabase.service` is unusual. Its `ExecStartPre`s: + +1. `supabase start --exclude edge-runtime` (idempotent, starts Docker containers) +2. **If `~/flow-local/.supabase-started` exists**, `supabase db reset` — wipes + and re-applies migrations +3. `touch ~/flow-local/.supabase-started` + +Then `ExecStart` runs `supabase functions serve` as the foreground process. +`ExecStopPost` runs `supabase stop` to tear down the Docker containers. + +The implication: **every restart after the first one wipes the database**. If +you `systemctl restart flow-supabase`, you lose all catalog state. If you want +to preserve it, delete the flag file beforehand or just don't restart. + +## Secrets you might think are real but aren't + +- **`super-secret-jwt-token-with-at-least-32-characters-long`** — the Supabase + JWT secret. The pre-minted `SYSTEM_USER_TOKEN` baked into `plane-link-*.env` + is just a JWT signed with it for user `ffffffff-ffff-ffff-ffff-ffffffffffff` + (the `support@estuary.dev` system user), with exp in 2055. +- **`AGE-SECRET-KEY-1UX6ZHA...`** — SOPS age key used by both reactor and + dekaf. Its public counterpart `age1z2qskpk...` is the `KMS_KEY` the + config-encryption service is configured with. Anything sops-encrypted by the + local stack opens with `export SOPS_AGE_KEY=AGE-SECRET-KEY-1UX...`. +- **`key-`**, base64-encoded — the HMAC auth key shared by + brokers, reactors, runtime-sidecar, and the agent (via the link payload). + For `local-cluster` that's `a2V5LWxvY2FsLWNsdXN0ZXI=`. + +If you ever wonder "is this real" — none of these are; they're checked into +the repo. If something else is hardcoded in mise tasks, assume the same. + +## TLS — one CA, one wildcard, everywhere + +`mise/tasks/local/tls-cert` generates a single self-signed CA and a single +server cert in `~/flow-local/` with SAN `DNS:*.flow.localhost,DNS:flow.localhost,IP:127.0.0.1`. +The cert is reused by every broker, reactor, and sidecar. Hosts use names like +`broker-.flow.localhost` and `reactor-.flow.localhost` precisely so +they validate against the wildcard. Certs are long-lived (365 days). + +`.flow.localhost` works without `/etc/hosts` entries: systemd-resolved +implements RFC 6761 and resolves any `*.localhost` to `::1`/`127.0.0.1`. + +## Where state lives + +| Path | What | +| --- | --- | +| `~/flow-local/ca.{crt,key}`, `server.{crt,key}` | TLS material | +| `~/flow-local/env/*.env` | Generated systemd env files, one per instance | +| `~/flow-local/etcd/` | Etcd data dir | +| `~/flow-local/fragments//` | Gazette journal fragments (file-only mode) | +| `~/flow-local/builds/` | Catalog builds emitted by the agent | +| `~/flow-local/.supabase-started` | Sentinel that triggers `db reset` on next supabase start | +| `~/.config/systemd/user/flow-*` | Linked unit files + drop-in topology | + +Binaries: the unit files run `~/cargo-target/debug/` because +`CARGO_TARGET_DIR=~/cargo-target` is set by mise, not under the repo. Most +units have an `ExecStartPre=cargo build -p ` so a unit restart picks up +your latest source — handy when iterating. + +## Provisioning tenant credentials + +After data plane(s) are started, register a new tenant and credentials: +```bash +mise run local:test-tenant --tenant acmeCo --user alice@example.com +source ~/flow-local/test-tenant.env # Written by test-tenant. +flowctl --profile local catalog list +``` + +flowctl picks up and uses `FLOW_AUTH_TOKEN` (a refresh token), +which takes precedence over loaded config. + +## flowctl against the local stack + +`flowctl --profile local` flips a single boolean in flowctl's config that +swaps every URL to a localhost equivalent: + +| | local | default | +| --- | --- | --- | +| agent | `http://localhost:8675` | `https://api.estuary.dev` | +| postgrest | `http://localhost:5431/rest/v1` | hosted | +| dashboard | `http://localhost:3000` | hosted | +| config-encryption | `http://localhost:8765` | hosted | + +The `--profile` flag also controls which config file in +`~/.config/flowctl/` is loaded. + +## What `local:stack` is actually publishing + +After the data plane comes up, `local:stack` opens a psql session and writes a +draft + publication of `ops-catalog/local-view.bundle.json` to the +`ops/dp/public/local-cluster` data plane, as the system user +`support@estuary.dev`. This is what makes `flowctl --profile local catalog list` +return the `ops/rollups/...` and `ops/dp/public/local-cluster/...` specs. +`local/ops-publication.sh` is the standalone version of that same SQL if you +want to publish your own bundle the same way. diff --git a/mise/tasks/ci/dekaf-e2e b/mise/tasks/ci/dekaf-e2e index 6a212dda889..ba4485b2759 100755 --- a/mise/tasks/ci/dekaf-e2e +++ b/mise/tasks/ci/dekaf-e2e @@ -2,7 +2,6 @@ set -euo pipefail #MISE description="Run Dekaf E2E tests" -#MISE depends=["local:test-tenant"] #USAGE flag "--filter " help="Test filter pattern for nextest" #USAGE flag "--update-snapshots" help="Update insta snapshots" #USAGE flag "--concurrency " help="How many tests to run at once" @@ -20,36 +19,18 @@ CARGO_TARGET_DIR="${CARGO_TARGET_DIR:-${HOME}/cargo-target}" DEKAF1_KAFKA_PORT=8050 # 8000 + 50 DEKAF2_KAFKA_PORT=8250 # 8200 + 50 -SYSTEM_USER_TOKEN="eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJodHRwOi8vMTI3LjAuMC4xOjU0MzEvYXV0aC92MSIsInN1YiI6ImZmZmZmZmZmLWZmZmYtZmZmZi1mZmZmLWZmZmZmZmZmZmZmZiIsImF1ZCI6ImF1dGhlbnRpY2F0ZWQiLCJleHAiOjI3MDAwMDAwMDAsImlhdCI6MTcwMDAwMDAwMCwiZW1haWwiOiJzdXBwb3J0QGVzdHVhcnkuZGV2Iiwicm9sZSI6ImF1dGhlbnRpY2F0ZWQiLCJpc19hbm9ueW1vdXMiOmZhbHNlfQ.Nb-N4s_YnObBHGivSTe_8FEniVUUpehzrRkF5JgNWWU" - -# Wait for agent to be ready by checking if it's listening -echo "Waiting for agent..." -TIMEOUT=120 -ELAPSED=0 -until nc -z localhost 8675 2>/dev/null; do - sleep 1 - ELAPSED=$((ELAPSED + 1)) - if [ "${ELAPSED}" -ge "${TIMEOUT}" ]; then - echo "Error: Agent failed to start within ${TIMEOUT} seconds" - exit 1 - fi -done -echo "Agent is ready" - echo "Building flowctl..." cargo build -p flowctl --quiet -# local-cluster with Dekaf is expected to already be running from local:stack -# Only set up local-cluster-2 for migration tests +# local-cluster with Dekaf is expected to already be running from local:stack. +# Bring up local-cluster-2 BEFORE provisioning the tenant so that its +# initialized storage mapping picks up both public planes. echo "Setting up local-cluster-2 dataplane with Dekaf..." mise run local:data-plane local-cluster-2 8200 --num-brokers 4 --num-reactors 1 --link --dekaf -echo "Updating storage mappings for migration tests..." -psql "postgresql://postgres:postgres@localhost:5432/postgres" -c " - UPDATE storage_mappings - SET spec = jsonb_set(spec::jsonb, '{data_planes}', '[\"ops/dp/public/local-cluster\", \"ops/dp/public/local-cluster-2\"]') - WHERE catalog_prefix = 'test/'; -" +echo "Provisioning test tenant..." +mise run local:test-tenant +source "${FLOW_LOCAL}/test-tenant.env" echo "Waiting for Dekaf instances..." TIMEOUT=60 @@ -72,7 +53,6 @@ echo "Running Dekaf E2E tests..." export DEKAF_BROKER="localhost:${DEKAF1_KAFKA_PORT}" export DEKAF_REGISTRY="http://localhost:$((DEKAF1_KAFKA_PORT + 1))" export SSL_CERT_FILE="${FLOW_LOCAL}/ca.crt" -export FLOW_ACCESS_TOKEN="${SYSTEM_USER_TOKEN}" export DEKAF_UPSTREAM_BROKER_URLS="tcp://localhost:29092" export DEKAF_ENCRYPTION_SECRET="test-encryption-secret-32-bytes!" diff --git a/mise/tasks/local/test-tenant b/mise/tasks/local/test-tenant index 2013e47e480..79247b3a3c6 100755 --- a/mise/tasks/local/test-tenant +++ b/mise/tasks/local/test-tenant @@ -1,31 +1,145 @@ #!/usr/bin/env bash set -euo pipefail -#MISE description="Provision test/ tenant for integration tests" +#MISE description="Provision a tenant via the onboarding API and emit flowctl credentials" +#MISE depends=["local:control-plane"] +#USAGE flag "--tenant " help="Tenant name without trailing slash (default: test)" +#USAGE flag "--user " help="Email of an auth.users row to own the tenant (default: alice@example.com). NOTE: onboarding API rejects users that already hold any grants, so each seeded user (alice/bob/carol/dave from seed.sql) can own at most one tenant." +TENANT="${usage_tenant:-test}" +USER_EMAIL="${usage_user:-alice@example.com}" PGURL="postgresql://postgres:postgres@localhost:5432/postgres" -DATA_PLANE_NAME="ops/dp/public/local-cluster" - -echo "Provisioning test/ tenant..." - -psql "${PGURL}" -c " -INSERT INTO public.storage_mappings (catalog_prefix, spec) -VALUES - ('test/', '{\"stores\": [{\"provider\": \"GCS\", \"bucket\": \"estuary-trial\", \"prefix\": \"collection-data/\"}], \"data_planes\": [\"${DATA_PLANE_NAME}\"]}'), - ('recovery/test/', '{\"stores\": [{\"provider\": \"GCS\", \"bucket\": \"estuary-trial\"}]}') -ON CONFLICT (catalog_prefix) DO NOTHING; -" - -# Grant admin access to support@estuary.dev for test/ prefix -psql "${PGURL}" -c " -INSERT INTO public.user_grants (user_id, object_role, capability) -VALUES ('ffffffff-ffff-ffff-ffff-ffffffffffff', 'test/', 'admin') -ON CONFLICT (user_id, object_role) DO NOTHING; -" - -# Grant role permissions so tasks within test/ can read/write to each other -psql "${PGURL}" -c " -INSERT INTO public.role_grants (subject_role, object_role, capability) -VALUES ('test/', 'test/', 'admin') -ON CONFLICT (subject_role, object_role) DO NOTHING; -" +ENV_FILE="${HOME}/flow-local/test-tenant.env" +AGENT_PORT=8675 + +# Onboarding rejects names that don't satisfy the catalog_tenant regex. +if [[ ! "${TENANT}" =~ ^[A-Za-z0-9._-]+$ ]]; then + echo "Error: --tenant '${TENANT}' must match [A-Za-z0-9._-]+ (no trailing slash, no '/')" >&2 + exit 1 +fi +# Guard against SQL injection through unquoted interpolations below. +if [[ ! "${USER_EMAIL}" =~ ^[A-Za-z0-9._%+-]+@[A-Za-z0-9.-]+$ ]]; then + echo "Error: --user '${USER_EMAIL}' is not a valid email address" >&2 + exit 1 +fi + +echo "Waiting for agent on :${AGENT_PORT}..." +ELAPSED=0 +TIMEOUT=120 +until nc -z localhost "${AGENT_PORT}" 2>/dev/null; do + sleep 1 + ELAPSED=$((ELAPSED + 1)) + if [ "${ELAPSED}" -ge "${TIMEOUT}" ]; then + echo "Error: agent did not start within ${TIMEOUT}s" >&2 + exit 1 + fi +done + +USER_ID=$(psql "${PGURL}" -tAqc \ + "SELECT id FROM auth.users WHERE email = '${USER_EMAIL}'") +if [ -z "${USER_ID}" ]; then + echo "Error: no auth.users row with email '${USER_EMAIL}'" >&2 + echo "Hint: seed.sql provisions alice/bob/carol/dave@example.com" >&2 + exit 1 +fi + +PREFIX="${TENANT}/" +TENANT_EXISTS=$(psql "${PGURL}" -tAqc \ + "SELECT 1 FROM public.tenants WHERE tenant = '${PREFIX}'") + +if [ -z "${TENANT_EXISTS}" ]; then + # Onboarding snapshots the set of `ops/dp/public/*` data planes into + # storage_mappings.data_planes at provisioning time. Adding a data plane + # later and re-running this task will NOT update that snapshot — bring up + # every data plane the tenant should use before provisioning. + echo "Provisioning tenant ${PREFIX} owned by ${USER_EMAIL} via onboarding API..." + + # Insert with the default queued status; the create_directive_task trigger + # enqueues an agent task and applied_directives_agent_notifications wakes + # the agent. user_claims must be set or the trigger won't fire. + APPLIED_ID=$(psql "${PGURL}" -tAqc " + INSERT INTO public.applied_directives (directive_id, user_id, user_claims) + SELECT id, '${USER_ID}'::uuid, '{\"requestedTenant\":\"${TENANT}\"}'::json_obj + FROM public.directives + WHERE token = '453e00cd-e12a-4ce5-b12d-3837aa385751' + RETURNING id + ") + if [ -z "${APPLIED_ID}" ]; then + echo "Error: betaOnboard directive (token 453e00cd-...) not found in directives table" >&2 + exit 1 + fi + + echo "Waiting for agent to apply directive ${APPLIED_ID}..." + ELAPSED=0 + TIMEOUT=60 + while true; do + STATUS=$(psql "${PGURL}" -tAqc \ + "SELECT job_status->>'type' FROM public.applied_directives WHERE id = '${APPLIED_ID}'") + case "${STATUS}" in + success) + break + ;; + queued) + ;; + *) + ERR=$(psql "${PGURL}" -tAqc \ + "SELECT job_status->>'error' FROM public.applied_directives WHERE id = '${APPLIED_ID}'") + echo "Error: betaOnboard returned status '${STATUS}': ${ERR}" >&2 + exit 1 + ;; + esac + sleep 1 + ELAPSED=$((ELAPSED + 1)) + if [ "${ELAPSED}" -ge "${TIMEOUT}" ]; then + echo "Error: directive did not apply within ${TIMEOUT}s (still ${STATUS})" >&2 + exit 1 + fi + done + echo "Tenant ${PREFIX} provisioned." +else + echo "Tenant ${PREFIX} already exists; skipping provisioning." +fi + +# Onboarding stamps the default per-tenant quotas (10 tasks / 500 collections). +# Raise the ceiling well above anything a test run could reach. +psql "${PGURL}" -tAqc \ + "UPDATE public.tenants SET tasks_quota = 1000, collections_quota = 1000 WHERE tenant = '${PREFIX}'" >/dev/null + +echo "Minting multi-use refresh token for ${USER_EMAIL}..." +# Insert directly (bypassing create_refresh_token's SECURITY DEFINER + auth_uid()) +# so we don't need to fake JWT claims. We control the DB here. +TOKEN_JSON=$(psql "${PGURL}" -tAqc " + WITH gen AS ( + SELECT gen_random_uuid()::text AS secret + ), + ins AS ( + INSERT INTO public.refresh_tokens (user_id, multi_use, valid_for, hash, detail) + SELECT '${USER_ID}'::uuid, true, interval '90 days', crypt(gen.secret, gen_salt('bf')), + 'local:test-tenant' + FROM gen + RETURNING id + ) + SELECT json_build_object('id', ins.id, 'secret', gen.secret)::text + FROM ins, gen +") +if [ -z "${TOKEN_JSON}" ]; then + echo "Error: failed to mint refresh token" >&2 + exit 1 +fi + +FLOW_AUTH_TOKEN=$(printf '%s' "${TOKEN_JSON}" | base64 -w0) + +mkdir -p "$(dirname "${ENV_FILE}")" +cat > "${ENV_FILE}" < Date: Fri, 15 May 2026 21:55:00 +0000 Subject: [PATCH 08/22] docs: consolidate runtime sidecar naming Use "runtime sidecar" consistently across the runtime-next README and the runtime-v2 plan, replacing the mix of "shuffle sidecar" and "runtime-sidecar process" phrasings. --- crates/runtime-next/README.md | 8 ++++---- plans/runtime-v2/plan.md | 19 ++++++++++++++----- 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/crates/runtime-next/README.md b/crates/runtime-next/README.md index 6587a75d6c2..7a8e1a5840e 100644 --- a/crates/runtime-next/README.md +++ b/crates/runtime-next/README.md @@ -32,10 +32,10 @@ Reactor machine │ │ (RocksDB + Go Recorder on the shard hosting the recovery log) │ └─ Capture: per-shard RocksDB with Go Recorder │ - └─ shuffle sidecar process (Rust, one per machine) + └─ runtime 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 + └─ Listens on the fixed sidecar port, shared fleet-wide ``` The Gazette consumer framework's transaction lifecycle is **bypassed @@ -92,7 +92,7 @@ src/ 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`. + one of these and registers it on the sidecar 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 @@ -136,7 +136,7 @@ documented inline in the proto. 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 +same runtime. The runtime 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. diff --git a/plans/runtime-v2/plan.md b/plans/runtime-v2/plan.md index 63ef8a52943..91ff8465b70 100644 --- a/plans/runtime-v2/plan.md +++ b/plans/runtime-v2/plan.md @@ -38,10 +38,10 @@ Reactor machine │ │ (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) + └─ runtime 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) + └─ Listens on the fixed sidecar port (same across the fleet) ``` Three layers interact: @@ -75,7 +75,7 @@ Three layers interact: - 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, +3. **Runtime 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: @@ -85,7 +85,7 @@ Three layers interact: - **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 + Both listen on the **sidecar 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 @@ -102,6 +102,15 @@ Three layers interact: a smaller reactor Rust + CGO surface. A sidecar crash fail-stops every joined session on the machine. + **Local stack exception.** Production continues to run one sidecar + per reactor machine. Local development runs one sidecar per data + plane on a deterministic port inside that plane's 100-port block: + `base_port + 60`. The local sidecar advertises + `https://reactor-${DATA_PLANE}.flow.localhost:${SIDECAR_PORT}` and + uses the same per-data-plane HMAC key and FQDN as the local reactor. + This preserves per-plane auth isolation without teaching the + sidecar to multiplex credentials for multiple local data planes. + 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. @@ -229,7 +238,7 @@ complete runtime in stages of increasing blast radius. - **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 + reactor. The runtime 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, From 68cc4aa997f66d36ac079ba6dfe34a4ab9c853bb Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 21:56:04 +0000 Subject: [PATCH 09/22] runtime-sidecar: binary hosting Shuffle and Leader gRPC services Add `crates/runtime-sidecar/`, the per-machine Rust process that hosts the Shuffle and Shuffle Leader gRPC services for all V2 tasks on a reactor machine. It listens on a fixed fleet-wide sidecar port, optionally terminates TLS, and is supervised with the same lifetime as the reactor process(es) it serves. --- Cargo.lock | 22 ++++ crates/runtime-sidecar/Cargo.toml | 32 ++++++ crates/runtime-sidecar/README.md | 23 ++++ crates/runtime-sidecar/src/lib.rs | 167 +++++++++++++++++++++++++++++ crates/runtime-sidecar/src/main.rs | 47 ++++++++ 5 files changed, 291 insertions(+) create mode 100644 crates/runtime-sidecar/Cargo.toml create mode 100644 crates/runtime-sidecar/README.md create mode 100644 crates/runtime-sidecar/src/lib.rs create mode 100644 crates/runtime-sidecar/src/main.rs diff --git a/Cargo.lock b/Cargo.lock index e6d83da5775..5b42938bfa8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7162,6 +7162,28 @@ dependencies = [ "zeroize", ] +[[package]] +name = "runtime-sidecar" +version = "0.0.0" +dependencies = [ + "anyhow", + "clap", + "flow-client-next", + "futures", + "gazette", + "proto-gazette", + "runtime-next", + "rustls 0.23.32", + "shuffle", + "tokens", + "tokio", + "tokio-stream", + "tonic", + "tracing", + "tracing-subscriber", + "url", +] + [[package]] name = "rusqlite" version = "0.32.1" diff --git a/crates/runtime-sidecar/Cargo.toml b/crates/runtime-sidecar/Cargo.toml new file mode 100644 index 00000000000..82475d328df --- /dev/null +++ b/crates/runtime-sidecar/Cargo.toml @@ -0,0 +1,32 @@ +[package] +name = "runtime-sidecar" +version.workspace = true +rust-version.workspace = true +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true + +[[bin]] +name = "runtime-sidecar" +path = "src/main.rs" + +[dependencies] +flow-client-next = { path = "../flow-client-next" } +gazette = { path = "../gazette" } +proto-gazette = { path = "../proto-gazette" } +runtime-next = { path = "../runtime-next" } +shuffle = { path = "../shuffle" } +tokens = { path = "../tokens" } + +anyhow = { workspace = true } +clap = { workspace = true } +futures = { workspace = true } +rustls = { workspace = true } +tokio = { workspace = true } +tokio-stream = { workspace = true } +tonic = { workspace = true, features = ["tls-aws-lc"] } +tracing = { workspace = true } +tracing-subscriber = { workspace = true } +url = { workspace = true } diff --git a/crates/runtime-sidecar/README.md b/crates/runtime-sidecar/README.md new file mode 100644 index 00000000000..aaee123c4b5 --- /dev/null +++ b/crates/runtime-sidecar/README.md @@ -0,0 +1,23 @@ +# runtime-sidecar + +Production sidecar process for the runtime-v2 architecture +(`plans/runtime-v2/plan.md`). One per reactor machine, supervised by +systemd, hosting two gRPC services on a fixed fleet-wide port: + +- **Shuffle Leader** — `runtime_next::leader::Service`, the per-task + Join rendezvous and HeadFSM/TailFSM coordination for tasks whose + shard zero is on this machine. +- **Shuffle** — `shuffle::Service`, the Session/Slice/Log RPCs. + +## Listeners + +`--listen-port` binds a TCP listener at `[::]:`. TLS is on if +`--certificate-file` and `--certificate-key-file` are both provided. + +## Auth + +`--data-plane-auth-keys` is whitespace- or comma-separated base64 +HMAC keys, matching gazette's `auth-keys` semantics. The first key +signs outgoing `/authorize/task` requests issued by the leader and +shuffle to obtain Gazette journal tokens. (Incoming-gRPC verification +against the full key list is wired in a follow-up change.) diff --git a/crates/runtime-sidecar/src/lib.rs b/crates/runtime-sidecar/src/lib.rs new file mode 100644 index 00000000000..e8d2a086409 --- /dev/null +++ b/crates/runtime-sidecar/src/lib.rs @@ -0,0 +1,167 @@ +use anyhow::Context; +use clap::Parser; +use std::path::PathBuf; + +/// Command-line arguments for the runtime-sidecar process. +/// +/// Naming aligns with sibling Rust services (`dekaf`) for unprefixed +/// `DATA_PLANE_*`, `CERTIFICATE_*`, and `AGENT_ENDPOINT` envs. The +/// reactor (Go consumer) uses `FLOW_*` and `CONSUMER_*` namespaced +/// envs because that's `go-flags`/gazette `mainboilerplate` convention; +/// we follow the unprefixed form here. +#[derive(Debug, Parser)] +#[command(about, version)] +pub struct Args { + #[arg(long = "log-format", env = "LOG_FORMAT", default_value = "text")] + pub log_format: LogFormat, + + /// TCP port to listen on, binding `[::]:`. + #[arg(long, env = "LISTEN_PORT")] + pub listen_port: u16, + + /// Externally-reachable URL of this sidecar, advertised to peer + /// shuffle clients (e.g. `https://reactor-foo.flow.localhost:9100`). + #[arg(long, env = "PEER_ENDPOINT")] + pub peer_endpoint: String, + + /// Fully-qualified domain name of the data-plane that this sidecar + /// belongs to; used as the issuer claim of authorization tokens. + #[arg(long, env = "DATA_PLANE_FQDN")] + pub data_plane_fqdn: String, + + /// Whitespace- or comma-separated base64 HMAC keys recognized by + /// the data plane. The first key signs outgoing `/authorize/task` + /// requests; all keys are accepted as verifiers for incoming + /// gRPC traffic (incoming verification is wired in a follow-up). + #[arg(long, env = "DATA_PLANE_AUTH_KEYS")] + pub data_plane_auth_keys: String, + + /// TLS server certificate PEM. Both `--certificate-file` and + /// `--certificate-key-file` must be provided together. + #[arg(long, env = "CERTIFICATE_FILE", requires = "certificate_key_file")] + pub certificate_file: Option, + + /// TLS server private key PEM. Required iff `--certificate-file` is set. + #[arg(long, env = "CERTIFICATE_KEY_FILE", requires = "certificate_file")] + pub certificate_key_file: Option, + + /// Estuary agent REST base URL used to issue `/authorize/task` calls. + #[arg(long, env = "AGENT_ENDPOINT")] + pub agent_endpoint: url::Url, + + /// Broker zone passed to `gazette::Router::new`. + #[arg(long, env = "GAZETTE_ZONE", default_value = "local")] + pub gazette_zone: String, + + /// On-disk shuffle log overflow threshold in bytes. Default is 2 GiB. + #[arg(long, env = "DISK_BACKLOG_THRESHOLD", default_value_t = 2 * 1024 * 1024 * 1024)] + pub disk_backlog_threshold: u64, +} + +#[derive(Debug, Clone, Copy, PartialEq, clap::ValueEnum)] +pub enum LogFormat { + Text, + Json, +} + +pub async fn run(args: Args) -> anyhow::Result<()> { + // Parse comma/whitespace-separated base64 HMAC keys. First key signs + // outgoing /authorize/task requests; the remaining keys are reserved + // for future incoming-gRPC verification (see plan). + let keys: Vec = args + .data_plane_auth_keys + .split(|c: char| c == ',' || c.is_whitespace()) + .filter(|s| !s.is_empty()) + .map(str::to_owned) + .collect(); + anyhow::ensure!( + !keys.is_empty(), + "--data-plane-auth-keys must contain at least one key" + ); + let signing_key = tokens::jwt::EncodingKey::from_base64_secret(&keys[0]) + .context("parsing first data-plane auth key (base64)")?; + let _verification_keys = keys; // TODO(runtime-v2): wire up incoming-gRPC verification. + + // Build REST + Router. + let api_client = flow_client_next::rest::Client::new(&args.agent_endpoint, "runtime-sidecar"); + let router = gazette::Router::new(&args.gazette_zone); + + // Two journal client factories: LIST|READ (shuffle reads source journals) + // and LIST|APPEND (leader publishes stats and ACK intents). + let read_factory = + flow_client_next::workflows::task_collection_auth::new_journal_client_factory( + api_client.clone(), + proto_gazette::capability::LIST | proto_gazette::capability::READ, + router.clone(), + args.data_plane_fqdn.clone(), + signing_key.clone(), + ); + let append_factory = + flow_client_next::workflows::task_collection_auth::new_journal_client_factory( + api_client, + proto_gazette::capability::LIST | proto_gazette::capability::APPEND, + router, + args.data_plane_fqdn, + signing_key, + ); + + let shuffle_svc = shuffle::Service::new( + args.peer_endpoint, + read_factory, + args.disk_backlog_threshold, + ); + let runtime_svc = runtime_next::Service::new(shuffle_svc.clone(), append_factory); + + // Build a TLS identity if both files were given. + // clap `requires` enforces both-or-neither. + let tls_identity = if let (Some(cert), Some(key)) = ( + args.certificate_file.as_ref(), + args.certificate_key_file.as_ref(), + ) { + let cert_bytes = tokio::fs::read(cert) + .await + .with_context(|| format!("reading {}", cert.display()))?; + let key_bytes = tokio::fs::read(key) + .await + .with_context(|| format!("reading {}", key.display()))?; + Some(tonic::transport::Identity::from_pem(cert_bytes, key_bytes)) + } else { + None + }; + + // SIGTERM (systemd) and SIGINT (interactive Ctrl+C) both initiate graceful shutdown. + let (shutdown_tx, mut shutdown_rx) = tokio::sync::broadcast::channel::<()>(1); + tokio::spawn(async move { + let mut term = tokio::signal::unix::signal(tokio::signal::unix::SignalKind::terminate()) + .expect("install SIGTERM handler"); + tokio::select! { + _ = term.recv() => tracing::info!("SIGTERM received"), + _ = tokio::signal::ctrl_c() => tracing::info!("SIGINT received"), + } + let _ = shutdown_tx.send(()); + }); + + let addr = format!("[::]:{}", args.listen_port); + let tcp = tokio::net::TcpListener::bind(&addr) + .await + .with_context(|| format!("binding TCP {addr}"))?; + tracing::info!(%addr, tls = tls_identity.is_some(), "runtime-sidecar listening on TCP"); + + let mut builder = tonic::transport::Server::builder(); + if let Some(identity) = tls_identity { + builder = builder + .tls_config(tonic::transport::ServerTlsConfig::new().identity(identity)) + .context("configuring TCP TLS")?; + } + builder + .add_service(runtime_svc.into_tonic_service()) + .add_service(shuffle_svc.into_tonic_service()) + .serve_with_incoming_shutdown( + tokio_stream::wrappers::TcpListenerStream::new(tcp), + async move { + let _ = shutdown_rx.recv().await; + }, + ) + .await + .context("serving runtime-sidecar TCP") +} diff --git a/crates/runtime-sidecar/src/main.rs b/crates/runtime-sidecar/src/main.rs new file mode 100644 index 00000000000..0ac98555743 --- /dev/null +++ b/crates/runtime-sidecar/src/main.rs @@ -0,0 +1,47 @@ +use clap::Parser; + +fn main() -> Result<(), anyhow::Error> { + // Required for libraries that use rustls (tonic TLS, gazette client TLS). + // See https://docs.rs/rustls/latest/rustls/crypto/struct.CryptoProvider.html + rustls::crypto::aws_lc_rs::default_provider() + .install_default() + .expect("failed to install default crypto provider"); + + let args = runtime_sidecar::Args::parse(); + install_tracing(args.log_format); + + let runtime = tokio::runtime::Builder::new_multi_thread() + .enable_all() + .build()?; + + let result = runtime.block_on(runtime.spawn(runtime_sidecar::run(args))); + runtime.shutdown_timeout(std::time::Duration::from_secs(5)); + result? +} + +/// Install a tracing subscriber that writes structured application logs to +/// stderr. The plan mandates application-style logs only, so we do not +/// install `ops::tracing::Layer`. +fn install_tracing(log_format: runtime_sidecar::LogFormat) { + let env_filter = tracing_subscriber::EnvFilter::try_from_default_env() + .unwrap_or_else(|_| tracing_subscriber::EnvFilter::new("info")); + + if log_format == runtime_sidecar::LogFormat::Json { + let subscriber = tracing_subscriber::fmt() + .with_env_filter(env_filter) + .with_writer(std::io::stderr) + .json() + .finish(); + tracing::subscriber::set_global_default(subscriber) + .expect("setting tracing default failed"); + } else { + let no_color = matches!(std::env::var("NO_COLOR"), Ok(v) if v == "1"); + let subscriber = tracing_subscriber::fmt() + .with_env_filter(env_filter) + .with_writer(std::io::stderr) + .with_ansi(!no_color) + .finish(); + tracing::subscriber::set_global_default(subscriber) + .expect("setting tracing default failed"); + } +} From 080437ff54c7fc36adb74a787ad48908455aa18c Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 21:58:08 +0000 Subject: [PATCH 10/22] go/runtime: wire up flagged V2 materialization app Implement the "controller" portion of the V2 runtime, which initiates the shard RPC lifecycle and drives the Join/Joined => Opened sequence. The new runtime is selected only for tasks having the `enable-runtime-v2` feature flag. Also add a new --shuffle-port flag, used to generate accessible endpoints for the sidecar of a given reactor. --- Cargo.lock | 1 + crates/bindings/Cargo.toml | 1 + crates/bindings/flow_bindings.h | 15 + crates/bindings/src/lib.rs | 1 + crates/bindings/src/task_service_v2.rs | 83 +++++ crates/runtime-next/src/task_service.rs | 2 +- go/bindings/build_test.go | 15 + go/bindings/task_service_v2.go | 162 +++++++++ go/protocols/catalog/build_load.go | 11 + go/runtime/flow_consumer.go | 36 +- go/runtime/flow_consumer_test.go | 57 +++ go/runtime/materialize_v2.go | 438 ++++++++++++++++++++++++ go/runtime/task.go | 60 +++- 13 files changed, 877 insertions(+), 5 deletions(-) create mode 100644 crates/bindings/src/task_service_v2.rs create mode 100644 go/bindings/task_service_v2.go create mode 100644 go/runtime/materialize_v2.go diff --git a/Cargo.lock b/Cargo.lock index 5b42938bfa8..0a928d9ec38 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1412,6 +1412,7 @@ dependencies = [ "prost", "proto-flow", "runtime", + "runtime-next", "serde", "serde_json", "thiserror 2.0.17", diff --git a/crates/bindings/Cargo.toml b/crates/bindings/Cargo.toml index ea67f81b669..0e21d9e49a9 100644 --- a/crates/bindings/Cargo.toml +++ b/crates/bindings/Cargo.toml @@ -18,6 +18,7 @@ derive = { path = "../derive" } ops = { path = "../ops" } proto-flow = { path = "../proto-flow" } runtime = { path = "../runtime" } +runtime-next = { path = "../runtime-next" } anyhow = { workspace = true } bytes = { workspace = true } diff --git a/crates/bindings/flow_bindings.h b/crates/bindings/flow_bindings.h index a395842f506..ccd2af6d1ea 100644 --- a/crates/bindings/flow_bindings.h +++ b/crates/bindings/flow_bindings.h @@ -112,6 +112,17 @@ typedef struct TaskService { uintptr_t err_cap; } TaskService; +typedef struct TaskServiceV2ImplPtr { + uint8_t _private[0]; +} TaskServiceV2ImplPtr; + +typedef struct TaskServiceV2 { + struct TaskServiceV2ImplPtr *svc_ptr; + uint8_t *err_ptr; + uintptr_t err_len; + uintptr_t err_cap; +} TaskServiceV2; + struct Channel *extract_create(int32_t log_level, int32_t log_dest_fd); void extract_invoke1(struct Channel *ch, struct In1 i); @@ -131,6 +142,10 @@ struct TaskService *new_task_service(const uint8_t *config_ptr, uint32_t config_ void task_service_drop(struct TaskService *svc); +struct TaskServiceV2 *new_task_service_v2(const uint8_t *config_ptr, uint32_t config_len); + +void task_service_v2_drop(struct TaskServiceV2 *svc); + struct Channel *upper_case_create(int32_t log_level, int32_t log_dest_fd); void upper_case_invoke1(struct Channel *ch, struct In1 i); diff --git a/crates/bindings/src/lib.rs b/crates/bindings/src/lib.rs index cadaba0747a..af6336dd159 100644 --- a/crates/bindings/src/lib.rs +++ b/crates/bindings/src/lib.rs @@ -2,4 +2,5 @@ mod extract; mod metrics; mod service; mod task_service; +mod task_service_v2; mod upper_case; diff --git a/crates/bindings/src/task_service_v2.rs b/crates/bindings/src/task_service_v2.rs new file mode 100644 index 00000000000..fb5708eb6e6 --- /dev/null +++ b/crates/bindings/src/task_service_v2.rs @@ -0,0 +1,83 @@ +use prost::Message; + +// Opaque pointer for a TaskService instance in the ABI. +#[repr(C)] +pub struct TaskServiceV2ImplPtr { + _private: [u8; 0], +} + +#[repr(C)] +pub struct TaskServiceV2 { + // Opaque service pointer. + svc_ptr: *mut TaskServiceV2ImplPtr, + + // Terminal error returned by the TaskService. + err_ptr: *mut u8, + err_len: usize, + err_cap: usize, +} + +#[unsafe(no_mangle)] +pub extern "C" fn new_task_service_v2( + config_ptr: *const u8, + config_len: u32, +) -> *mut TaskServiceV2 { + let config = unsafe { std::slice::from_raw_parts(config_ptr, config_len as usize) }; + let config = proto_flow::runtime::TaskServiceConfig::decode(config).unwrap(); + + let log_file = unsafe { + use std::os::unix::io::FromRawFd; + std::fs::File::from_raw_fd(config.log_file_fd) + }; + + let svc_abi = match runtime_next::TaskService::new(config, log_file) { + Ok(svc) => { + let svc_ptr = Box::leak(Box::new(svc)) as *mut runtime_next::TaskService + as *mut TaskServiceV2ImplPtr; + + TaskServiceV2 { + svc_ptr, + err_ptr: 0 as *mut u8, + err_len: 0, + err_cap: 0, + } + } + Err(err) => { + let mut err = format!("{:?}", err); + let err_ptr = err.as_mut_ptr(); + let err_cap = err.capacity(); + let err_len = err.len(); + std::mem::forget(err); + + TaskServiceV2 { + svc_ptr: 0 as *mut TaskServiceV2ImplPtr, + err_ptr, + err_len, + err_cap, + } + } + }; + + Box::leak(Box::new(svc_abi)) +} + +#[unsafe(no_mangle)] +pub extern "C" fn task_service_v2_drop(svc: *mut TaskServiceV2) { + let TaskServiceV2 { + svc_ptr, + err_ptr, + err_len, + err_cap, + } = *unsafe { Box::from_raw(svc) }; + + if svc_ptr != 0 as *mut TaskServiceV2ImplPtr { + let svc = unsafe { Box::from_raw(svc_ptr as *mut runtime_next::TaskService) }; + svc.graceful_stop(); + } + let err_ptr = if err_cap == 0 { + std::ptr::NonNull::dangling().as_ptr() + } else { + err_ptr + }; + unsafe { String::from_raw_parts(err_ptr, err_len, err_cap) }; +} diff --git a/crates/runtime-next/src/task_service.rs b/crates/runtime-next/src/task_service.rs index bdddee39797..b2be5214e4a 100644 --- a/crates/runtime-next/src/task_service.rs +++ b/crates/runtime-next/src/task_service.rs @@ -31,7 +31,7 @@ impl TaskService { 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 availability_zone = std::env::var("CONSUMER_ZONE").context("CONSUMER_ZONE not set")?; let data_plane_signing_key = first_consumer_auth_key()?; let log_handler = ::ops::new_encoded_json_write_handler(std::sync::Arc::new( diff --git a/go/bindings/build_test.go b/go/bindings/build_test.go index 035d8f2ccaf..737ba9ea412 100644 --- a/go/bindings/build_test.go +++ b/go/bindings/build_test.go @@ -75,6 +75,21 @@ func TestBuildCatalog(t *testing.T) { require.NoError(t, err) cupaloy.SnapshotT(t, out) }) + t.Run("collection-for-journal", func(t *testing.T) { + // Prefix match: a partitioned journal of `a/collection`. + out, err := catalog.LoadCollectionForJournal(db, + "a/collection/ffffffffffffffff/foo=bar/pivot=00") + require.NoError(t, err) + require.Equal(t, "a/collection", out.Name.String()) + + // Exact-match without trailing partition path is not supported. + _, err = catalog.LoadCollectionForJournal(db, "a/collection") + require.ErrorIs(t, err, sql.ErrNoRows) + + // No prefix match. + _, err = catalog.LoadCollectionForJournal(db, "no/such/journal") + require.ErrorIs(t, err, sql.ErrNoRows) + }) return nil })) diff --git a/go/bindings/task_service_v2.go b/go/bindings/task_service_v2.go new file mode 100644 index 00000000000..1c77c76978c --- /dev/null +++ b/go/bindings/task_service_v2.go @@ -0,0 +1,162 @@ +package bindings + +// #include "../../crates/bindings/flow_bindings.h" +import "C" +import ( + "context" + "errors" + "fmt" + "io" + "os" + "os/signal" + "reflect" + "syscall" + "unsafe" + + "github.com/estuary/flow/go/protocols/ops" + pr "github.com/estuary/flow/go/protocols/runtime" + grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus" + "github.com/sirupsen/logrus" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" +) + +// TaskServiceV2 wraps the runtime-next CGO `TaskService`. Its lifecycle and +// pipe-FD log plumbing mirrors `TaskService` exactly; the symbols differ so +// the two runtimes can be linked side-by-side during the V2 rollout. +type TaskServiceV2 struct { + config pr.TaskServiceConfig + cSvc *C.TaskServiceV2 + conn *grpc.ClientConn + lwaCh <-chan struct{} +} + +func NewTaskServiceV2( + config pr.TaskServiceConfig, + logHandler func(ops.Log), +) (*TaskServiceV2, error) { + + // We must ignore SIGPIPE! + // + // This implementation uses a graceful shutdown where it will block on Drop + // waiting for all client RPCs to complete. HOWEVER, Rust's hyper crate will + // *NOT* wait for the complete shutdown of any underlying transports, + // and will immediately close their descriptors. This can cause EPIPE errors + // when the HTTP/2 transport coroutines attempt reads or writes over those + // transports -- both from Go and also from Rust. + // + // So, we must mask SIGPIPE so that these become EPIPE errno results, which + // both Go and Rust handle reasonably. + signal.Ignore(syscall.SIGPIPE) + + var logReader, wDescriptor, err = Pipe() + if err != nil { + return nil, fmt.Errorf("creating logging pipe: %w", err) + } + config.LogFileFd = int32(wDescriptor) + + // Log so that we can know the association between tasks and log file descriptors, + // for troubleshooting issues with missing logs. + logrus.WithFields(logrus.Fields{ + "task_name": config.TaskName, + "log_fd_write": wDescriptor, + "log_fd_read": logReader.Fd(), + }).Info("created new pipe for task logs") + + // Rust services produce canonical JSON encodings of ops::Log into `wDescriptor`. + // Parse each and pass to our `publisher`. + var lwaCh = make(chan struct{}) + go func() { + defer close(lwaCh) + var _, err = io.Copy(ops.NewLogWriteAdapter(logHandler), logReader) + if err != nil { + logrus.WithFields(logrus.Fields{ + "error": err, + "service": config.TaskName, + }).Error("failed to process cgo V2 service channel logs") + } + }() + + udsFile, err := os.CreateTemp("", "run-sock") + if err != nil { + return nil, fmt.Errorf("creating task service socket file: %w", err) + } + config.UdsPath = udsFile.Name() + + // Rust service will re-create the path. + if err = udsFile.Close(); err != nil { + return nil, fmt.Errorf("closing task service socket file: %w", err) + } + if err = os.Remove(config.UdsPath); err != nil { + return nil, fmt.Errorf("removing task service socket file: %w", err) + } + + configBytes, err := config.Marshal() + if err != nil { + return nil, err + } + var h = (*reflect.SliceHeader)(unsafe.Pointer(&configBytes)) + + var svc = &TaskServiceV2{ + config: config, + cSvc: C.new_task_service_v2( + (*C.uint8_t)(unsafe.Pointer(h.Data)), + C.uint32_t(h.Len), + ), + lwaCh: lwaCh, + } + + if err := svc.err(); err != nil { + svc.Drop() + return nil, err + } + + svc.conn, err = grpc.DialContext( + context.Background(), + "unix://"+config.UdsPath, + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithUnaryInterceptor(grpc_prometheus.UnaryClientInterceptor), + grpc.WithStreamInterceptor(grpc_prometheus.StreamClientInterceptor), + grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(maxMessageSize), grpc.MaxCallSendMsgSize(maxMessageSize)), + ) + + if err != nil { + svc.Drop() + return nil, err + } + + return svc, nil +} + +func (s *TaskServiceV2) Conn() *grpc.ClientConn { + return s.conn +} + +func (s *TaskServiceV2) Drop() { + if s.conn != nil { + _ = s.conn.Close() + s.conn = nil + } + if s.cSvc != nil { + C.task_service_v2_drop(s.cSvc) + s.cSvc = nil + } + if s.lwaCh != nil { + // Block until log read loop reads error or EOF. + // This happens only after all Rust references of the Pipe have been + // dropped and the descriptor has been closed. + <-s.lwaCh + s.lwaCh = nil + } + _ = os.Remove(s.config.UdsPath) // Best effort. +} + +func (s *TaskServiceV2) err() error { + var err error + if s.cSvc.err_len != 0 { + err = errors.New(C.GoStringN( + (*C.char)(unsafe.Pointer(s.cSvc.err_ptr)), + C.int(s.cSvc.err_len))) + } + return err +} diff --git a/go/protocols/catalog/build_load.go b/go/protocols/catalog/build_load.go index ae6cdb6d404..82475448085 100644 --- a/go/protocols/catalog/build_load.go +++ b/go/protocols/catalog/build_load.go @@ -81,6 +81,17 @@ func LoadCollection(db *sql.DB, name string) (*pf.CollectionSpec, error) { return out, loadOneSpec(db, `SELECT spec FROM built_collections WHERE collection = ?;`, out, name) } +// LoadCollectionForJournal loads the CollectionSpec whose name is the +// slash-delimited prefix of `journal`. +func LoadCollectionForJournal(db *sql.DB, journal string) (*pf.CollectionSpec, error) { + var out = new(pf.CollectionSpec) + return out, loadOneSpec(db, + `SELECT spec FROM built_collections + WHERE substr(?, 1, length(collection) + 1) = collection || '/' + ORDER BY length(collection) DESC LIMIT 1;`, + out, journal) +} + // LoadAllCaptures loads all captures. func LoadAllCaptures(db *sql.DB) ([]*pf.CaptureSpec, error) { var out []*pf.CaptureSpec diff --git a/go/runtime/flow_consumer.go b/go/runtime/flow_consumer.go index 3b81d1f3da9..cf0e3169bed 100644 --- a/go/runtime/flow_consumer.go +++ b/go/runtime/flow_consumer.go @@ -42,6 +42,7 @@ type FlowConsumerConfig struct { DataPlaneFQDN string `long:"data-plane-fqdn" env:"DATA_PLANE_FQDN" description:"Fully-qualified domain name of the data-plane to which this reactor belongs"` Network string `long:"network" env:"NETWORK" description:"The Docker network that connector containers are given access to. Defaults to the bridge network"` ProxyRuntimes int `long:"proxy-runtimes" default:"2" description:"The number of proxy connector runtimes that may run concurrently"` + SidecarPort uint16 `long:"sidecar-port" env:"SIDECAR_PORT" description:"Port of the runtime-sidecar co-located on every reactor machine (fleet-wide, fixed). Required when any task uses the runtime-v2 feature flag."` TestAPIs bool `long:"test-apis" description:"Enable APIs exclusively used while running catalog tests"` } `group:"flow" namespace:"flow" env-namespace:"FLOW"` } @@ -58,6 +59,19 @@ func (c *FlowConsumerConfig) Execute(args []string) error { }.Execute(args) } +// SidecarEndpoint maps a reactor's `ProcessSpec.Endpoint` to the dial-able URL +// of the runtime-sidecar co-located with that reactor. The sidecar listens on a +// fixed fleet-wide port (--sidecar-port); we preserve the reactor URL's scheme +// (TLS choice mirrors the reactor) and substitute the host's port. +func (c *FlowConsumerConfig) SidecarEndpoint(reactor pb.Endpoint) (string, error) { + if c.Flow.SidecarPort == 0 { + return "", fmt.Errorf("--sidecar-port (or env SIDECAR_PORT) is required for runtime-v2 tasks") + } + var u = reactor.URL() + u.Host = fmt.Sprintf("%s:%d", u.Hostname(), c.Flow.SidecarPort) + return u.String(), nil +} + // Plane returns the data-plane context in which this FlowConsumerConfig is running. func (c *FlowConsumerConfig) Plane() pr.Plane { // AllowLocal flag is only passed in allowed local contexts. @@ -152,16 +166,32 @@ func (f *FlowConsumer) NewStore(shard consumer.Shard, rec *recoverylog.Recorder) return d, nil } case ops.TaskType_materialization.String(): - if m, err := newMaterializeApp(f, shard, rec); err != nil { - return nil, err + if useRuntimeV2(shard.Spec().LabelSet) { + if m, err := newMaterializeAppV2(f, shard, rec); err != nil { + return nil, err + } else { + return m, nil + } } else { - return m, nil + if m, err := newMaterializeApp(f, shard, rec); err != nil { + return nil, err + } else { + return m, nil + } } default: return nil, fmt.Errorf("don't know how to serve catalog task type %q", taskType) } } +// runtimeV2Flag is the shard-label flag selecting the runtime-next (V2) path. +const runtimeV2Flag = labels.FlagPrefix + "enable-runtime-v2" + +// useRuntimeV2 reports whether the shard's labels select the runtime-next path. +func useRuntimeV2(set pf.LabelSet) bool { + return set.ValueOf(runtimeV2Flag) == "true" +} + // NewMessage panics if called. func (f *FlowConsumer) NewMessage(*pb.JournalSpec) (message.Message, error) { panic("NewMessage is never called") diff --git a/go/runtime/flow_consumer_test.go b/go/runtime/flow_consumer_test.go index 742a9a6313d..8ab1c53182a 100644 --- a/go/runtime/flow_consumer_test.go +++ b/go/runtime/flow_consumer_test.go @@ -3,7 +3,10 @@ package runtime import ( "testing" + "github.com/estuary/flow/go/labels" + pf "github.com/estuary/flow/go/protocols/flow" pr "github.com/estuary/flow/go/protocols/runtime" + pb "go.gazette.dev/core/broker/protocol" "github.com/stretchr/testify/require" ) @@ -52,3 +55,57 @@ func TestFlowConsumerConfig_Plane(t *testing.T) { }) } } + +func TestUseRuntimeV2(t *testing.T) { + tests := []struct { + name string + labels []pb.Label + want bool + }{ + { + name: "no flag", + labels: []pb.Label{{Name: labels.TaskName, Value: "task"}}, + want: false, + }, + { + name: "enable-runtime-v2=true", + labels: []pb.Label{{Name: labels.FlagPrefix + "enable-runtime-v2", Value: "true"}}, + want: true, + }, + { + name: "enable-runtime-v2=false", + labels: []pb.Label{{Name: labels.FlagPrefix + "enable-runtime-v2", Value: "false"}}, + want: false, + }, + { + name: "unrelated flag", + labels: []pb.Label{{Name: labels.FlagPrefix + "some-other-flag", Value: "true"}}, + want: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + var set = pf.LabelSet{Labels: tt.labels} + require.Equal(t, tt.want, useRuntimeV2(set)) + }) + } +} + +func TestSidecarEndpoint(t *testing.T) { + var config = FlowConsumerConfig{} + config.Flow.SidecarPort = 9100 + + got, err := config.SidecarEndpoint(pb.Endpoint("https://reactor-foo.flow.localhost:8080")) + require.NoError(t, err) + require.Equal(t, "https://reactor-foo.flow.localhost:9100", got) + + got, err = config.SidecarEndpoint(pb.Endpoint("http://10.0.0.5:8080")) + require.NoError(t, err) + require.Equal(t, "http://10.0.0.5:9100", got) + + // No port configured: error. + config.Flow.SidecarPort = 0 + _, err = config.SidecarEndpoint(pb.Endpoint("https://reactor-foo.flow.localhost:8080")) + require.ErrorContains(t, err, "sidecar-port") +} diff --git a/go/runtime/materialize_v2.go b/go/runtime/materialize_v2.go new file mode 100644 index 00000000000..0ec106382e9 --- /dev/null +++ b/go/runtime/materialize_v2.go @@ -0,0 +1,438 @@ +package runtime + +import ( + "database/sql" + "fmt" + "io" + "os" + "strings" + + "github.com/estuary/flow/go/bindings" + "github.com/estuary/flow/go/flow" + "github.com/estuary/flow/go/labels" + "github.com/estuary/flow/go/protocols/catalog" + pf "github.com/estuary/flow/go/protocols/flow" + pr "github.com/estuary/flow/go/protocols/runtime" + "github.com/estuary/flow/go/shuffle" + "github.com/sirupsen/logrus" + "go.gazette.dev/core/allocator" + pb "go.gazette.dev/core/broker/protocol" + "go.gazette.dev/core/consumer" + pc "go.gazette.dev/core/consumer/protocol" + "go.gazette.dev/core/consumer/recoverylog" + "go.gazette.dev/core/message" +) + +// materializeAppV2 is the runtime-next backed materialization Application. +// It bypasses the Gazette transaction lifecycle entirely (drains +// StartReadingMessages, panics in BeginTxn / ConsumeMessage / FinalizeTxn / +// StartCommit) and instead drives the V2 protocol over the per-shard +// `Shard.Materialize` bidi stream. The Rust side handles all document I/O, +// shuffle, combining, publishing, and stat aggregation; the Go controller +// only manages session startup (Join → Joined → Task → Opened) and +// teardown (Stop → Stopped) per term. +type materializeAppV2 struct { + *taskBase[*pf.MaterializationSpec] + + client pr.Shard_MaterializeClient + // shuffleDir hosts per-shard shuffle files for this assignment. + shuffleDir string + + // respCh is fed by a long-lived pump goroutine on `m.client.Recv()`. + respCh <-chan recvResult +} + +// recvResult is one outcome of m.client.Recv(): either a Materialize +// response or a terminal error (io.EOF on graceful close). +type recvResult struct { + resp *pr.Materialize + err error +} + +var _ application = (*materializeAppV2)(nil) + +func newMaterializeAppV2(host *FlowConsumer, shard consumer.Shard, recorder *recoverylog.Recorder) (*materializeAppV2, error) { + if host.config.Flow.SidecarPort == 0 { + return nil, fmt.Errorf("runtime-v2 requires --sidecar-port (or env SIDECAR_PORT)") + } + + var shuffleDir, err = os.MkdirTemp("", "flow-runtime-v2-shuffle-") + if err != nil { + return nil, fmt.Errorf("creating runtime-v2 shuffle tempdir: %w", err) + } + + var base *taskBase[*pf.MaterializationSpec] + base, err = newTaskBaseV2[*pf.MaterializationSpec](host, shard, recorder, extractMaterializationSpec) + if err != nil { + _ = os.RemoveAll(shuffleDir) + return nil, err + } + go base.heartbeatLoop(shard) + + var client pr.Shard_MaterializeClient + client, err = pr.NewShardClient(base.svc.Conn()).Materialize(shard.Context()) + if err != nil { + base.drop() + _ = os.RemoveAll(shuffleDir) + return nil, fmt.Errorf("opening V2 Shard.Materialize stream: %w", err) + } + + // SessionLoop is the first message of the stream and lasts its lifetime. + // It carries the RocksDB handle that runtime-next opens and reuses across + // every leader session within this stream. + var rocksDBDescriptor *pr.RocksDBDescriptor + if recorder != nil { + rocksDBDescriptor = bindings.NewRocksDBDescriptor(recorder) + } + _ = client.Send(&pr.Materialize{ + SessionLoop: &pr.SessionLoop{ + RocksdbDescriptor: rocksDBDescriptor, + }, + }) + + var respCh = make(chan recvResult, 4) + go func() { + defer close(respCh) + for { + var resp, err = client.Recv() + // Bias towards send, but bail if full AND cancelled. + select { + case respCh <- recvResult{resp: resp, err: err}: + default: + select { + case respCh <- recvResult{resp: resp, err: err}: + case <-shard.Context().Done(): + return + } + } + if err != nil { + return + } + } + }() + + return &materializeAppV2{ + taskBase: base, + client: client, + shuffleDir: shuffleDir, + respCh: respCh, + }, nil +} + +func (m *materializeAppV2) recv() (*pr.Materialize, error) { + var r, ok = <-m.respCh + if !ok { + return nil, io.ErrUnexpectedEOF + } + return r.resp, r.err +} + +func (m *materializeAppV2) RestoreCheckpoint(shard consumer.Shard) (pf.Checkpoint, error) { + if err := shard.Context().Err(); err != nil { + return pf.Checkpoint{}, err + } else if err := m.initTerm(shard); err != nil { + m.term.cancel() + return pf.Checkpoint{}, err + } + // Note that RestoreCheckpoint is called at the transition from + // STANDBY to PRIMARY: the shard is updated to PRIMARY only after its + // return, and before StartReadingMessages. + + // The Rust runtime owns checkpoint persistence, recovery, ACK intent + // publishing, and the core transaction loop. Nothing to recover here. + return pf.Checkpoint{}, nil +} + +func (m *materializeAppV2) StartReadingMessages(shard consumer.Shard, _ pc.Checkpoint, _ *flow.Timepoint, ch chan<- consumer.EnvelopeOrError) { + go m.runOneSession(shard, ch) +} + +// runOneSession drives one session of the V2 protocol from end to end: +// +// 1. Load and marshal per-term task inputs. +// 2. Build & send Join once every task shard is PRIMARY-assigned, retrying +// on disagreement until consensus. +// 3. Send Task and wait for Opened (which carries the connector Container). +// 4. Run the steady-state select loop until term cancellation, shard +// cancellation, or a stream error. +// +// On term cancellation (spec update) we send Stop and wait for Stopped, then +// return nil so the framework loops back through RestoreCheckpoint. Any +// non-nil return is forwarded onto `ch` by the deferred sentinel before +// the channel is closed. +func (m *materializeAppV2) runOneSession(shard consumer.Shard, ch chan<- consumer.EnvelopeOrError) (err error) { + defer func() { + if err != nil { + ch <- consumer.EnvelopeOrError{Error: err} + } + close(ch) + }() + + // 1. Load and marshal everything that doesn't depend on Etcd topology. + var opsStatsJournal = m.term.labels.StatsJournal + var opsStatsSpec *pf.CollectionSpec + if opsStatsSpec, err = m.loadOpsCollectionSpec(opsStatsJournal); err != nil { + return err + } + var specBytes []byte + if specBytes, err = m.term.taskSpec.Marshal(); err != nil { + return fmt.Errorf("marshaling MaterializationSpec: %w", err) + } + + // 2. Build & send Join, repeating on disagreement until consensus. + var waitForRevision int64 + for { + var ks = m.host.service.State.KS + ks.Mu.RLock() + err = ks.WaitForRevision(m.term.ctx, waitForRevision) + ks.Mu.RUnlock() + if err != nil { + return fmt.Errorf("awaiting Etcd revision %d: %w", waitForRevision, err) + } + var join, rev, err = m.buildJoin() + if err != nil { + return fmt.Errorf("building Join: %w", err) + } + if join == nil { + waitForRevision = rev + 1 + continue + } + if err := m.client.Send(&pr.Materialize{Join: join}); err != nil { + return fmt.Errorf("sending Join: %w", err) + } + var resp *pr.Materialize + if resp, err = m.recv(); err != nil { + return fmt.Errorf("receiving Joined: %w", pf.UnwrapGRPCError(err)) + } + if resp.Joined == nil { + return fmt.Errorf("expected Joined, got %#v", resp) + } + if resp.Joined.MaxEtcdRevision == 0 { + break // Consensus. + } + // Disagreement: await the indicated revision, then re-poll topology. + waitForRevision = resp.Joined.MaxEtcdRevision + } + + // 3. Send Task and wait for Opened. + if err := m.client.Send(&pr.Materialize{ + Task: &pr.Task{ + Spec: specBytes, + OpsStatsJournal: string(opsStatsJournal), + OpsStatsSpec: opsStatsSpec, + Preview: false, + MaxTransactions: 0, + }, + }); err != nil { + return fmt.Errorf("sending Task: %w", err) + } + resp, err := m.recv() + if err != nil { + return fmt.Errorf("receiving Opened: %w", pf.UnwrapGRPCError(err)) + } + if resp.Opened == nil { + return fmt.Errorf("expected Opened, got %#v", resp) + } + m.container.Store(resp.Opened.Container) + + // 4. Steady-state: drive teardown signals and surface stream errors. + // termDone is nil-ed once we've sent Stop, so the case stops firing. + // Future CloseNow plumbing slots in as another case alongside termDone. + var termDone = m.term.ctx.Done() + for { + select { + case <-termDone: + // Spec update: initiate graceful drain. The leader replies with + // Stopped, read from `respCh` below. + _ = m.client.Send(&pr.Materialize{Stop: &pr.Stop{}}) + termDone = nil + + case <-shard.Context().Done(): + return shard.Context().Err() // Immediate, non-graceful shutdown. + + case r, ok := <-m.respCh: + if !ok || r.err == io.EOF { + return nil + } + if r.err != nil { + return pf.UnwrapGRPCError(r.err) + } + if r.resp.Stopped != nil { + return nil // Graceful drain complete. + } + logrus.WithFields(logrus.Fields{ + "shardId": shard.Spec().Id, + "msg": r.resp, + }).Panic("unexpected Rust runtime message after Opened") + } + } +} + +// loadOpsCollectionSpec resolves an ops CollectionSpec for `journal` from +// the current term's build DB. catalog.LoadCollectionForJournal inverts the +// `/=/...` shape produced by +// activate::ops_partition_spec via SQL prefix match against +// built_collections. +func (m *materializeAppV2) loadOpsCollectionSpec(journal pb.Journal) (*pf.CollectionSpec, error) { + var spec *pf.CollectionSpec + var build = m.host.builds.Open(m.term.labels.Build) + defer build.Close() + + if err := build.Extract(func(db *sql.DB) error { + var s, err = catalog.LoadCollectionForJournal(db, string(journal)) + if err != nil { + return err + } + spec = s + return nil + }); err != nil { + return nil, fmt.Errorf("loading ops CollectionSpec for %q: %w", journal, err) + } + return spec, nil +} + +func (m *materializeAppV2) Destroy() { + if m.client != nil { + _ = m.client.CloseSend() + m.client = nil + } + m.taskBase.drop() + m.taskBase.opsCancel() + _ = os.RemoveAll(m.shuffleDir) +} + +func (m *materializeAppV2) ReplayRange(_ consumer.Shard, _ pb.Journal, _, _ pb.Offset) message.Iterator { + panic("runtime-v2: ReplayRange unreachable (no Gazette message pipeline)") +} + +func (m *materializeAppV2) ReadThrough(_ pb.Offsets) (pb.Offsets, error) { + return pb.Offsets{}, nil +} + +func (m *materializeAppV2) BeginTxn(_ consumer.Shard) error { + panic("runtime-v2: BeginTxn unreachable (StartReadingMessages drains)") +} +func (m *materializeAppV2) ConsumeMessage(_ consumer.Shard, _ message.Envelope, _ *message.Publisher) error { + panic("runtime-v2: ConsumeMessage unreachable (StartReadingMessages drains)") +} +func (m *materializeAppV2) FinalizeTxn(_ consumer.Shard, _ *message.Publisher) error { + panic("runtime-v2: FinalizeTxn unreachable (StartReadingMessages drains)") +} +func (m *materializeAppV2) StartCommit(_ consumer.Shard, _ pf.Checkpoint, _ consumer.OpFutures) consumer.OpFuture { + panic("runtime-v2: StartCommit unreachable (StartReadingMessages drains)") +} +func (m *materializeAppV2) FinishedTxn(_ consumer.Shard, _ consumer.OpFuture) {} + +func (m *materializeAppV2) Coordinator() *shuffle.Coordinator { + panic("runtime-v2: Coordinator unreachable (no Go shuffle)") +} + +// buildJoin enumerates the task topology under read lock. If any shard does +// not yet have a PRIMARY assignment, it returns nil plus the observed Etcd +// revision so the caller can wait and try again. A non-nil Join describes every +// shard, sorted by ShardID (which matches the proto's ordering contract on +// (key_begin, r_clock_begin) ascending). +func (m *materializeAppV2) buildJoin() (*pr.Join, int64, error) { + var ks = m.host.service.State.KS + var itemPrefix = allocator.ItemKey(ks, taskShardPrefix(m.term.shardSpec.Id)) + + ks.Mu.RLock() + defer ks.Mu.RUnlock() + var state = m.host.service.State + var rev = ks.Header.Revision + + if state.LocalMemberInd == -1 { + return nil, rev, fmt.Errorf("local reactor member not present in Etcd state") + } + var selfEndpoint = state.Members[state.LocalMemberInd]. + Decoded.(allocator.Member).MemberValue.(*pc.ConsumerSpec).Endpoint + var selfSidecar, err = m.host.config.SidecarEndpoint(selfEndpoint) + if err != nil { + return nil, rev, err + } + + var ( + shards []*pr.Join_Shard + shardIdx = -1 + ) + for i, kv := range state.Items.Prefixed(itemPrefix) { + var spec = kv.Decoded.(allocator.Item).ItemValue.(*pc.ShardSpec) + var asn, createRev, ok = primaryAssignment(state, spec.Id) + if !ok { + return nil, rev, nil + } + var labeling, err = labels.ParseShardLabels(spec.LabelSet) + if err != nil { + return nil, rev, fmt.Errorf("parsing labels for %s: %w", spec.Id, err) + } + shards = append(shards, &pr.Join_Shard{ + Id: spec.Id.String(), + Labeling: &labeling, + Reactor: &pb.ProcessSpec_ID{Zone: asn.MemberZone, Suffix: asn.MemberSuffix}, + EtcdCreateRevision: createRev, + }) + if spec.Id == m.term.shardSpec.Id { + shardIdx = i + } + } + if len(shards) == 0 { + return nil, rev, nil + } + if shardIdx < 0 { + return nil, rev, fmt.Errorf("local shard %s not in topology", m.term.shardSpec.Id) + } + + var leaderEndpoint, ok = memberEndpoint(state, *shards[0].Reactor) + if !ok { + return nil, rev, fmt.Errorf("leader reactor %+v not present in Etcd state", shards[0].Reactor) + } + var leaderSidecar string + if leaderSidecar, err = m.host.config.SidecarEndpoint(leaderEndpoint); err != nil { + return nil, rev, err + } + + return &pr.Join{ + EtcdModRevision: rev, + Shards: shards, + ShardIndex: uint32(shardIdx), + ShuffleDirectory: m.shuffleDir, + ShuffleEndpoint: selfSidecar, + LeaderEndpoint: leaderSidecar, + }, rev, nil +} + +// taskShardPrefix returns the keyspace directory under which all of the +// task's shard items live. ShardIDs are `//`; +// the prefix is everything up to and including the last `/`. +func taskShardPrefix(id pc.ShardID) string { + var s = id.String() + return s[:strings.LastIndexByte(s, '/')+1] +} + +// memberEndpoint returns the ConsumerSpec endpoint for `id`. Caller must hold +// `state.KS.Mu` for read. +func memberEndpoint(state *allocator.State, id pb.ProcessSpec_ID) (pb.Endpoint, bool) { + for _, kv := range state.Members { + var member = kv.Decoded.(allocator.Member) + if member.Zone != id.Zone || member.Suffix != id.Suffix { + continue + } + var spec = member.MemberValue.(*pc.ConsumerSpec) + return spec.Endpoint, true + } + return "", false +} + +// primaryAssignment returns the slot-0 PRIMARY allocator.Assignment for `id` +// and its Etcd CreateRevision. Caller must hold `state.KS.Mu` for read. +func primaryAssignment(state *allocator.State, id pc.ShardID) (allocator.Assignment, int64, bool) { + for _, akv := range state.KS.KeyValues.Prefixed( + allocator.ItemAssignmentsPrefix(state.KS, id.String())) { + var asn = akv.Decoded.(allocator.Assignment) + var status = asn.AssignmentValue.(*pc.ReplicaStatus) + if asn.Slot == 0 && status.Code == pc.ReplicaStatus_PRIMARY { + return asn, akv.Raw.CreateRevision, true + } + } + return allocator.Assignment{}, 0, false +} diff --git a/go/runtime/task.go b/go/runtime/task.go index 2095701aaa1..ec63e8f62c1 100644 --- a/go/runtime/task.go +++ b/go/runtime/task.go @@ -30,8 +30,16 @@ import ( "go.gazette.dev/core/consumer/recoverylog" "go.gazette.dev/core/keyspace" "go.gazette.dev/core/message" + "google.golang.org/grpc" ) +// taskService is the subset of bindings.TaskService / bindings.TaskServiceV2 +// behavior consumed by taskBase, so a single struct can host either runtime. +type taskService interface { + Conn() *grpc.ClientConn + Drop() +} + type taskBase[TaskSpec pf.Task] struct { container atomic.Pointer[pr.Container] // Current Container of this shard, or nil. extractFn func(*sql.DB, string) (TaskSpec, error) // Extracts a TaskSpec from a build DB. @@ -39,7 +47,7 @@ type taskBase[TaskSpec pf.Task] struct { opsCancel context.CancelFunc // Cancels ops.Publisher context. opsPublisher *OpsPublisher // ops.Publisher of task ops.Logs and ops.Stats. recorder *recoverylog.Recorder // Recorder of the shard's recovery log. - svc *bindings.TaskService // Associated Rust runtime service. + svc taskService // Associated Rust runtime service (legacy or V2). term taskTerm[TaskSpec] // Current task term. termCount int // Number of initialized task terms. } @@ -75,6 +83,7 @@ func newTaskBase[TaskSpec pf.Task]( term, err := newTaskTerm[TaskSpec](nil, extractFn, host, opsPublisher, shard) if err != nil { + opsCancel() return nil, err } @@ -88,6 +97,7 @@ func newTaskBase[TaskSpec pf.Task]( opsPublisher.PublishLog, ) if err != nil { + opsCancel() return nil, fmt.Errorf("creating task service: %w", err) } @@ -103,6 +113,54 @@ func newTaskBase[TaskSpec pf.Task]( }, nil } +// newTaskBaseV2 mirrors newTaskBase but instantiates the runtime-next +// (V2) TaskService. The legacy and V2 services are independently linked +// CGO services with distinct gRPC surfaces; only the constructor differs. +func newTaskBaseV2[TaskSpec pf.Task]( + host *FlowConsumer, + shard consumer.Shard, + recorder *recoverylog.Recorder, + extractFn func(*sql.DB, string) (TaskSpec, error), +) (*taskBase[TaskSpec], error) { + + var opsCtx, opsCancel = context.WithCancel(host.opsContext) + opsCtx = pprof.WithLabels(opsCtx, pprof.Labels( + "shard", shard.Spec().Id.String(), + )) + var opsPublisher = NewOpsPublisher(message.NewPublisher( + client.NewAppendService(opsCtx, host.service.Journals), nil)) + + term, err := newTaskTerm[TaskSpec](nil, extractFn, host, opsPublisher, shard) + if err != nil { + opsCancel() + return nil, err + } + + svc, err := bindings.NewTaskServiceV2( + pr.TaskServiceConfig{ + ContainerNetwork: host.config.Flow.Network, + TaskName: term.labels.TaskName, + Plane: host.config.Plane(), + }, + opsPublisher.PublishLog, + ) + if err != nil { + opsCancel() + return nil, fmt.Errorf("creating V2 task service: %w", err) + } + + return &taskBase[TaskSpec]{ + container: atomic.Pointer[pr.Container]{}, + extractFn: extractFn, + host: host, + opsCancel: opsCancel, + opsPublisher: opsPublisher, + recorder: recorder, + svc: svc, + term: *term, + }, nil +} + func (t *taskBase[TaskSpec]) initTerm(shard consumer.Shard) error { var next, err = newTaskTerm[TaskSpec](&t.term, t.extractFn, t.host, t.opsPublisher, shard) if err != nil { From 9990ab507e9bb71092147d15e54514fdf0efbac5 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 22:00:30 +0000 Subject: [PATCH 11/22] publisher: split Binding into Mapped and Fixed variants Add a Fixed binding that targets a single pre-existing journal by name, distinct from Mapped bindings which dynamically resolve documents to a collection's partitions (creating them on demand). Use Fixed for the ops stats journal, which activate pre-creates and which never needs partition mapping. This lets the runtime drop ops_stats_spec from the Task proto and removes catalog.LoadCollectionForJournal along with its Go caller, both of which existed only to recover the ops stats CollectionSpec so a Mapped binding could be narrowed to its single partition. --- crates/e2e-support/tests/hello_world.rs | 4 +- crates/flowctl/src/raw/preview_next/driver.rs | 1 - crates/proto-flow/src/runtime.rs | 9 +- crates/proto-flow/src/runtime.serde.rs | 18 - crates/publisher/src/binding.rs | 73 ++- crates/publisher/src/lib.rs | 55 +- crates/publisher/src/mapping.rs | 55 +- crates/publisher/src/publisher.rs | 101 ++-- .../src/leader/materialize/startup.rs | 4 - crates/runtime-next/src/publish.rs | 25 +- .../src/shard/materialize/startup.rs | 4 - go/bindings/build_test.go | 16 - go/protocols/catalog/build_load.go | 11 - go/protocols/runtime/runtime.pb.go | 533 ++++++++---------- go/protocols/runtime/runtime.proto | 6 +- go/runtime/materialize_v2.go | 30 - 16 files changed, 421 insertions(+), 524 deletions(-) diff --git a/crates/e2e-support/tests/hello_world.rs b/crates/e2e-support/tests/hello_world.rs index c6e4b30c2da..1ed0c65e9b1 100644 --- a/crates/e2e-support/tests/hello_world.rs +++ b/crates/e2e-support/tests/hello_world.rs @@ -53,7 +53,7 @@ async fn hello_world(build: Arc, journal_client: gazette::journal .as_ref() .expect("built collection should have a spec"); - let binding = publisher::Binding::from_collection_spec(collection_spec, None) + let binding = publisher::Binding::from_collection_spec(collection_spec) .expect("should build binding from collection spec"); let factory: gazette::journal::ClientFactory = Arc::new({ @@ -118,7 +118,7 @@ async fn hello_world(build: Arc, journal_client: gazette::journal .expect("ACK write should succeed"); // Snapshot the partition listing from the Publisher's own watch. - let (_client, partitions) = publisher.binding_client(0); + let (_client, partitions) = publisher.mapped_binding_client(0); let partitions_watch = partitions.ready().await; let splits = partitions_watch.token(); let splits = splits.result().expect("partitions should be available"); diff --git a/crates/flowctl/src/raw/preview_next/driver.rs b/crates/flowctl/src/raw/preview_next/driver.rs index 93188122a71..48fafe5e431 100644 --- a/crates/flowctl/src/raw/preview_next/driver.rs +++ b/crates/flowctl/src/raw/preview_next/driver.rs @@ -162,7 +162,6 @@ async fn drive_one_shard( task: Some(proto::Task { spec: spec_bytes.clone(), ops_stats_journal: String::new(), - ops_stats_spec: None, preview: true, max_transactions: target_txns, }), diff --git a/crates/proto-flow/src/runtime.rs b/crates/proto-flow/src/runtime.rs index f9d128883ff..7f9c4956840 100644 --- a/crates/proto-flow/src/runtime.rs +++ b/crates/proto-flow/src/runtime.rs @@ -485,17 +485,16 @@ pub struct Joined { /// 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)] +#[derive(Clone, PartialEq, Eq, Hash, ::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. + /// Collection journal partition to which task stats are written. + /// The journal is pre-created by activate; the runtime appends directly + /// without consulting the partitions watch or creating new partitions. #[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, diff --git a/crates/proto-flow/src/runtime.serde.rs b/crates/proto-flow/src/runtime.serde.rs index a9288fe234a..99a5cacfde3 100644 --- a/crates/proto-flow/src/runtime.serde.rs +++ b/crates/proto-flow/src/runtime.serde.rs @@ -7629,9 +7629,6 @@ impl serde::Serialize for Task { if !self.ops_stats_journal.is_empty() { len += 1; } - if self.ops_stats_spec.is_some() { - len += 1; - } if self.preview { len += 1; } @@ -7647,9 +7644,6 @@ impl serde::Serialize for Task { 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)?; } @@ -7669,8 +7663,6 @@ impl<'de> serde::Deserialize<'de> for Task { "spec", "ops_stats_journal", "opsStatsJournal", - "ops_stats_spec", - "opsStatsSpec", "preview", "max_transactions", "maxTransactions", @@ -7680,7 +7672,6 @@ impl<'de> serde::Deserialize<'de> for Task { enum GeneratedField { Spec, OpsStatsJournal, - OpsStatsSpec, Preview, MaxTransactions, } @@ -7706,7 +7697,6 @@ impl<'de> serde::Deserialize<'de> for Task { 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)), @@ -7730,7 +7720,6 @@ impl<'de> serde::Deserialize<'de> for Task { { 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()? { @@ -7749,12 +7738,6 @@ impl<'de> serde::Deserialize<'de> for Task { } 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")); @@ -7774,7 +7757,6 @@ impl<'de> serde::Deserialize<'de> for Task { 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(), }) diff --git a/crates/publisher/src/binding.rs b/crates/publisher/src/binding.rs index d82f4f2f0be..07261392ec2 100644 --- a/crates/publisher/src/binding.rs +++ b/crates/publisher/src/binding.rs @@ -2,8 +2,18 @@ use anyhow::Context; use proto_flow::flow; use proto_gazette::broker; -/// Metadata for mapping documents to collection partitions. -pub struct Binding { +/// Metadata for routing publications to a specific journal. +pub enum Binding { + /// `Mapped` bindings dynamically resolve documents to one of a collection's + /// physical partitions, creating partitions on-demand. + Mapped(MappedBinding), + /// `Fixed` bindings target a single, pre-existing journal by name. + Fixed(FixedBinding), +} + +/// Routes documents to a collection's physical partitions via key hashing +/// and partition-field extraction. +pub struct MappedBinding { /// Target collection name (for logging/debugging). pub collection: models::Collection, /// Pre-built key extractors for the collection key pointers. @@ -16,9 +26,14 @@ pub struct Binding { pub partitions_template: broker::JournalSpec, /// Maximum number of allowed partitions for this binding. pub partitions_limit: usize, - /// Collection partitions prefix ("{partitions_template.name}/"), or a - /// more-specific prefix or journal name to which this binding is scoped. - pub partitions_prefix_or_name: String, + /// Collection partitions prefix ("{partitions_template.name}/"). + pub partitions_prefix: String, +} + +/// Routes documents to a single named journal that already exists. +pub struct FixedBinding { + /// Journal to which the binding publishes. + pub journal: String, } impl Binding { @@ -33,25 +48,17 @@ impl Binding { .as_ref() .with_context(|| format!("capture binding {index} missing collection"))?; - Self::from_collection_spec(collection_spec, None).with_context(|| { + Self::from_collection_spec(collection_spec).with_context(|| { format!("building binding for collection {}", collection_spec.name) }) }) .collect() } - /// Build a Binding from a built CollectionSpec. - /// - /// If `partitions_prefix_or_name` is Some, the Binding will authorize-to and - /// watch only that sub-prefix or specific journal. When None, the Binding - /// authorizes to all partitions of the collection. + /// Build a Mapped Binding from a built CollectionSpec. /// - /// `partitions_prefix_or_name` must be prefixed by the CollectionSpec's - /// actual partition template prefix, or this routine errors. - pub fn from_collection_spec( - spec: &flow::CollectionSpec, - partitions_prefix_or_name: Option<&str>, - ) -> anyhow::Result { + /// The Binding authorizes to and watches all partitions of the collection. + pub fn from_collection_spec(spec: &flow::CollectionSpec) -> anyhow::Result { let flow::CollectionSpec { name, key, @@ -67,17 +74,6 @@ impl Binding { .clone(); let partitions_prefix = format!("{}/", &partitions_template.name); - let partitions_prefix_or_name = if let Some(fixed) = partitions_prefix_or_name { - if !fixed.starts_with(&partitions_prefix) { - anyhow::bail!( - "prefix or name {fixed} must begin with collection prefix {partitions_prefix}" - ); - } - fixed.to_string() - } else { - partitions_prefix - }; - let policy = doc::SerPolicy::noop(); let key_extractors = extractors::for_key(key, projections, &policy).context("building key extractors")?; @@ -94,14 +90,31 @@ impl Binding { 100 }; - Ok(Self { + Ok(Self::Mapped(MappedBinding { collection: models::Collection::new(name), key_extractors, partition_fields: partition_fields.clone(), partition_extractors, partitions_template, partitions_limit, - partitions_prefix_or_name, + partitions_prefix, + })) + } + + /// Build a Fixed Binding that publishes to a single named journal. + /// The binding skips the partitions watch and partition-mapping machinery. + pub fn for_fixed_journal(journal: impl Into) -> Self { + Self::Fixed(FixedBinding { + journal: journal.into(), }) } + + /// AuthZ object string for this binding's lazy journal Client. For Mapped + /// bindings this is the partitions prefix; for Fixed it's the journal name. + pub(crate) fn authz_object(&self) -> &str { + match self { + Self::Mapped(b) => &b.partitions_prefix, + Self::Fixed(b) => &b.journal, + } + } } diff --git a/crates/publisher/src/lib.rs b/crates/publisher/src/lib.rs index 8ab94e9d5c1..17ddb09997b 100644 --- a/crates/publisher/src/lib.rs +++ b/crates/publisher/src/lib.rs @@ -6,34 +6,55 @@ pub mod publisher; pub mod watch; pub use appender::{Appender, AppenderGroup}; -pub use binding::Binding; +pub use binding::{Binding, FixedBinding, MappedBinding}; pub use publisher::Publisher; -/// Boxed closure for lazy initialization of a partitions watch and journal Client. -/// Callers of `Binding::from_collection_spec` provide this to control how the -/// journal Client and partitions watch are created. -type PartitionsClientInit = Box< +/// Boxed closure for lazy initialization of a Mapped binding's partitions +/// watch and journal Client. +type MappedClientInit = Box< dyn FnOnce() -> ( gazette::journal::Client, tokens::PendingWatch>, ) + Send, >; -/// LazyPartitionsClient uses a LazyCell to defer initialization of a partitions -/// watch and a paired journal Client for List, Apply, and Append RPCs. +/// Boxed closure for lazy initialization of a Fixed binding's journal Client. +type FixedClientInit = Box gazette::journal::Client + Send>; + +/// LazyBindingClient defers initialization of per-binding journal resources +/// until first use. +/// +/// Mapped bindings need both a journal Client and a long-lived list-watch +/// stream of partitions. Fixed bindings only need a Client (the journal is +/// known by name; no listing is required). /// /// An instantiated client and watch each consume background resources: -/// periodic token refreshes for the client, and a long-lived list RPC for the watch. -/// However, many (most?) bindings and collections are infrequently written and -/// a Publisher instance may never interact with the binding during its lifetime, -/// so avoid paying this cost until we know it's needed. -type LazyPartitionsClient = std::sync::LazyLock< - ( - gazette::journal::Client, - tokens::PendingWatch>, +/// periodic token refreshes for the client, and a long-lived list RPC for the +/// watch. However, many (most?) bindings and collections are infrequently +/// written and a Publisher instance may never interact with the binding during +/// its lifetime, so avoid paying this cost until we know it's needed. +pub(crate) enum LazyBindingClient { + Mapped( + std::sync::LazyLock< + ( + gazette::journal::Client, + tokens::PendingWatch>, + ), + MappedClientInit, + >, ), - PartitionsClientInit, ->; + Fixed(std::sync::LazyLock), +} + +impl LazyBindingClient { + /// Force initialization and return the underlying journal Client. + pub(crate) fn client(&self) -> &gazette::journal::Client { + match self { + Self::Mapped(lazy) => &lazy.0, + Self::Fixed(lazy) => &**lazy, + } + } +} /// Sanity-check that `intents` is non-empty NDJSON: terminated by a newline, /// with every line a syntactically-valid JSON document. diff --git a/crates/publisher/src/mapping.rs b/crates/publisher/src/mapping.rs index f01253ea0be..57d7619949d 100644 --- a/crates/publisher/src/mapping.rs +++ b/crates/publisher/src/mapping.rs @@ -14,8 +14,14 @@ use proto_gazette::broker; /// /// Port of Go's `Mapper.Map` (`go/flow/mapping.go`). pub(crate) async fn map_partition( - binding: &super::Binding, - client: &super::LazyPartitionsClient, + binding: &super::MappedBinding, + lazy: &std::sync::LazyLock< + ( + gazette::journal::Client, + tokens::PendingWatch>, + ), + crate::MappedClientInit, + >, doc: &N, prefix: String, packed_key: bytes::BytesMut, @@ -23,7 +29,7 @@ pub(crate) async fn map_partition( let (mut prefix, packed_key, key_hash) = extract_mapping_context(binding, doc, prefix, packed_key)?; - let (client, partitions) = &(**client); + let (client, partitions) = &(**lazy); let partitions = partitions.ready().await; loop { @@ -72,7 +78,7 @@ pub(crate) async fn map_partition( } fn extract_mapping_context( - binding: &super::Binding, + binding: &super::MappedBinding, doc: &N, mut prefix: String, mut packed_key: bytes::BytesMut, @@ -150,7 +156,7 @@ fn pick_partition( // Panics if field extraction fails, as build_logical_prefix() should have // already been called. fn build_partition_apply( - binding: &super::Binding, + binding: &super::MappedBinding, doc: &N, ) -> tonic::Result<(String, broker::ApplyRequest)> { let mut spec = binding.partitions_template.clone(); @@ -173,13 +179,6 @@ fn build_partition_apply( spec.name = name.clone(); spec.labels = Some(labels); - if !name.starts_with(&binding.partitions_prefix_or_name) { - return Err(tonic::Status::invalid_argument(format!( - "candidate partition to create is {name}, but this publisher is restricted to {}", - binding.partitions_prefix_or_name - ))); - } - Ok(( name, broker::ApplyRequest { @@ -287,8 +286,8 @@ mod test { assert_eq!(pick_partition(&p, "coll/a=1/", 0), None); } - /// Build a test Binding from a built CollectionSpec. - fn test_binding(spec: &flow::CollectionSpec) -> super::super::Binding { + /// Build a test MappedBinding from a built CollectionSpec. + fn test_binding(spec: &flow::CollectionSpec) -> super::super::MappedBinding { let flow::CollectionSpec { name, partition_template, @@ -299,21 +298,21 @@ mod test { } = spec; let partition_template = partition_template.clone().unwrap(); - let partitions_prefix_or_name = format!("{}/", &partition_template.name); + let partitions_prefix = format!("{}/", &partition_template.name); let policy = doc::SerPolicy::noop(); let key_extractors = extractors::for_key(key, projections, &policy).unwrap(); let partition_extractors = extractors::for_fields(partition_fields, projections, &policy).unwrap(); - super::super::Binding { + super::super::MappedBinding { collection: models::Collection::new(name), key_extractors, partition_fields: partition_fields.clone(), partition_extractors, partitions_template: partition_template, partitions_limit: 100, - partitions_prefix_or_name, + partitions_prefix, } } @@ -331,7 +330,7 @@ mod test { .unwrap(); let spec = spec.as_ref().unwrap(); - let mut binding = test_binding(spec); + let binding = test_binding(spec); // extract_mapping_context encodes partition field values into a logical prefix. let (prefix_1, _, _) = extract_mapping_context( @@ -375,25 +374,5 @@ mod test { "change": request.changes.into_iter().next().unwrap(), }) ); - - // A more-specific prefix that still covers the candidate partition is OK. - binding.partitions_prefix_or_name = - "example/collection/2020202020202020/a_bool=%_true/".to_string(); - build_partition_apply( - &binding, - &json!({"a_key": "k", "a_bool": true, "a_str": "hello"}), - ) - .unwrap(); - - // A sibling prefix that does NOT cover the candidate partition is rejected. - binding.partitions_prefix_or_name = - "example/collection/2020202020202020/a_bool=%_false/".to_string(); - let err = build_partition_apply( - &binding, - &json!({"a_key": "k", "a_bool": true, "a_str": "hello"}), - ) - .unwrap_err(); - assert_eq!(err.code(), tonic::Code::InvalidArgument); - insta::assert_snapshot!(err.message(), @"candidate partition to create is example/collection/2020202020202020/a_bool=%_true/a_str=hello/pivot=00, but this publisher is restricted to example/collection/2020202020202020/a_bool=%_false/"); } } diff --git a/crates/publisher/src/publisher.rs b/crates/publisher/src/publisher.rs index 482a5ec60bd..79e786e06d7 100644 --- a/crates/publisher/src/publisher.rs +++ b/crates/publisher/src/publisher.rs @@ -10,8 +10,9 @@ pub struct Publisher { authz_subject: String, // Bindings of this Publisher. bindings: Vec, - // Lazily-initialized journal Client and partitions watch for each `bindings` entry. - binding_clients: Vec, + // Lazily-initialized journal Client (and, for Mapped bindings, partitions + // watch) for each `bindings` entry. + binding_clients: Vec, // Factory for building journal Clients on demand. client_factory: gazette::journal::ClientFactory, // Clock used to stamp published document UUIDs. @@ -31,7 +32,7 @@ impl Publisher { /// (one per entry of `bindings`), and to build ephemeral clients inside /// `write_intents` for ACK intents that do not match any current binding. /// `authz_subject` is passed through to this factory without modification, - /// and `Binding::partitions_prefix_or_name` is the AuthZ object. + /// and a binding's `authz_object()` is the AuthZ object. /// /// The `producer` identifies this Publisher as a distinct writer and is /// embedded in every UUID it generates. The `clock` provides a monotonic @@ -46,17 +47,26 @@ impl Publisher { let binding_clients = bindings .iter() .map(|b| { - let client_factory = client_factory.clone(); + let factory = client_factory.clone(); let authz_subject = authz_subject.clone(); - let authz_object = b.partitions_prefix_or_name.clone(); + let authz_object = b.authz_object().to_string(); - let init: crate::PartitionsClientInit = Box::new(move || { - let client = client_factory(authz_subject, authz_object.clone()); - let partitions = crate::watch::watch_partitions(client.clone(), &authz_object); - (client, partitions) - }); - - std::sync::LazyLock::new(init) + match b { + super::Binding::Mapped(_) => { + let init: crate::MappedClientInit = Box::new(move || { + let client = factory(authz_subject, authz_object.clone()); + let partitions = + crate::watch::watch_partitions(client.clone(), &authz_object); + (client, partitions) + }); + super::LazyBindingClient::Mapped(std::sync::LazyLock::new(init)) + } + super::Binding::Fixed(_) => { + let init: crate::FixedClientInit = + Box::new(move || factory(authz_subject, authz_object)); + super::LazyBindingClient::Fixed(std::sync::LazyLock::new(init)) + } + } }) .collect(); @@ -89,11 +99,13 @@ impl Publisher { /// Enqueue a document for publication to the appropriate journal partition. /// /// Assigns a UUID with the given `flags` and passes it to `doc`, which - /// returns `(binding_index, document)`. The document is mapped to a physical - /// partition (creating one if needed, which may issue an Apply RPC), serialized - /// as newline-delimited JSON into the partition's Appender buffer, and - /// checkpoint'd. The checkpoint may start a background Append RPC if the - /// buffer exceeds the flush threshold. + /// returns `(binding_index, document)`. For Mapped bindings the document + /// is mapped to a physical partition (creating one if needed, which may + /// issue an Apply RPC). For Fixed bindings the binding's journal is used + /// directly, with no key extraction or partition mapping. The document is + /// serialized as newline-delimited JSON into the partition's Appender + /// buffer, and checkpoint'd. The checkpoint may start a background Append + /// RPC if the buffer exceeds the flush threshold. pub async fn enqueue( &mut self, doc: impl FnOnce(uuid::Uuid) -> (usize, N), @@ -105,18 +117,24 @@ impl Publisher { // Sequence the document. let uuid = proto_gazette::uuid::build(self.producer, self.clock.tick(), flags); - let (binding, doc) = doc(uuid); - - let (mut journal, mut packed_key) = super::mapping::map_partition( - &self.bindings[binding], - &self.binding_clients[binding], - &doc, - prefix, - packed_key, - ) - .await?; - - let (client, _partitions) = &(*self.binding_clients[binding]); + let (binding_idx, doc) = doc(uuid); + + let (mut journal, mut packed_key) = match &self.bindings[binding_idx] { + super::Binding::Mapped(mapped) => { + let super::LazyBindingClient::Mapped(lazy) = &self.binding_clients[binding_idx] + else { + unreachable!("Mapped binding has Mapped lazy client"); + }; + super::mapping::map_partition(mapped, lazy, &doc, prefix, packed_key).await? + } + super::Binding::Fixed(fixed) => { + let mut prefix = prefix; + prefix.push_str(&fixed.journal); + (prefix, packed_key) + } + }; + + let client = self.binding_clients[binding_idx].client(); let appender = self.appenders.activate(&journal, client); // Enqueue the serialization to the Appender's buffer, then checkpoint. @@ -167,8 +185,9 @@ impl Publisher { /// Takes the output of `intents::build_transaction_intents()` — per-journal /// NDJSON `Bytes` — and appends each to its journal in parallel. For each /// journal, this uses a hybrid client strategy: - /// - If the journal is a prefix-match of an existing binding's - /// `auth_prefix_or_name`, reuse that binding's client. + /// - If the journal matches a binding, reuse that binding's client. + /// For Mapped bindings the match is a prefix-match on the binding's + /// partitions prefix; for Fixed bindings it's an exact name match. /// - Otherwise, build an ephemeral client. This supports recovered ACK /// intents that may reference journals no longer bound to the current /// task (e.g. from a prior published task). For this class of journals, @@ -192,8 +211,11 @@ impl Publisher { let binding_client = self .bindings .iter() - .position(|b| journal.starts_with(&b.partitions_prefix_or_name)) - .map(|i| &(*self.binding_clients[i]).0); + .position(|b| match b { + super::Binding::Mapped(m) => journal.starts_with(&m.partitions_prefix), + super::Binding::Fixed(f) => journal == f.journal, + }) + .map(|i| self.binding_clients[i].client()); let appender = if let Some(client) = binding_client { self.appenders.activate(&journal, client) @@ -226,15 +248,20 @@ impl Publisher { Ok(()) } - /// Access the lazy Client and partitions watch for the binding at `index`. - /// Primarily used by tests. - pub fn binding_client( + /// Access the lazy Client and partitions watch for the Mapped binding at + /// `index`. Panics if the binding is Fixed. Primarily used by tests. + pub fn mapped_binding_client( &self, index: usize, ) -> &( gazette::journal::Client, tokens::PendingWatch>, ) { - &*self.binding_clients[index] + match &self.binding_clients[index] { + super::LazyBindingClient::Mapped(lazy) => &**lazy, + super::LazyBindingClient::Fixed(_) => { + panic!("binding {index} is Fixed, not Mapped") + } + } } } diff --git a/crates/runtime-next/src/leader/materialize/startup.rs b/crates/runtime-next/src/leader/materialize/startup.rs index 6d99328088c..5d09efb6c24 100644 --- a/crates/runtime-next/src/leader/materialize/startup.rs +++ b/crates/runtime-next/src/leader/materialize/startup.rs @@ -78,7 +78,6 @@ pub(super) async fn run( // Build task definition. let proto::Task { ops_stats_journal, - ops_stats_spec, preview, max_transactions, spec: spec_bytes, @@ -94,13 +93,10 @@ pub(super) async fn run( 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")? diff --git a/crates/runtime-next/src/publish.rs b/crates/runtime-next/src/publish.rs index 4ef8a81b4c0..2fcf695184b 100644 --- a/crates/runtime-next/src/publish.rs +++ b/crates/runtime-next/src/publish.rs @@ -11,12 +11,11 @@ //! 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. +//! Construction is decided in `startup::run` based on the `preview` flag in +//! `L:Task`: `false` ⇒ `Real`, `true` ⇒ `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; @@ -36,28 +35,24 @@ pub enum Publisher { } 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. + /// Build a real `Publisher` backed by a `publisher::Publisher` for the + /// pre-created `ops_stats_journal` 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, + I: IntoIterator, { let mut bindings = Vec::new(); - bindings.push(publisher::Binding::from_collection_spec( - ops_stats_spec, - Some(ops_stats_journal), - )?); + bindings.push(publisher::Binding::for_fixed_journal(ops_stats_journal)); for spec in collection_specs { - bindings.push(publisher::Binding::from_collection_spec(spec, None)?); + bindings.push(publisher::Binding::from_collection_spec(spec)?); } let mut producer: [u8; 6] = rand::random(); @@ -101,7 +96,7 @@ impl Publisher { Self::Real(p) => { p.enqueue( |uuid| { - // Binding index 0 is ops_stats_spec. + // Binding index 0 is the fixed ops_stats journal. stats.meta.as_mut().unwrap().uuid = uuid.to_string(); (0, serde_json::to_value(&stats).unwrap()) }, diff --git a/crates/runtime-next/src/shard/materialize/startup.rs b/crates/runtime-next/src/shard/materialize/startup.rs index cc7b3b23568..88b82af7281 100644 --- a/crates/runtime-next/src/shard/materialize/startup.rs +++ b/crates/runtime-next/src/shard/materialize/startup.rs @@ -108,7 +108,6 @@ where let proto::Task { max_transactions: _, ops_stats_journal, - ops_stats_spec, preview, spec: spec_bytes, } = l_task; @@ -126,13 +125,10 @@ where 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")? diff --git a/go/bindings/build_test.go b/go/bindings/build_test.go index 737ba9ea412..72aadb3f147 100644 --- a/go/bindings/build_test.go +++ b/go/bindings/build_test.go @@ -75,22 +75,6 @@ func TestBuildCatalog(t *testing.T) { require.NoError(t, err) cupaloy.SnapshotT(t, out) }) - t.Run("collection-for-journal", func(t *testing.T) { - // Prefix match: a partitioned journal of `a/collection`. - out, err := catalog.LoadCollectionForJournal(db, - "a/collection/ffffffffffffffff/foo=bar/pivot=00") - require.NoError(t, err) - require.Equal(t, "a/collection", out.Name.String()) - - // Exact-match without trailing partition path is not supported. - _, err = catalog.LoadCollectionForJournal(db, "a/collection") - require.ErrorIs(t, err, sql.ErrNoRows) - - // No prefix match. - _, err = catalog.LoadCollectionForJournal(db, "no/such/journal") - require.ErrorIs(t, err, sql.ErrNoRows) - }) - return nil })) } diff --git a/go/protocols/catalog/build_load.go b/go/protocols/catalog/build_load.go index 82475448085..ae6cdb6d404 100644 --- a/go/protocols/catalog/build_load.go +++ b/go/protocols/catalog/build_load.go @@ -81,17 +81,6 @@ func LoadCollection(db *sql.DB, name string) (*pf.CollectionSpec, error) { return out, loadOneSpec(db, `SELECT spec FROM built_collections WHERE collection = ?;`, out, name) } -// LoadCollectionForJournal loads the CollectionSpec whose name is the -// slash-delimited prefix of `journal`. -func LoadCollectionForJournal(db *sql.DB, journal string) (*pf.CollectionSpec, error) { - var out = new(pf.CollectionSpec) - return out, loadOneSpec(db, - `SELECT spec FROM built_collections - WHERE substr(?, 1, length(collection) + 1) = collection || '/' - ORDER BY length(collection) DESC LIMIT 1;`, - out, journal) -} - // LoadAllCaptures loads all captures. func LoadAllCaptures(db *sql.DB) ([]*pf.CaptureSpec, error) { var out []*pf.CaptureSpec diff --git a/go/protocols/runtime/runtime.pb.go b/go/protocols/runtime/runtime.pb.go index 9271a001780..ce416e7fc5f 100644 --- a/go/protocols/runtime/runtime.pb.go +++ b/go/protocols/runtime/runtime.pb.go @@ -1514,10 +1514,10 @@ var xxx_messageInfo_Joined proto.InternalMessageInfo 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. + // Collection journal partition to which task stats are written. + // The journal is pre-created by activate; the runtime appends directly + // without consulting the partitions watch or creating new partitions. 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. @@ -2920,244 +2920,243 @@ func init() { } var fileDescriptor_73af6e0737ce390c = []byte{ - // 3790 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, 0x58, 0xa2, 0xe4, 0xf6, 0xd8, 0x16, 0xe9, 0xb1, 0x1d, - 0x73, 0x25, 0x6a, 0xc8, 0xa5, 0xbd, 0xbb, 0xb6, 0xb0, 0x5a, 0x8b, 0x1c, 0x92, 0x36, 0xb5, 0x94, - 0xc4, 0x14, 0x29, 0x21, 0xc9, 0xa5, 0xd1, 0xec, 0x2a, 0x0e, 0x5b, 0xec, 0xe9, 0x6a, 0x77, 0xf5, - 0x90, 0x9a, 0xbd, 0xe6, 0x12, 0x20, 0x87, 0x5c, 0x72, 0xcb, 0x25, 0x39, 0x05, 0x08, 0x92, 0x43, - 0xae, 0x7b, 0xc8, 0x29, 0x40, 0x7c, 0x0c, 0x82, 0x20, 0xc8, 0x49, 0x40, 0x36, 0xd7, 0xdc, 0x36, - 0x01, 0x12, 0x21, 0x87, 0x45, 0x7d, 0xfa, 0x33, 0x3f, 0x5a, 0xa2, 0x7c, 0xf0, 0x41, 0x62, 0xd7, - 0xfb, 0xf5, 0xab, 0x57, 0xef, 0x57, 0xaf, 0x07, 0xda, 0x3d, 0xb6, 0x16, 0x84, 0x2c, 0x62, 0x0e, - 0xf3, 0xf8, 0x5a, 0x38, 0xf0, 0x23, 0xb7, 0x4f, 0xe3, 0xbf, 0x1d, 0x89, 0x41, 0x15, 0xbd, 0x6c, - 0xdd, 0x3c, 0x0e, 0xd9, 0x19, 0x0d, 0x13, 0x86, 0xe4, 0x41, 0x11, 0xb6, 0x96, 0x1d, 0xe6, 0xf3, - 0x41, 0xff, 0x32, 0x8a, 0x91, 0xd7, 0x11, 0x1a, 0xba, 0xe7, 0x54, 0xff, 0xd1, 0x14, 0xef, 0x8d, - 0x50, 0x9c, 0x78, 0xec, 0x42, 0xfe, 0xa7, 0xb1, 0xb7, 0x46, 0xb0, 0x7d, 0x3b, 0xa2, 0xa1, 0x6b, - 0x7b, 0xee, 0xaf, 0x68, 0xf6, 0x59, 0xd3, 0xb6, 0x46, 0x68, 0x59, 0x20, 0xff, 0x69, 0xdc, 0xe8, - 0xb6, 0xf9, 0xe9, 0xe0, 0xe4, 0xc4, 0xa3, 0xf1, 0x5f, 0x4d, 0xb3, 0xd8, 0x63, 0x3d, 0x26, 0x1f, - 0xd7, 0xc4, 0x93, 0x82, 0xb6, 0xff, 0x21, 0x07, 0x0b, 0x47, 0x36, 0x3f, 0x3b, 0xa4, 0xe1, 0xb9, - 0xeb, 0xd0, 0x2e, 0xf3, 0x4f, 0xdc, 0x1e, 0xba, 0x09, 0x35, 0x8f, 0xf5, 0xac, 0x13, 0xd7, 0xa3, - 0xd6, 0x09, 0x31, 0x73, 0xcb, 0xb9, 0x95, 0x12, 0x36, 0x3c, 0xd6, 0xdb, 0x75, 0x3d, 0xba, 0x4b, - 0xd0, 0xbb, 0x60, 0x44, 0x36, 0x3f, 0xb3, 0x7c, 0xbb, 0x4f, 0xcd, 0xfc, 0x72, 0x6e, 0xc5, 0xc0, - 0x55, 0x01, 0x78, 0x64, 0xf7, 0x29, 0x7a, 0x07, 0xaa, 0x03, 0xc2, 0xad, 0xc0, 0x8e, 0x4e, 0xcd, - 0x82, 0xc4, 0x55, 0x06, 0x84, 0x1f, 0xd8, 0xd1, 0x29, 0xba, 0x0d, 0x0b, 0x0e, 0xf3, 0x23, 0xdb, - 0xf5, 0x69, 0x68, 0xf9, 0x34, 0xba, 0x60, 0xe1, 0x99, 0x59, 0x94, 0x34, 0xcd, 0x04, 0xf1, 0x48, - 0xc1, 0xd1, 0x47, 0x50, 0x0a, 0x3c, 0xdb, 0xa7, 0x66, 0x79, 0x39, 0xb7, 0xd2, 0xd8, 0x68, 0x74, - 0xe2, 0x63, 0x3c, 0x10, 0x50, 0xac, 0x90, 0xed, 0xff, 0x2b, 0x42, 0xe3, 0x50, 0x6d, 0x14, 0xd3, - 0x6f, 0x06, 0x94, 0x47, 0x68, 0x0f, 0x2a, 0xcf, 0xd8, 0x20, 0xf4, 0x6d, 0x4f, 0x6a, 0x6e, 0x6c, - 0xad, 0xbd, 0x7c, 0xb1, 0x74, 0xbb, 0xc7, 0x3a, 0x3d, 0xfb, 0x57, 0x34, 0x8a, 0x68, 0x87, 0xd0, - 0xf3, 0x35, 0x87, 0x85, 0x74, 0x6d, 0xcc, 0x01, 0x3a, 0x0f, 0x14, 0x1b, 0x8e, 0xf9, 0xd1, 0x0d, - 0x28, 0x87, 0x34, 0xf0, 0xec, 0xa1, 0xdc, 0x65, 0x15, 0xeb, 0x95, 0xd8, 0xe3, 0xf1, 0xc0, 0xf5, - 0x88, 0xe5, 0x92, 0x78, 0x8f, 0x72, 0xbd, 0x47, 0xd0, 0x2e, 0x94, 0xd9, 0xc9, 0x09, 0xa7, 0x91, - 0xdc, 0x58, 0x61, 0xab, 0xf3, 0xf2, 0xc5, 0xd2, 0xad, 0x57, 0x79, 0xf9, 0x63, 0xc9, 0x85, 0x35, - 0x37, 0x7a, 0x08, 0x40, 0x7d, 0x62, 0x69, 0x59, 0xa5, 0x2b, 0xc9, 0x32, 0xa8, 0x4f, 0xd4, 0x23, - 0xba, 0x0d, 0xa5, 0xd0, 0xf6, 0x7b, 0xca, 0x9a, 0xb5, 0x8d, 0xf9, 0x8e, 0x74, 0x43, 0x2c, 0x40, - 0x87, 0x01, 0x75, 0xb6, 0x8a, 0xdf, 0xbe, 0x58, 0x7a, 0x0b, 0x2b, 0x1a, 0x74, 0x08, 0x35, 0x87, - 0xb1, 0x90, 0xb8, 0xbe, 0x1d, 0xb1, 0xd0, 0xac, 0x48, 0x2b, 0xfe, 0xf8, 0xe5, 0x8b, 0xa5, 0x3b, - 0xd3, 0x5e, 0x3e, 0x11, 0x26, 0x9d, 0xc3, 0x53, 0x3b, 0x24, 0x7b, 0xdb, 0x38, 0x2b, 0x05, 0xad, - 0x03, 0x84, 0x94, 0x33, 0x6f, 0x10, 0xb9, 0xcc, 0x37, 0xab, 0x52, 0x8d, 0x66, 0x27, 0xe1, 0xf9, - 0x9a, 0xda, 0x84, 0x86, 0x38, 0x43, 0x83, 0x3e, 0x84, 0x39, 0xed, 0xc3, 0x96, 0xeb, 0x13, 0xfa, - 0xdc, 0x34, 0x96, 0x73, 0x2b, 0x73, 0xb8, 0xae, 0x81, 0x7b, 0x02, 0x86, 0x3e, 0x03, 0x90, 0x11, - 0x67, 0x4b, 0xb1, 0x20, 0xc5, 0x2e, 0xaa, 0xdd, 0x75, 0x99, 0xe7, 0x51, 0x47, 0xc0, 0xc5, 0x16, - 0x71, 0x86, 0x0e, 0x75, 0x61, 0x3e, 0x0d, 0x31, 0xc5, 0x5a, 0x93, 0xac, 0xef, 0x28, 0xd6, 0x87, - 0xa3, 0x48, 0xc9, 0x3f, 0xce, 0xd1, 0xfe, 0x97, 0x22, 0xcc, 0x27, 0xbe, 0xc7, 0x03, 0xe6, 0x73, - 0x8a, 0x56, 0xa0, 0xcc, 0x23, 0x3b, 0x1a, 0x70, 0xe9, 0x7b, 0x8d, 0x8d, 0x66, 0x27, 0x36, 0x4f, - 0xe7, 0x50, 0xc2, 0xb1, 0xc6, 0x0b, 0xca, 0x53, 0xb9, 0x67, 0xe9, 0x5b, 0xd3, 0x6c, 0xa1, 0xf1, - 0xe8, 0x63, 0x68, 0x44, 0x34, 0xec, 0xbb, 0xbe, 0xed, 0x59, 0x34, 0x0c, 0x59, 0xa8, 0x7d, 0x6e, - 0x2e, 0x86, 0xee, 0x08, 0x20, 0xfa, 0x7d, 0xa8, 0x87, 0xd4, 0x26, 0x56, 0x74, 0x1a, 0xb2, 0x41, - 0xef, 0xf4, 0x8a, 0xfe, 0x57, 0x13, 0x32, 0x8e, 0x94, 0x08, 0xe1, 0x84, 0x17, 0xa1, 0x1b, 0x51, - 0x4b, 0x68, 0x72, 0x55, 0x27, 0x94, 0x12, 0xc4, 0x96, 0xd0, 0x1e, 0x94, 0xec, 0x90, 0xfa, 0xb6, - 0x74, 0xc2, 0xfa, 0xd6, 0xa7, 0x2f, 0x5f, 0x2c, 0xad, 0xf5, 0xdc, 0xe8, 0x74, 0x70, 0xdc, 0x71, - 0x58, 0x7f, 0x8d, 0xf2, 0x68, 0x60, 0x87, 0x43, 0x95, 0x26, 0x27, 0x12, 0x67, 0x67, 0x53, 0xb0, - 0x62, 0x25, 0x01, 0x7d, 0x0c, 0x45, 0xc2, 0x1c, 0x6e, 0x56, 0x96, 0x0b, 0x2b, 0xb5, 0x8d, 0x9a, - 0x3a, 0xb5, 0x43, 0xcf, 0x75, 0xa8, 0x76, 0x65, 0x89, 0x46, 0x5f, 0x43, 0x45, 0x45, 0x10, 0x37, - 0xab, 0xcb, 0x85, 0x2b, 0x68, 0x1f, 0xb3, 0x0b, 0x3f, 0x1b, 0x0c, 0x5c, 0x62, 0x05, 0x76, 0x18, - 0x71, 0xd3, 0x90, 0xaf, 0xd5, 0x51, 0xf4, 0xe4, 0xc9, 0xde, 0xf6, 0x81, 0x00, 0xeb, 0x57, 0x1b, - 0x82, 0x50, 0x02, 0x84, 0xd3, 0x07, 0xb6, 0x73, 0x46, 0x89, 0x75, 0x46, 0x87, 0x26, 0xcc, 0x52, - 0xd6, 0x50, 0x44, 0xbf, 0xa4, 0xc3, 0x36, 0x81, 0x05, 0xcc, 0x9c, 0x33, 0xbe, 0xbd, 0xb5, 0x4d, - 0xb9, 0x13, 0xba, 0x81, 0x88, 0x9d, 0x55, 0x40, 0xa1, 0x00, 0x92, 0x63, 0x8b, 0xfa, 0xe7, 0x56, - 0x9f, 0xf6, 0x83, 0x28, 0x94, 0x1e, 0x56, 0xc6, 0x4d, 0x8d, 0xd9, 0xf1, 0xcf, 0x1f, 0x4a, 0x38, - 0xfa, 0x00, 0xea, 0x31, 0xb5, 0xcc, 0xc2, 0x2a, 0x43, 0xd7, 0x34, 0x4c, 0x64, 0xe2, 0xf6, 0x9f, - 0xe7, 0xc1, 0xe8, 0xc6, 0x19, 0x17, 0xbd, 0x0d, 0x15, 0x37, 0xb0, 0x6c, 0x42, 0x94, 0x4c, 0x03, - 0x97, 0xdd, 0x60, 0x93, 0x90, 0x10, 0xfd, 0x14, 0xe6, 0x74, 0x9a, 0xb6, 0x02, 0x26, 0xf6, 0x9d, - 0x97, 0x3b, 0x58, 0x50, 0x3b, 0xd0, 0x99, 0xfa, 0x80, 0x85, 0x11, 0xae, 0xfb, 0xe9, 0x82, 0xa3, - 0x43, 0x58, 0xe8, 0xdb, 0x41, 0x40, 0x89, 0x75, 0xca, 0x78, 0xa4, 0x79, 0x0b, 0x92, 0xf7, 0x93, - 0x24, 0x8f, 0x27, 0xef, 0xef, 0x3c, 0x94, 0xb4, 0x5f, 0x33, 0x1e, 0x49, 0xf6, 0x1d, 0x3f, 0x0a, - 0x87, 0x22, 0xdc, 0x46, 0xa0, 0xe8, 0x7d, 0x80, 0x01, 0xb7, 0x7b, 0xd4, 0x0a, 0xed, 0x88, 0x4a, - 0xef, 0xce, 0x63, 0x43, 0x42, 0xb0, 0x1d, 0xd1, 0xd6, 0x16, 0x2c, 0x4e, 0x93, 0x83, 0x9a, 0x50, - 0x10, 0xb6, 0xcf, 0xc9, 0xdc, 0x21, 0x1e, 0xd1, 0x22, 0x94, 0xce, 0x6d, 0x6f, 0x10, 0x97, 0x2e, - 0xb5, 0xb8, 0x9b, 0xff, 0x3c, 0xd7, 0xfe, 0x9b, 0x3c, 0x2c, 0x74, 0xed, 0x20, 0x1a, 0x84, 0x71, - 0x35, 0xd9, 0x79, 0x2e, 0x72, 0xa7, 0xa8, 0x7d, 0x96, 0x47, 0xcf, 0xa9, 0xa7, 0xc3, 0xba, 0xd1, - 0x11, 0xd5, 0x77, 0x9f, 0xf5, 0x3a, 0xfb, 0x02, 0x8a, 0xab, 0x1e, 0xeb, 0xc9, 0x27, 0xb4, 0x97, - 0x1e, 0x15, 0x49, 0x0e, 0x50, 0x87, 0x78, 0x2b, 0xd9, 0xfb, 0xc4, 0x11, 0xe3, 0x05, 0xcd, 0x95, - 0x39, 0xf5, 0x3d, 0xa8, 0xf3, 0xc8, 0x0e, 0x23, 0xcb, 0x61, 0xfd, 0xbe, 0x1b, 0xc9, 0xa8, 0xaf, - 0x6d, 0xfc, 0x5e, 0x6a, 0xc0, 0x71, 0x4d, 0x45, 0x8a, 0x09, 0xa3, 0xae, 0xa4, 0xc6, 0x35, 0x9e, - 0x2e, 0x5a, 0x18, 0x6a, 0x19, 0x1c, 0xea, 0x02, 0xd2, 0x42, 0x2c, 0xe7, 0x94, 0x3a, 0x67, 0x01, - 0x73, 0xfd, 0x48, 0x6e, 0x4d, 0x24, 0xcf, 0x24, 0x63, 0x75, 0x13, 0x1c, 0x5e, 0xd0, 0xf4, 0x29, - 0xa8, 0xfd, 0xff, 0x45, 0x40, 0x89, 0x0a, 0x2a, 0xfd, 0x09, 0x6b, 0xad, 0x83, 0x91, 0xd4, 0x72, - 0x2d, 0x12, 0x4d, 0x9e, 0x39, 0x4e, 0x89, 0xd0, 0x5d, 0x28, 0xb3, 0x80, 0xfa, 0x94, 0x68, 0x33, - 0xb5, 0x27, 0x77, 0x98, 0x88, 0xef, 0x3c, 0x96, 0x94, 0x58, 0x73, 0xa0, 0xfb, 0x50, 0x75, 0x14, - 0x11, 0xd1, 0xf6, 0xf9, 0xe8, 0x32, 0x6e, 0x0d, 0x22, 0x38, 0xe1, 0x42, 0xbb, 0x00, 0x19, 0x1b, - 0x14, 0x67, 0xd9, 0x38, 0x23, 0x23, 0xb5, 0x4a, 0x86, 0xb3, 0xf5, 0x10, 0xca, 0x4a, 0xb7, 0xef, - 0xc5, 0xba, 0xad, 0xa7, 0x50, 0x8d, 0x95, 0x15, 0x9e, 0x7f, 0x46, 0x87, 0x96, 0x4a, 0x12, 0x52, - 0x50, 0x1d, 0x1b, 0x67, 0x74, 0x78, 0x20, 0x01, 0xa2, 0xad, 0x12, 0x59, 0xc9, 0x15, 0x45, 0x89, - 0xc7, 0x54, 0x79, 0x49, 0xd5, 0x4c, 0x11, 0x8a, 0xb8, 0x75, 0x01, 0x90, 0xbe, 0x05, 0x2d, 0x43, - 0x49, 0x94, 0x23, 0xae, 0xb5, 0x03, 0xe9, 0xd6, 0xa2, 0x50, 0x71, 0xac, 0x10, 0xe8, 0x2b, 0xa8, - 0x05, 0xcc, 0xf3, 0xac, 0x90, 0xf2, 0x81, 0x17, 0x49, 0xb1, 0x8d, 0xcb, 0xed, 0x73, 0xc0, 0x3c, - 0x0f, 0x4b, 0x6a, 0x0c, 0x41, 0xf2, 0xdc, 0x7e, 0x04, 0x90, 0x62, 0x50, 0x0d, 0x2a, 0x7b, 0x8f, - 0x9e, 0x6e, 0xee, 0xef, 0x6d, 0x37, 0xdf, 0x42, 0x06, 0x94, 0xf0, 0xce, 0xe6, 0xf6, 0x1f, 0x36, - 0x73, 0x68, 0x0e, 0x8c, 0x47, 0x8f, 0x8f, 0x2c, 0xb5, 0xcc, 0xa3, 0x3a, 0x54, 0xbb, 0x8f, 0x1f, - 0xef, 0x5b, 0x8f, 0x77, 0x77, 0x9b, 0x05, 0xc1, 0x84, 0x77, 0x0e, 0x8f, 0x36, 0xf1, 0x51, 0xb3, - 0xd8, 0xfe, 0xaf, 0x1c, 0x34, 0xb7, 0x65, 0xaf, 0xfd, 0x03, 0x08, 0xd5, 0x0d, 0x28, 0x0a, 0x87, - 0xd4, 0x2e, 0x78, 0x33, 0x61, 0x1e, 0x57, 0x50, 0xba, 0x2f, 0x96, 0xb4, 0xad, 0x55, 0x28, 0x8a, - 0x15, 0xfa, 0x08, 0x1a, 0xfc, 0x1b, 0x4f, 0x54, 0xd9, 0xf3, 0x13, 0x6e, 0x0d, 0x42, 0x57, 0x27, - 0xe1, 0xba, 0x82, 0x3e, 0x3d, 0xe1, 0x4f, 0x42, 0xb7, 0xfd, 0xdf, 0x05, 0x58, 0x88, 0xa5, 0xbd, - 0x49, 0xb0, 0x7d, 0x31, 0x16, 0x6c, 0x1f, 0x4c, 0xe8, 0x3a, 0x33, 0xd6, 0xb6, 0xc0, 0x08, 0x06, - 0xc7, 0x9e, 0xcb, 0x4f, 0xa7, 0x04, 0xdb, 0x24, 0xf7, 0x41, 0x4c, 0x8b, 0x53, 0x36, 0xf4, 0x73, - 0xa8, 0x9c, 0x78, 0x03, 0x29, 0xa1, 0x38, 0x16, 0xec, 0x93, 0x12, 0x76, 0x15, 0x25, 0x8e, 0x59, - 0xbe, 0xef, 0x18, 0x8b, 0xc0, 0x48, 0x94, 0x14, 0x97, 0x9a, 0xbe, 0xfd, 0xdc, 0x72, 0x3c, 0xe6, - 0x9c, 0xe9, 0xd2, 0x5a, 0xed, 0xdb, 0xcf, 0xbb, 0x62, 0x3d, 0x16, 0x81, 0xf9, 0x57, 0x8a, 0xc0, - 0xc2, 0x8c, 0x08, 0xbc, 0x0d, 0x15, 0xbd, 0xb1, 0xef, 0x0e, 0xbf, 0xf6, 0x9f, 0xe5, 0xe0, 0x7a, - 0xda, 0x8c, 0xfe, 0x00, 0x5c, 0xbd, 0xfd, 0xeb, 0x1c, 0xdc, 0x18, 0xd1, 0xe8, 0x4d, 0xbc, 0x71, - 0x33, 0x75, 0x07, 0xa5, 0x4c, 0xda, 0x1e, 0x4c, 0x7f, 0xc7, 0xa4, 0x4f, 0xbc, 0x96, 0x39, 0x7f, - 0x5d, 0x84, 0x46, 0x97, 0xf5, 0x8f, 0x5d, 0x3f, 0xb9, 0x2e, 0xae, 0xeb, 0xd0, 0x55, 0x3c, 0xef, - 0x65, 0xf4, 0xcd, 0x92, 0x65, 0x02, 0x17, 0xdd, 0x81, 0x82, 0x4d, 0x62, 0x85, 0xdf, 0x9d, 0xc5, - 0xb0, 0x49, 0x08, 0x16, 0x74, 0xad, 0x7f, 0xcb, 0xeb, 0x40, 0xbf, 0x0f, 0xd5, 0x63, 0xd7, 0x27, - 0xae, 0xdf, 0x13, 0x1a, 0x16, 0x46, 0x6b, 0xd5, 0xe4, 0xdb, 0x3a, 0x5b, 0x8a, 0x18, 0x27, 0x5c, - 0xad, 0x3f, 0xcd, 0x43, 0x45, 0x43, 0x11, 0x82, 0xe2, 0xc9, 0xc0, 0x53, 0x47, 0x5f, 0xc5, 0xf2, - 0x39, 0xee, 0x75, 0x44, 0x97, 0x66, 0xa8, 0x5e, 0xe7, 0x73, 0xa8, 0x05, 0x21, 0x7b, 0xa6, 0xae, - 0x41, 0x71, 0x0f, 0xd6, 0x54, 0xfd, 0xdb, 0x41, 0x82, 0xd0, 0x6d, 0x68, 0x96, 0x14, 0xdd, 0x83, - 0x1a, 0x77, 0x4e, 0x69, 0xdf, 0xb6, 0x9e, 0x71, 0xe6, 0xcb, 0x68, 0xad, 0x6f, 0xbd, 0xf7, 0xf2, - 0xc5, 0x92, 0x49, 0x7d, 0x87, 0x09, 0x15, 0xd6, 0x04, 0xa2, 0x83, 0xed, 0x8b, 0x87, 0x94, 0xcb, - 0x36, 0x0c, 0x14, 0xc3, 0x03, 0xce, 0x7c, 0xd4, 0x01, 0xe0, 0x34, 0xb4, 0x02, 0xe6, 0xb9, 0xce, - 0x50, 0x5e, 0x1d, 0x92, 0x7e, 0xf9, 0x90, 0x86, 0x07, 0x12, 0x8c, 0x0d, 0x1e, 0x3f, 0xca, 0xb1, - 0x81, 0xec, 0xaf, 0xa3, 0x50, 0x5e, 0x0f, 0x0c, 0x5c, 0x91, 0x6d, 0x74, 0x14, 0x8a, 0x5b, 0xb8, - 0x6c, 0xd1, 0x54, 0xb7, 0x6f, 0x60, 0xbd, 0x6a, 0xf9, 0x50, 0xd8, 0x24, 0x04, 0x99, 0x50, 0xd1, - 0x06, 0xd2, 0x4d, 0x5e, 0xbc, 0x44, 0x3f, 0x83, 0x2a, 0x61, 0x8e, 0xd2, 0x3f, 0xff, 0x0a, 0xfa, - 0x57, 0x08, 0x73, 0xa4, 0xf2, 0x8b, 0x50, 0x3a, 0x09, 0x99, 0xaf, 0x5a, 0xae, 0x2a, 0x56, 0x8b, - 0xf6, 0xbf, 0xe7, 0x60, 0x3e, 0x39, 0x27, 0x7d, 0xdf, 0x9b, 0xfd, 0x72, 0x13, 0x2a, 0x84, 0x7a, - 0x34, 0xd2, 0xae, 0x5d, 0xc5, 0xf1, 0x72, 0x44, 0xad, 0xc2, 0x95, 0xd4, 0x2a, 0x66, 0xd4, 0x1a, - 0xcb, 0x4d, 0xa5, 0xf1, 0xdc, 0xf4, 0x21, 0xcc, 0x29, 0x7b, 0xc5, 0x14, 0xf2, 0xf2, 0x85, 0xeb, - 0x0a, 0xa8, 0x88, 0xda, 0x6f, 0xc3, 0xf5, 0x2e, 0xf3, 0x7d, 0xea, 0x44, 0x2c, 0x3c, 0x08, 0xd9, - 0xf3, 0xa1, 0x76, 0xc4, 0xf6, 0x5f, 0xe4, 0xe0, 0xc6, 0x38, 0x46, 0x6f, 0xfd, 0x01, 0x54, 0xc4, - 0x95, 0x81, 0x72, 0xae, 0xe7, 0x2c, 0xeb, 0x2f, 0x5f, 0x2c, 0xad, 0xbe, 0xca, 0xdd, 0x6a, 0xc7, - 0x27, 0x2a, 0x27, 0xc7, 0x02, 0xc4, 0xe9, 0x07, 0x42, 0xb8, 0xe5, 0x12, 0xdd, 0x95, 0x57, 0xe4, - 0x7a, 0x8f, 0xa0, 0x16, 0x14, 0x3c, 0xd6, 0xd3, 0xf5, 0xa6, 0x1a, 0x67, 0x38, 0x2c, 0x80, 0xed, - 0xbf, 0x2f, 0x40, 0xf1, 0x01, 0x73, 0x7d, 0x74, 0x0b, 0x16, 0x68, 0xe4, 0x10, 0xab, 0xcf, 0x88, - 0x15, 0xd2, 0x73, 0x97, 0x8b, 0x1b, 0xbd, 0xd0, 0xaa, 0x80, 0xe7, 0x05, 0xe2, 0x21, 0x23, 0x58, - 0x83, 0xd1, 0x6d, 0x28, 0xf3, 0x53, 0x3b, 0x24, 0xf1, 0x6d, 0xe6, 0x5a, 0x12, 0x84, 0x42, 0x94, - 0x1a, 0x5e, 0x60, 0x4d, 0x82, 0x96, 0xa0, 0x26, 0x9f, 0xf4, 0x04, 0xa2, 0x20, 0xcf, 0x18, 0x24, - 0x48, 0xcd, 0x1f, 0x6e, 0xc3, 0x42, 0x3c, 0xa4, 0x20, 0x6e, 0x28, 0xcd, 0x34, 0x8c, 0x67, 0x5a, - 0x1a, 0xb1, 0x1d, 0xc3, 0xd1, 0x8f, 0x20, 0x86, 0x59, 0x54, 0xdb, 0x40, 0x1e, 0x98, 0x81, 0xe7, - 0x35, 0x3c, 0x36, 0x0d, 0xfa, 0x04, 0xe6, 0x3d, 0x79, 0xfd, 0x4f, 0x29, 0x55, 0x58, 0x34, 0x14, - 0x38, 0x26, 0x6c, 0xfd, 0x5d, 0x0e, 0x4a, 0x52, 0x67, 0xd4, 0x80, 0xbc, 0x4b, 0x74, 0xf3, 0x90, - 0x77, 0x09, 0xea, 0x40, 0xd5, 0xb3, 0x8f, 0xa9, 0x27, 0x9c, 0x33, 0xaf, 0xb3, 0xb1, 0xcc, 0x88, - 0x82, 0x7a, 0x5f, 0x63, 0x70, 0x42, 0x83, 0x36, 0xa0, 0x12, 0x52, 0x5b, 0x68, 0xaa, 0xad, 0x6d, - 0xa6, 0x23, 0x89, 0x83, 0x90, 0x39, 0x94, 0xf3, 0xc3, 0x80, 0x3a, 0x9d, 0xbd, 0x6d, 0x1c, 0x13, - 0xa2, 0x75, 0x58, 0x94, 0x86, 0x77, 0x42, 0x6a, 0x47, 0x34, 0xb5, 0xbd, 0x1c, 0x3e, 0x60, 0x24, - 0x70, 0x5d, 0x89, 0x8a, 0xcd, 0xdf, 0xfe, 0x0c, 0xca, 0xc2, 0xce, 0x94, 0x88, 0x43, 0x13, 0x15, - 0x57, 0xf2, 0x8f, 0x1f, 0x5a, 0xdf, 0x7e, 0xbe, 0x13, 0x39, 0xc9, 0xa1, 0xb5, 0xff, 0x29, 0x07, - 0xc5, 0x23, 0x9b, 0x9f, 0x89, 0xb4, 0xc7, 0x03, 0xea, 0xe8, 0x2e, 0x58, 0x3e, 0x0b, 0x41, 0x2c, - 0xe0, 0x96, 0x4c, 0xf1, 0x56, 0x3c, 0xfb, 0x53, 0x6e, 0x34, 0xcf, 0x02, 0x2e, 0x2b, 0x80, 0x9e, - 0xed, 0xa1, 0xbb, 0xd0, 0x48, 0x69, 0xa5, 0xa4, 0xc2, 0x25, 0x33, 0xa3, 0x7a, 0xcc, 0x2e, 0x56, - 0x22, 0xa4, 0x03, 0xa1, 0x28, 0xbd, 0xd0, 0x11, 0x18, 0x2f, 0xc5, 0xc1, 0x8a, 0xad, 0x44, 0xa1, - 0xed, 0x73, 0x5b, 0xe7, 0xda, 0x92, 0xf4, 0x15, 0xb1, 0x93, 0xa3, 0x0c, 0xb8, 0xfd, 0x57, 0x25, - 0xa8, 0x60, 0xea, 0xb0, 0x73, 0x59, 0xfe, 0x6a, 0xb6, 0x73, 0x66, 0xb9, 0x7e, 0x44, 0xfd, 0x28, - 0x2e, 0x0a, 0xcb, 0x69, 0x3d, 0x56, 0x64, 0x9d, 0x4d, 0xe7, 0x6c, 0x4f, 0x91, 0xa8, 0xab, 0x31, - 0xd8, 0x09, 0x00, 0x6d, 0xc0, 0x75, 0x75, 0x3d, 0x8c, 0x28, 0x11, 0xcd, 0x0b, 0xa7, 0xba, 0x85, - 0xc9, 0xcb, 0x16, 0xe6, 0x5a, 0x82, 0xec, 0x0a, 0x9c, 0xea, 0x66, 0xee, 0x03, 0x4a, 0x79, 0x64, - 0x12, 0x71, 0x69, 0x7c, 0xe6, 0x0b, 0x9d, 0x78, 0x6e, 0xbc, 0xab, 0x11, 0x78, 0x21, 0x21, 0x8e, - 0x41, 0x68, 0x15, 0x16, 0x9d, 0x38, 0x2b, 0x48, 0x5b, 0xd2, 0x4c, 0x95, 0xc0, 0x8d, 0x04, 0x27, - 0x6c, 0x47, 0xd1, 0x2a, 0xa0, 0x53, 0xb1, 0xc7, 0x51, 0x05, 0x4b, 0x6a, 0x7c, 0xa1, 0x30, 0x19, - 0xed, 0xee, 0xc2, 0xbc, 0xa6, 0x4e, 0x54, 0x2b, 0xcf, 0x52, 0xad, 0xa1, 0x28, 0x13, 0xbd, 0x3e, - 0x80, 0xba, 0x67, 0xf3, 0xc8, 0xb2, 0x83, 0xc0, 0x73, 0x29, 0x91, 0xa3, 0xcb, 0x3a, 0xae, 0x09, - 0xd8, 0xa6, 0x02, 0xa1, 0x4d, 0x58, 0xf0, 0x68, 0xcf, 0x76, 0x86, 0xd9, 0xc6, 0xb1, 0x7a, 0x49, - 0xe3, 0xd8, 0x54, 0xe4, 0x99, 0x5b, 0xd3, 0xe7, 0x20, 0x3a, 0x43, 0xeb, 0x8c, 0x0e, 0xe3, 0x49, - 0xd0, 0xfb, 0x13, 0x67, 0xf6, 0xd0, 0x7e, 0xfe, 0x4b, 0x3a, 0xd4, 0x07, 0x56, 0xe9, 0xab, 0x15, - 0xba, 0x05, 0xd7, 0xa2, 0xd0, 0xed, 0xf5, 0x44, 0x65, 0xb4, 0x43, 0xbb, 0xcf, 0x95, 0xd9, 0x40, - 0xaa, 0x39, 0xa7, 0x51, 0x07, 0x12, 0xd3, 0xba, 0x07, 0xf3, 0x63, 0x07, 0x9f, 0x9d, 0x65, 0x18, - 0x53, 0x66, 0x19, 0xf5, 0xcc, 0x2c, 0xa3, 0x75, 0x17, 0xea, 0x59, 0x1d, 0xbe, 0x6b, 0x0e, 0x92, - 0xe5, 0x6d, 0xff, 0x6b, 0x19, 0x2a, 0x07, 0x34, 0xe4, 0x2e, 0x8f, 0xd0, 0x75, 0x28, 0x73, 0xfa, - 0x8d, 0xe5, 0x33, 0xc9, 0x5a, 0xc4, 0x25, 0x4e, 0xbf, 0x79, 0xc4, 0xc4, 0x99, 0xaa, 0x7a, 0x66, - 0x65, 0x3d, 0x58, 0x55, 0xba, 0xa6, 0xc2, 0xa4, 0xda, 0x8f, 0x3b, 0x7a, 0x61, 0xcc, 0xd1, 0xf5, - 0xbb, 0xae, 0xe6, 0xe8, 0xc5, 0xd7, 0x75, 0xf4, 0xd2, 0x6b, 0x38, 0xfa, 0x3a, 0xdc, 0x48, 0x1d, - 0x3d, 0xb0, 0x23, 0xe7, 0x94, 0xea, 0x33, 0x53, 0x65, 0xb4, 0x99, 0x60, 0x0f, 0x14, 0x72, 0x86, - 0xb3, 0x57, 0x66, 0x38, 0xfb, 0x67, 0x70, 0x43, 0x9b, 0x71, 0xdc, 0xe7, 0xab, 0xd2, 0x94, 0x8b, - 0x0a, 0xfb, 0xf5, 0xa8, 0x9b, 0x4f, 0x09, 0x11, 0xe3, 0xaa, 0x21, 0x02, 0xaf, 0x18, 0x22, 0xb5, - 0x2b, 0x87, 0x48, 0x7d, 0x2c, 0x44, 0xe2, 0xd3, 0x9e, 0x1e, 0x22, 0x1b, 0x70, 0x5d, 0x5b, 0x64, - 0x34, 0x52, 0xcc, 0x39, 0x69, 0x90, 0x6b, 0x0a, 0x79, 0x94, 0x0d, 0x95, 0x59, 0x61, 0xd5, 0xf8, - 0x81, 0x85, 0x55, 0x1b, 0x0c, 0xbd, 0x77, 0x4a, 0x66, 0xc4, 0x55, 0xfb, 0xaf, 0x73, 0x50, 0x12, - 0xe7, 0x30, 0x9c, 0x5a, 0xe9, 0x4c, 0xa8, 0x9c, 0x0b, 0x09, 0xba, 0xa1, 0x35, 0x70, 0xbc, 0x14, - 0xd7, 0x57, 0x79, 0xac, 0x92, 0x45, 0xa5, 0xe1, 0xaa, 0x00, 0xc8, 0xc2, 0x15, 0x9f, 0x79, 0xcc, - 0xab, 0x7a, 0x0e, 0x79, 0xe6, 0x4f, 0x35, 0xff, 0xfa, 0x8c, 0x8c, 0xae, 0xdc, 0x1c, 0x8d, 0x66, - 0x74, 0xd1, 0x8d, 0xb6, 0x9f, 0x41, 0x25, 0x76, 0x98, 0x3b, 0x80, 0x54, 0x79, 0x4b, 0x6e, 0x97, - 0x71, 0x29, 0x37, 0xf0, 0x82, 0xc2, 0x6c, 0xa7, 0x88, 0x4b, 0x82, 0x2a, 0x3f, 0x3d, 0xa8, 0xda, - 0xbf, 0xcd, 0xe9, 0x3b, 0xd4, 0xeb, 0x19, 0xe5, 0xe3, 0xf8, 0xab, 0x57, 0x61, 0xea, 0x57, 0xaf, - 0xf8, 0x7b, 0xd7, 0x87, 0x97, 0x56, 0x33, 0x79, 0x75, 0xa4, 0xe8, 0x27, 0x19, 0x8f, 0x2e, 0x49, - 0x8f, 0x4e, 0x2f, 0xce, 0xf2, 0xba, 0x36, 0xd5, 0x9d, 0xdf, 0xc8, 0x5f, 0x00, 0xaa, 0x32, 0x55, - 0x3c, 0x62, 0x17, 0xed, 0x32, 0x14, 0x0f, 0x23, 0x16, 0xb4, 0x0d, 0xa8, 0x88, 0xbf, 0x01, 0x25, - 0xed, 0x3f, 0x80, 0xda, 0x21, 0xe5, 0x62, 0xa3, 0xfb, 0x8c, 0x05, 0x33, 0xee, 0xf8, 0xb9, 0xab, - 0xdc, 0xf1, 0xff, 0xf6, 0x06, 0xd4, 0x32, 0xf7, 0x6f, 0x74, 0x27, 0x63, 0xf5, 0xda, 0xc6, 0x3b, - 0x9d, 0xec, 0xe7, 0xe9, 0xf8, 0xca, 0x2a, 0x0d, 0xaa, 0x0e, 0xe4, 0x4b, 0x98, 0x13, 0x7f, 0xad, - 0x50, 0x5f, 0x15, 0x92, 0x41, 0xc3, 0x28, 0x9f, 0x42, 0x2a, 0xc6, 0xba, 0x60, 0x48, 0xae, 0x16, - 0x5f, 0x40, 0xf5, 0xdc, 0xf6, 0x5c, 0x62, 0x47, 0xf1, 0xd1, 0xbd, 0x3f, 0xf5, 0x9d, 0x4f, 0x35, - 0x11, 0x4e, 0xc8, 0xd1, 0x3d, 0x30, 0xe2, 0xe7, 0x78, 0xc4, 0xb4, 0x34, 0xfd, 0xbd, 0x31, 0x33, - 0xc1, 0x29, 0x07, 0xfa, 0x19, 0xd4, 0xb9, 0xb2, 0xa9, 0xe5, 0x31, 0x16, 0x98, 0x8b, 0x3a, 0xeb, - 0xc5, 0xe6, 0xcb, 0x18, 0x1c, 0xd7, 0x78, 0xc6, 0xfa, 0x1f, 0x40, 0xf1, 0x19, 0x73, 0x7d, 0xf3, - 0xba, 0x64, 0x98, 0x1b, 0xb9, 0x53, 0x60, 0x89, 0x42, 0x9f, 0x40, 0xf9, 0x99, 0xec, 0x7c, 0xcd, - 0x1b, 0xda, 0x1d, 0xb3, 0x44, 0x94, 0x60, 0x8d, 0x16, 0xb2, 0x22, 0x9b, 0x9f, 0x99, 0x6f, 0x8f, - 0xc9, 0x12, 0x0d, 0x30, 0x96, 0x28, 0x74, 0x4b, 0xf4, 0xea, 0xb2, 0xd3, 0x30, 0x4d, 0xfd, 0xf9, - 0x70, 0xac, 0x03, 0xc1, 0x31, 0x01, 0xfa, 0x08, 0x4a, 0x22, 0xd9, 0x0f, 0xcd, 0x77, 0x24, 0x65, - 0xfa, 0x25, 0x5d, 0xe6, 0x19, 0xac, 0x90, 0x42, 0x62, 0x5c, 0x12, 0x5a, 0x63, 0x12, 0x75, 0x98, - 0xe3, 0x98, 0x40, 0x28, 0x28, 0x67, 0x26, 0xef, 0x8e, 0x29, 0x98, 0x19, 0x92, 0x7c, 0x9a, 0xcc, - 0x19, 0xdf, 0x1b, 0x9b, 0x93, 0x64, 0x1c, 0x6b, 0x7c, 0xc2, 0x78, 0x07, 0x8a, 0x1e, 0xb3, 0x89, - 0xb9, 0xa2, 0xfd, 0x6c, 0x1a, 0xcb, 0x3e, 0xb3, 0x09, 0x96, 0x64, 0xe2, 0x1d, 0xe2, 0x2f, 0x25, - 0xe6, 0x8f, 0x2e, 0x79, 0xc7, 0xbe, 0x24, 0xc1, 0x9a, 0x14, 0xad, 0x43, 0x49, 0x8e, 0x8e, 0xcc, - 0x5b, 0x63, 0x91, 0x91, 0xe5, 0x91, 0x13, 0x25, 0xac, 0x08, 0xd1, 0x4f, 0xd3, 0x21, 0xd5, 0xed, - 0xb1, 0x21, 0xd1, 0x04, 0x4f, 0x66, 0x32, 0x25, 0xde, 0xc4, 0x23, 0x16, 0x52, 0x73, 0xf5, 0x92, - 0x37, 0x1d, 0x0a, 0x0a, 0xac, 0x08, 0xc5, 0x86, 0xe4, 0x03, 0x31, 0xef, 0x5c, 0xb2, 0x21, 0xc9, - 0x22, 0xae, 0xa8, 0xf2, 0x2f, 0xea, 0x8e, 0x7d, 0x26, 0xea, 0x48, 0xd6, 0xe5, 0x19, 0xac, 0xd3, - 0x3f, 0x10, 0xa1, 0x3d, 0x68, 0xc8, 0xa5, 0x68, 0x5b, 0x94, 0x98, 0xb5, 0xb1, 0xf1, 0xec, 0x84, - 0x18, 0x4a, 0xb4, 0xa0, 0x39, 0x9e, 0x5d, 0xa2, 0x2d, 0xd9, 0xeb, 0xf9, 0xec, 0xc2, 0xa3, 0xa4, - 0x47, 0xcd, 0xf5, 0x4b, 0xd4, 0xd9, 0x4c, 0xe9, 0x70, 0x96, 0x09, 0xed, 0x40, 0x3d, 0xb3, 0x24, - 0xe6, 0x8f, 0xc7, 0x66, 0xd5, 0x33, 0x84, 0x10, 0x3c, 0xc2, 0x26, 0x7c, 0x3a, 0x50, 0x05, 0xd7, - 0xdc, 0x18, 0xf3, 0x69, 0x5d, 0x88, 0x71, 0x4c, 0x80, 0xd6, 0xc1, 0x08, 0xe2, 0xe2, 0x6c, 0x7e, - 0x3a, 0x36, 0xbc, 0x4c, 0xca, 0x36, 0x4e, 0x89, 0x50, 0x07, 0x0c, 0xd5, 0xe2, 0xf9, 0xec, 0xc2, - 0xfc, 0x4c, 0xf7, 0x5f, 0xc9, 0x40, 0x4f, 0x27, 0x6e, 0x5c, 0x75, 0xf4, 0x93, 0x88, 0x1a, 0x1e, - 0xb1, 0xc0, 0xfc, 0xf9, 0x58, 0xd4, 0x88, 0x7c, 0x8e, 0x25, 0x4a, 0x28, 0xcc, 0x55, 0x76, 0x37, - 0xef, 0x8d, 0x29, 0xac, 0xb3, 0x3e, 0x8e, 0x09, 0x5a, 0x7f, 0x9c, 0x4b, 0xa6, 0xe1, 0xaf, 0x3f, - 0x78, 0xfd, 0x2a, 0x5b, 0xf2, 0x32, 0x5d, 0x5e, 0xfe, 0x92, 0x2e, 0xef, 0x5a, 0xc2, 0x91, 0x99, - 0xa1, 0xff, 0x04, 0x8a, 0x22, 0xc0, 0xd0, 0x1d, 0xa8, 0x26, 0xbd, 0x68, 0x6e, 0x56, 0x2f, 0x9a, - 0x90, 0xb4, 0x7e, 0x9b, 0x87, 0xb2, 0x0a, 0x4c, 0xf4, 0xe5, 0xc4, 0x58, 0xf4, 0xc3, 0x4b, 0xe2, - 0x78, 0x72, 0x2a, 0xaa, 0x5a, 0x17, 0x39, 0x96, 0x0b, 0x2d, 0xf5, 0x85, 0xf8, 0x78, 0x18, 0x51, - 0x75, 0x19, 0x29, 0x8a, 0xd6, 0x45, 0xe1, 0x9e, 0x08, 0xd4, 0x96, 0xc0, 0xb4, 0xfe, 0x27, 0x97, - 0xce, 0x51, 0x17, 0xa1, 0xa4, 0x66, 0x3b, 0xaa, 0x24, 0xab, 0x05, 0x5a, 0x81, 0x66, 0xdf, 0xf5, - 0x2d, 0xce, 0x06, 0xa1, 0x33, 0x7a, 0xa3, 0x6e, 0xf4, 0x5d, 0xff, 0x50, 0x82, 0x55, 0x07, 0xbf, - 0xa2, 0xae, 0xfe, 0x23, 0x94, 0x05, 0x4d, 0x69, 0x3f, 0xcf, 0x52, 0xae, 0x02, 0x52, 0x54, 0xc4, - 0x22, 0xcc, 0xe1, 0x56, 0xc4, 0x22, 0xdb, 0x93, 0x35, 0xaa, 0x88, 0x9b, 0x1a, 0xb3, 0xcd, 0x1c, - 0x7e, 0x24, 0xe0, 0xa8, 0x03, 0xd7, 0x62, 0x6a, 0xb9, 0x1d, 0x4d, 0x5e, 0x92, 0xe4, 0x0b, 0x1a, - 0x25, 0xb7, 0xa3, 0xe8, 0xdb, 0x30, 0xa7, 0xfb, 0x13, 0x8b, 0x50, 0x2f, 0xd2, 0x3f, 0xb2, 0xc0, - 0x35, 0xd5, 0x88, 0x6c, 0x0b, 0x50, 0xeb, 0x0b, 0x28, 0xc9, 0x2c, 0x75, 0x49, 0x07, 0x96, 0x9b, - 0xde, 0x81, 0xb5, 0xfe, 0x37, 0x97, 0xce, 0xd9, 0x2f, 0x1b, 0x64, 0x4f, 0xc9, 0x88, 0x53, 0x8f, - 0xec, 0x35, 0x3b, 0xc0, 0xd6, 0xf0, 0xbb, 0x4e, 0xec, 0x16, 0x2c, 0xa8, 0x0c, 0x9f, 0x35, 0xae, - 0x72, 0x81, 0x79, 0x85, 0x48, 0x6d, 0xbb, 0x0a, 0x48, 0xd3, 0x66, 0x4d, 0x5b, 0x50, 0x27, 0xa1, - 0x30, 0xa9, 0x65, 0x5b, 0x15, 0x28, 0xc9, 0x94, 0xdb, 0xfa, 0xc7, 0x1c, 0x94, 0x55, 0xf2, 0x7d, - 0x65, 0xa7, 0x55, 0xe4, 0x53, 0x46, 0xf9, 0xaf, 0xb2, 0x1f, 0x95, 0xe0, 0xa7, 0xec, 0x47, 0x21, - 0x46, 0xf6, 0xa3, 0x69, 0xa7, 0xec, 0x47, 0x61, 0x32, 0xfb, 0xf9, 0x93, 0xdc, 0xe8, 0xaf, 0x01, - 0x5e, 0xdb, 0x19, 0xbe, 0xbf, 0xec, 0xb1, 0x09, 0x73, 0x23, 0xb5, 0xe4, 0x0a, 0x8e, 0xf9, 0x25, - 0xd4, 0x32, 0x15, 0xe0, 0x0a, 0x02, 0xee, 0x43, 0x3d, 0x5b, 0x42, 0x5e, 0x5f, 0xc2, 0xad, 0xdb, - 0x50, 0x92, 0x3f, 0x4a, 0x44, 0x00, 0xe5, 0x83, 0x27, 0x5b, 0xfb, 0x7b, 0xdd, 0xe6, 0x5b, 0xa8, - 0x06, 0x95, 0x03, 0xbc, 0xf7, 0x74, 0xf3, 0x68, 0xa7, 0x99, 0x43, 0x06, 0x94, 0xf6, 0x1f, 0x77, - 0x37, 0xf7, 0x9b, 0xf9, 0x8d, 0x07, 0x50, 0xd5, 0x3f, 0x1a, 0x0b, 0xd1, 0x2f, 0xa0, 0xa2, 0x9f, - 0xd1, 0xdb, 0x69, 0xa2, 0x1f, 0xf9, 0x39, 0x63, 0xcb, 0x9c, 0x44, 0xa8, 0x06, 0x76, 0x3d, 0xb7, - 0xb1, 0x0f, 0x55, 0xfd, 0x41, 0x22, 0x44, 0xf7, 0xa1, 0xa2, 0x9f, 0x33, 0xb2, 0x46, 0x3f, 0x2b, - 0x65, 0x64, 0x8d, 0x7d, 0xc7, 0x58, 0xc9, 0xad, 0xe7, 0x36, 0x4e, 0xa1, 0x31, 0x3a, 0xea, 0x47, - 0x4f, 0x61, 0x5e, 0x3e, 0x24, 0x60, 0x8e, 0x6e, 0x66, 0xeb, 0xca, 0xe4, 0x07, 0x83, 0xd6, 0xd2, - 0x4c, 0x7c, 0xe6, 0x4d, 0x5f, 0x41, 0x79, 0x5f, 0xfd, 0xb6, 0xed, 0xde, 0xe8, 0x45, 0x63, 0x71, - 0x5a, 0x10, 0xb5, 0xa6, 0x42, 0xa5, 0xa0, 0xdd, 0x78, 0xf6, 0xfd, 0x66, 0x72, 0xb6, 0x7e, 0xf1, - 0xed, 0x7f, 0xdc, 0x7c, 0xeb, 0xdb, 0xdf, 0xdc, 0xcc, 0xfd, 0xf3, 0x6f, 0x6e, 0xe6, 0xfe, 0xf2, - 0x3f, 0x6f, 0xe6, 0xfe, 0x68, 0xf5, 0x95, 0x7e, 0x98, 0xa6, 0xe5, 0x1d, 0x97, 0x25, 0xe8, 0xd3, - 0xdf, 0x05, 0x00, 0x00, 0xff, 0xff, 0x8b, 0xc4, 0xc3, 0xd1, 0x81, 0x2c, 0x00, 0x00, + // 3774 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x3a, 0x3b, 0x70, 0x1c, 0x47, + 0x76, 0xda, 0xff, 0xce, 0xdb, 0xc5, 0x62, 0xb7, 0x09, 0x52, 0xa3, 0x95, 0x44, 0x40, 0x2b, 0xc9, + 0xc2, 0xf1, 0xb3, 0xc0, 0x41, 0xba, 0x3b, 0x49, 0x75, 0x3c, 0x11, 0x58, 0x00, 0x12, 0x78, 0x20, + 0x09, 0x37, 0x40, 0x96, 0xed, 0x64, 0x6a, 0x30, 0xdd, 0x58, 0x0c, 0x31, 0x3b, 0x3d, 0x9a, 0x9e, + 0x05, 0xb8, 0x97, 0x3a, 0x71, 0xd5, 0x05, 0x4e, 0x9c, 0x39, 0xb1, 0x23, 0x57, 0xb9, 0xec, 0xc0, + 0xe9, 0x05, 0x8e, 0x1c, 0x28, 0x74, 0xb9, 0x5c, 0x2e, 0x47, 0xac, 0xf2, 0x39, 0x75, 0x76, 0x76, + 0x95, 0xcd, 0x72, 0x70, 0xd5, 0x9f, 0xf9, 0xec, 0x0f, 0x22, 0x21, 0x05, 0x0a, 0xc8, 0x9d, 0x7e, + 0xbf, 0x79, 0xfd, 0xfa, 0xfd, 0xfa, 0x0d, 0xa0, 0xd3, 0x67, 0x6b, 0x41, 0xc8, 0x22, 0xe6, 0x30, + 0x8f, 0xaf, 0x85, 0x43, 0x3f, 0x72, 0x07, 0x34, 0xfe, 0xed, 0x4a, 0x0c, 0xaa, 0xe8, 0x65, 0xfb, + 0xe6, 0x71, 0xc8, 0xce, 0x68, 0x98, 0x30, 0x24, 0x0f, 0x8a, 0xb0, 0xbd, 0xe2, 0x30, 0x9f, 0x0f, + 0x07, 0x97, 0x51, 0x8c, 0xbd, 0x8e, 0xd0, 0xd0, 0x3d, 0xa7, 0xfa, 0x47, 0x53, 0xbc, 0x33, 0x46, + 0x71, 0xe2, 0xb1, 0x0b, 0xf9, 0x9f, 0xc6, 0xde, 0x1a, 0xc3, 0x0e, 0xec, 0x88, 0x86, 0xae, 0xed, + 0xb9, 0xbf, 0xa2, 0xd9, 0x67, 0x4d, 0xdb, 0x1e, 0xa3, 0x65, 0x81, 0xfc, 0xa7, 0x71, 0xe3, 0xdb, + 0xe6, 0xa7, 0xc3, 0x93, 0x13, 0x8f, 0xc6, 0xbf, 0x9a, 0x66, 0xa9, 0xcf, 0xfa, 0x4c, 0x3e, 0xae, + 0x89, 0x27, 0x05, 0xed, 0xfc, 0x63, 0x0e, 0x5a, 0x47, 0x36, 0x3f, 0x3b, 0xa4, 0xe1, 0xb9, 0xeb, + 0xd0, 0x1e, 0xf3, 0x4f, 0xdc, 0x3e, 0xba, 0x09, 0x35, 0x8f, 0xf5, 0xad, 0x13, 0xd7, 0xa3, 0xd6, + 0x09, 0x31, 0x73, 0x2b, 0xb9, 0xd5, 0x12, 0x36, 0x3c, 0xd6, 0xdf, 0x75, 0x3d, 0xba, 0x4b, 0xd0, + 0xdb, 0x60, 0x44, 0x36, 0x3f, 0xb3, 0x7c, 0x7b, 0x40, 0xcd, 0xfc, 0x4a, 0x6e, 0xd5, 0xc0, 0x55, + 0x01, 0x78, 0x64, 0x0f, 0x28, 0x7a, 0x0b, 0xaa, 0x43, 0xc2, 0xad, 0xc0, 0x8e, 0x4e, 0xcd, 0x82, + 0xc4, 0x55, 0x86, 0x84, 0x1f, 0xd8, 0xd1, 0x29, 0xba, 0x0d, 0x2d, 0x87, 0xf9, 0x91, 0xed, 0xfa, + 0x34, 0xb4, 0x7c, 0x1a, 0x5d, 0xb0, 0xf0, 0xcc, 0x2c, 0x4a, 0x9a, 0x66, 0x82, 0x78, 0xa4, 0xe0, + 0xe8, 0x03, 0x28, 0x05, 0x9e, 0xed, 0x53, 0xb3, 0xbc, 0x92, 0x5b, 0x6d, 0x6c, 0x34, 0xba, 0xf1, + 0x31, 0x1e, 0x08, 0x28, 0x56, 0xc8, 0xce, 0xff, 0x15, 0xa1, 0x71, 0xa8, 0x36, 0x8a, 0xe9, 0xd7, + 0x43, 0xca, 0x23, 0xb4, 0x07, 0x95, 0x67, 0x6c, 0x18, 0xfa, 0xb6, 0x27, 0x35, 0x37, 0xb6, 0xd6, + 0x5e, 0xbe, 0x58, 0xbe, 0xdd, 0x67, 0xdd, 0xbe, 0xfd, 0x2b, 0x1a, 0x45, 0xb4, 0x4b, 0xe8, 0xf9, + 0x9a, 0xc3, 0x42, 0xba, 0x36, 0xe1, 0x00, 0xdd, 0x07, 0x8a, 0x0d, 0xc7, 0xfc, 0xe8, 0x06, 0x94, + 0x43, 0x1a, 0x78, 0xf6, 0x48, 0xee, 0xb2, 0x8a, 0xf5, 0x4a, 0xec, 0xf1, 0x78, 0xe8, 0x7a, 0xc4, + 0x72, 0x49, 0xbc, 0x47, 0xb9, 0xde, 0x23, 0x68, 0x17, 0xca, 0xec, 0xe4, 0x84, 0xd3, 0x48, 0x6e, + 0xac, 0xb0, 0xd5, 0x7d, 0xf9, 0x62, 0xf9, 0xd6, 0xab, 0xbc, 0xfc, 0xb1, 0xe4, 0xc2, 0x9a, 0x1b, + 0x3d, 0x04, 0xa0, 0x3e, 0xb1, 0xb4, 0xac, 0xd2, 0x95, 0x64, 0x19, 0xd4, 0x27, 0xea, 0x11, 0xdd, + 0x86, 0x52, 0x68, 0xfb, 0x7d, 0x65, 0xcd, 0xda, 0xc6, 0x62, 0x57, 0xba, 0x21, 0x16, 0xa0, 0xc3, + 0x80, 0x3a, 0x5b, 0xc5, 0x6f, 0x5e, 0x2c, 0xbf, 0x81, 0x15, 0x0d, 0x3a, 0x84, 0x9a, 0xc3, 0x58, + 0x48, 0x5c, 0xdf, 0x8e, 0x58, 0x68, 0x56, 0xa4, 0x15, 0x7f, 0xfc, 0xf2, 0xc5, 0xf2, 0xdd, 0x59, + 0x2f, 0x9f, 0x0a, 0x93, 0xee, 0xe1, 0xa9, 0x1d, 0x92, 0xbd, 0x6d, 0x9c, 0x95, 0x82, 0xd6, 0x01, + 0x42, 0xca, 0x99, 0x37, 0x8c, 0x5c, 0xe6, 0x9b, 0x55, 0xa9, 0x46, 0xb3, 0x9b, 0xf0, 0x7c, 0x45, + 0x6d, 0x42, 0x43, 0x9c, 0xa1, 0x41, 0xef, 0xc3, 0x82, 0xf6, 0x61, 0xcb, 0xf5, 0x09, 0x7d, 0x6e, + 0x1a, 0x2b, 0xb9, 0xd5, 0x05, 0x5c, 0xd7, 0xc0, 0x3d, 0x01, 0x43, 0x9f, 0x00, 0xc8, 0x88, 0xb3, + 0xa5, 0x58, 0x90, 0x62, 0x97, 0xd4, 0xee, 0x7a, 0xcc, 0xf3, 0xa8, 0x23, 0xe0, 0x62, 0x8b, 0x38, + 0x43, 0x87, 0x7a, 0xb0, 0x98, 0x86, 0x98, 0x62, 0xad, 0x49, 0xd6, 0xb7, 0x14, 0xeb, 0xc3, 0x71, + 0xa4, 0xe4, 0x9f, 0xe4, 0xe8, 0xfc, 0x4b, 0x11, 0x16, 0x13, 0xdf, 0xe3, 0x01, 0xf3, 0x39, 0x45, + 0xab, 0x50, 0xe6, 0x91, 0x1d, 0x0d, 0xb9, 0xf4, 0xbd, 0xc6, 0x46, 0xb3, 0x1b, 0x9b, 0xa7, 0x7b, + 0x28, 0xe1, 0x58, 0xe3, 0x05, 0xe5, 0xa9, 0xdc, 0xb3, 0xf4, 0xad, 0x59, 0xb6, 0xd0, 0x78, 0xf4, + 0x21, 0x34, 0x22, 0x1a, 0x0e, 0x5c, 0xdf, 0xf6, 0x2c, 0x1a, 0x86, 0x2c, 0xd4, 0x3e, 0xb7, 0x10, + 0x43, 0x77, 0x04, 0x10, 0xfd, 0x21, 0xd4, 0x43, 0x6a, 0x13, 0x2b, 0x3a, 0x0d, 0xd9, 0xb0, 0x7f, + 0x7a, 0x45, 0xff, 0xab, 0x09, 0x19, 0x47, 0x4a, 0x84, 0x70, 0xc2, 0x8b, 0xd0, 0x8d, 0xa8, 0x25, + 0x34, 0xb9, 0xaa, 0x13, 0x4a, 0x09, 0x62, 0x4b, 0x68, 0x0f, 0x4a, 0x76, 0x48, 0x7d, 0x5b, 0x3a, + 0x61, 0x7d, 0xeb, 0xe3, 0x97, 0x2f, 0x96, 0xd7, 0xfa, 0x6e, 0x74, 0x3a, 0x3c, 0xee, 0x3a, 0x6c, + 0xb0, 0x46, 0x79, 0x34, 0xb4, 0xc3, 0x91, 0x4a, 0x93, 0x53, 0x89, 0xb3, 0xbb, 0x29, 0x58, 0xb1, + 0x92, 0x80, 0x3e, 0x84, 0x22, 0x61, 0x0e, 0x37, 0x2b, 0x2b, 0x85, 0xd5, 0xda, 0x46, 0x4d, 0x9d, + 0xda, 0xa1, 0xe7, 0x3a, 0x54, 0xbb, 0xb2, 0x44, 0xa3, 0xaf, 0xa0, 0xa2, 0x22, 0x88, 0x9b, 0xd5, + 0x95, 0xc2, 0x15, 0xb4, 0x8f, 0xd9, 0x85, 0x9f, 0x0d, 0x87, 0x2e, 0xb1, 0x02, 0x3b, 0x8c, 0xb8, + 0x69, 0xc8, 0xd7, 0xea, 0x28, 0x7a, 0xf2, 0x64, 0x6f, 0xfb, 0x40, 0x80, 0xf5, 0xab, 0x0d, 0x41, + 0x28, 0x01, 0xc2, 0xe9, 0x03, 0xdb, 0x39, 0xa3, 0xc4, 0x3a, 0xa3, 0x23, 0x13, 0xe6, 0x29, 0x6b, + 0x28, 0xa2, 0x5f, 0xd2, 0x51, 0x87, 0x40, 0x0b, 0x33, 0xe7, 0x8c, 0x6f, 0x6f, 0x6d, 0x53, 0xee, + 0x84, 0x6e, 0x20, 0x62, 0xe7, 0x0e, 0xa0, 0x50, 0x00, 0xc9, 0xb1, 0x45, 0xfd, 0x73, 0x6b, 0x40, + 0x07, 0x41, 0x14, 0x4a, 0x0f, 0x2b, 0xe3, 0xa6, 0xc6, 0xec, 0xf8, 0xe7, 0x0f, 0x25, 0x1c, 0xbd, + 0x07, 0xf5, 0x98, 0x5a, 0x66, 0x61, 0x95, 0xa1, 0x6b, 0x1a, 0x26, 0x32, 0x71, 0xe7, 0x2f, 0xf2, + 0x60, 0xf4, 0xe2, 0x8c, 0x8b, 0xde, 0x84, 0x8a, 0x1b, 0x58, 0x36, 0x21, 0x4a, 0xa6, 0x81, 0xcb, + 0x6e, 0xb0, 0x49, 0x48, 0x88, 0x7e, 0x0a, 0x0b, 0x3a, 0x4d, 0x5b, 0x01, 0x13, 0xfb, 0xce, 0xcb, + 0x1d, 0xb4, 0xd4, 0x0e, 0x74, 0xa6, 0x3e, 0x60, 0x61, 0x84, 0xeb, 0x7e, 0xba, 0xe0, 0xe8, 0x10, + 0x5a, 0x03, 0x3b, 0x08, 0x28, 0xb1, 0x4e, 0x19, 0x8f, 0x34, 0x6f, 0x41, 0xf2, 0x7e, 0x94, 0xe4, + 0xf1, 0xe4, 0xfd, 0xdd, 0x87, 0x92, 0xf6, 0x2b, 0xc6, 0x23, 0xc9, 0xbe, 0xe3, 0x47, 0xe1, 0x48, + 0x84, 0xdb, 0x18, 0x14, 0xbd, 0x0b, 0x30, 0xe4, 0x76, 0x9f, 0x5a, 0xa1, 0x1d, 0x51, 0xe9, 0xdd, + 0x79, 0x6c, 0x48, 0x08, 0xb6, 0x23, 0xda, 0xde, 0x82, 0xa5, 0x59, 0x72, 0x50, 0x13, 0x0a, 0xc2, + 0xf6, 0x39, 0x99, 0x3b, 0xc4, 0x23, 0x5a, 0x82, 0xd2, 0xb9, 0xed, 0x0d, 0xe3, 0xd2, 0xa5, 0x16, + 0x9f, 0xe7, 0x3f, 0xcd, 0x75, 0xfe, 0x36, 0x0f, 0xad, 0x9e, 0x1d, 0x44, 0xc3, 0x30, 0xae, 0x26, + 0x3b, 0xcf, 0x45, 0xee, 0x14, 0xb5, 0xcf, 0xf2, 0xe8, 0x39, 0xf5, 0x74, 0x58, 0x37, 0xba, 0xa2, + 0xfa, 0xee, 0xb3, 0x7e, 0x77, 0x5f, 0x40, 0x71, 0xd5, 0x63, 0x7d, 0xf9, 0x84, 0xf6, 0xd2, 0xa3, + 0x22, 0xc9, 0x01, 0xea, 0x10, 0x6f, 0x27, 0x7b, 0x9f, 0x3a, 0x62, 0xdc, 0xd2, 0x5c, 0x99, 0x53, + 0xdf, 0x83, 0x3a, 0x8f, 0xec, 0x30, 0xb2, 0x1c, 0x36, 0x18, 0xb8, 0x91, 0x8c, 0xfa, 0xda, 0xc6, + 0x1f, 0xa4, 0x06, 0x9c, 0xd4, 0x54, 0xa4, 0x98, 0x30, 0xea, 0x49, 0x6a, 0x5c, 0xe3, 0xe9, 0xa2, + 0x8d, 0xa1, 0x96, 0xc1, 0xa1, 0x1e, 0x20, 0x2d, 0xc4, 0x72, 0x4e, 0xa9, 0x73, 0x16, 0x30, 0xd7, + 0x8f, 0xe4, 0xd6, 0x44, 0xf2, 0x4c, 0x32, 0x56, 0x2f, 0xc1, 0xe1, 0x96, 0xa6, 0x4f, 0x41, 0x9d, + 0xff, 0x2f, 0x02, 0x4a, 0x54, 0x50, 0xe9, 0x4f, 0x58, 0x6b, 0x1d, 0x8c, 0xa4, 0x96, 0x6b, 0x91, + 0x68, 0xfa, 0xcc, 0x71, 0x4a, 0x84, 0x3e, 0x87, 0x32, 0x0b, 0xa8, 0x4f, 0x89, 0x36, 0x53, 0x67, + 0x7a, 0x87, 0x89, 0xf8, 0xee, 0x63, 0x49, 0x89, 0x35, 0x07, 0xba, 0x0f, 0x55, 0x47, 0x11, 0x11, + 0x6d, 0x9f, 0x0f, 0x2e, 0xe3, 0xd6, 0x20, 0x82, 0x13, 0x2e, 0xb4, 0x0b, 0x90, 0xb1, 0x41, 0x71, + 0x9e, 0x8d, 0x33, 0x32, 0x52, 0xab, 0x64, 0x38, 0xdb, 0x0f, 0xa1, 0xac, 0x74, 0xfb, 0x5e, 0xac, + 0xdb, 0x7e, 0x0a, 0xd5, 0x58, 0x59, 0xe1, 0xf9, 0x67, 0x74, 0x64, 0xa9, 0x24, 0x21, 0x05, 0xd5, + 0xb1, 0x71, 0x46, 0x47, 0x07, 0x12, 0x20, 0xda, 0x2a, 0x91, 0x95, 0x5c, 0x51, 0x94, 0x78, 0x4c, + 0x95, 0x97, 0x54, 0xcd, 0x14, 0xa1, 0x88, 0xdb, 0x17, 0x00, 0xe9, 0x5b, 0xd0, 0x0a, 0x94, 0x44, + 0x39, 0xe2, 0x5a, 0x3b, 0x90, 0x6e, 0x2d, 0x0a, 0x15, 0xc7, 0x0a, 0x81, 0xbe, 0x84, 0x5a, 0xc0, + 0x3c, 0xcf, 0x0a, 0x29, 0x1f, 0x7a, 0x91, 0x14, 0xdb, 0xb8, 0xdc, 0x3e, 0x07, 0xcc, 0xf3, 0xb0, + 0xa4, 0xc6, 0x10, 0x24, 0xcf, 0x9d, 0x47, 0x00, 0x29, 0x06, 0xd5, 0xa0, 0xb2, 0xf7, 0xe8, 0xe9, + 0xe6, 0xfe, 0xde, 0x76, 0xf3, 0x0d, 0x64, 0x40, 0x09, 0xef, 0x6c, 0x6e, 0xff, 0x71, 0x33, 0x87, + 0x16, 0xc0, 0x78, 0xf4, 0xf8, 0xc8, 0x52, 0xcb, 0x3c, 0xaa, 0x43, 0xb5, 0xf7, 0xf8, 0xf1, 0xbe, + 0xf5, 0x78, 0x77, 0xb7, 0x59, 0x10, 0x4c, 0x78, 0xe7, 0xf0, 0x68, 0x13, 0x1f, 0x35, 0x8b, 0x9d, + 0xff, 0xca, 0x41, 0x73, 0x5b, 0xf6, 0xda, 0x3f, 0x80, 0x50, 0xdd, 0x80, 0xa2, 0x70, 0x48, 0xed, + 0x82, 0x37, 0x13, 0xe6, 0x49, 0x05, 0xa5, 0xfb, 0x62, 0x49, 0xdb, 0xbe, 0x03, 0x45, 0xb1, 0x42, + 0x1f, 0x40, 0x83, 0x7f, 0xed, 0x89, 0x2a, 0x7b, 0x7e, 0xc2, 0xad, 0x61, 0xe8, 0xea, 0x24, 0x5c, + 0x57, 0xd0, 0xa7, 0x27, 0xfc, 0x49, 0xe8, 0x76, 0xfe, 0xbb, 0x00, 0xad, 0x58, 0xda, 0x77, 0x09, + 0xb6, 0xcf, 0x26, 0x82, 0xed, 0xbd, 0x29, 0x5d, 0xe7, 0xc6, 0xda, 0x16, 0x18, 0xc1, 0xf0, 0xd8, + 0x73, 0xf9, 0xe9, 0x8c, 0x60, 0x9b, 0xe6, 0x3e, 0x88, 0x69, 0x71, 0xca, 0x86, 0x7e, 0x0e, 0x95, + 0x13, 0x6f, 0x28, 0x25, 0x14, 0x27, 0x82, 0x7d, 0x5a, 0xc2, 0xae, 0xa2, 0xc4, 0x31, 0xcb, 0xf7, + 0x1d, 0x63, 0x11, 0x18, 0x89, 0x92, 0xe2, 0x52, 0x33, 0xb0, 0x9f, 0x5b, 0x8e, 0xc7, 0x9c, 0x33, + 0x5d, 0x5a, 0xab, 0x03, 0xfb, 0x79, 0x4f, 0xac, 0x27, 0x22, 0x30, 0xff, 0x4a, 0x11, 0x58, 0x98, + 0x13, 0x81, 0xb7, 0xa1, 0xa2, 0x37, 0xf6, 0xed, 0xe1, 0xd7, 0xf9, 0xf3, 0x1c, 0x5c, 0x4f, 0x9b, + 0xd1, 0x1f, 0x80, 0xab, 0x77, 0x7e, 0x93, 0x83, 0x1b, 0x63, 0x1a, 0x7d, 0x17, 0x6f, 0xdc, 0x4c, + 0xdd, 0x41, 0x29, 0x93, 0xb6, 0x07, 0xb3, 0xdf, 0x31, 0xed, 0x13, 0xaf, 0x65, 0xce, 0xdf, 0x14, + 0xa1, 0xd1, 0x63, 0x83, 0x63, 0xd7, 0x4f, 0xae, 0x8b, 0xeb, 0x3a, 0x74, 0x15, 0xcf, 0x3b, 0x19, + 0x7d, 0xb3, 0x64, 0x99, 0xc0, 0x45, 0x77, 0xa1, 0x60, 0x93, 0x58, 0xe1, 0xb7, 0xe7, 0x31, 0x6c, + 0x12, 0x82, 0x05, 0x5d, 0xfb, 0xdf, 0xf2, 0x3a, 0xd0, 0xef, 0x43, 0xf5, 0xd8, 0xf5, 0x89, 0xeb, + 0xf7, 0x85, 0x86, 0x85, 0xf1, 0x5a, 0x35, 0xfd, 0xb6, 0xee, 0x96, 0x22, 0xc6, 0x09, 0x57, 0xfb, + 0xd7, 0x79, 0xa8, 0x68, 0x28, 0x42, 0x50, 0x3c, 0x19, 0x7a, 0xea, 0xe8, 0xab, 0x58, 0x3e, 0xc7, + 0xbd, 0x8e, 0xe8, 0xd2, 0x0c, 0xd5, 0xeb, 0x7c, 0x0a, 0xb5, 0x20, 0x64, 0xcf, 0xd4, 0x35, 0x28, + 0xee, 0xc1, 0x9a, 0xaa, 0x7f, 0x3b, 0x48, 0x10, 0xba, 0x0d, 0xcd, 0x92, 0xa2, 0x7b, 0x50, 0xe3, + 0xce, 0x29, 0x1d, 0xd8, 0xd6, 0x33, 0xce, 0x7c, 0x19, 0xad, 0xf5, 0xad, 0x77, 0x5e, 0xbe, 0x58, + 0x36, 0xa9, 0xef, 0x30, 0xa1, 0xc2, 0x9a, 0x40, 0x74, 0xb1, 0x7d, 0xf1, 0x90, 0x72, 0xd9, 0x86, + 0x81, 0x62, 0x78, 0xc0, 0x99, 0x8f, 0xba, 0x00, 0x9c, 0x86, 0x56, 0xc0, 0x3c, 0xd7, 0x19, 0xc9, + 0xab, 0x43, 0xd2, 0x2f, 0x1f, 0xd2, 0xf0, 0x40, 0x82, 0xb1, 0xc1, 0xe3, 0x47, 0x39, 0x36, 0x90, + 0xfd, 0x75, 0x14, 0xca, 0xeb, 0x81, 0x81, 0x2b, 0xb2, 0x8d, 0x8e, 0x42, 0x71, 0x0b, 0x97, 0x2d, + 0x9a, 0xea, 0xf6, 0x0d, 0xac, 0x57, 0x6d, 0x1f, 0x0a, 0x9b, 0x84, 0x20, 0x13, 0x2a, 0xda, 0x40, + 0xba, 0xc9, 0x8b, 0x97, 0xe8, 0x67, 0x50, 0x25, 0xcc, 0x51, 0xfa, 0xe7, 0x5f, 0x41, 0xff, 0x0a, + 0x61, 0x8e, 0x54, 0x7e, 0x09, 0x4a, 0x27, 0x21, 0xf3, 0x55, 0xcb, 0x55, 0xc5, 0x6a, 0xd1, 0xf9, + 0xf7, 0x1c, 0x2c, 0x26, 0xe7, 0xa4, 0xef, 0x7b, 0xf3, 0x5f, 0x6e, 0x42, 0x85, 0x50, 0x8f, 0x46, + 0xda, 0xb5, 0xab, 0x38, 0x5e, 0x8e, 0xa9, 0x55, 0xb8, 0x92, 0x5a, 0xc5, 0x8c, 0x5a, 0x13, 0xb9, + 0xa9, 0x34, 0x99, 0x9b, 0xde, 0x87, 0x05, 0x65, 0xaf, 0x98, 0x42, 0x5e, 0xbe, 0x70, 0x5d, 0x01, + 0x15, 0x51, 0xe7, 0x4d, 0xb8, 0xde, 0x63, 0xbe, 0x4f, 0x9d, 0x88, 0x85, 0x07, 0x21, 0x7b, 0x3e, + 0xd2, 0x8e, 0xd8, 0xf9, 0xcb, 0x1c, 0xdc, 0x98, 0xc4, 0xe8, 0xad, 0x3f, 0x80, 0x8a, 0xb8, 0x32, + 0x50, 0xce, 0xf5, 0x9c, 0x65, 0xfd, 0xe5, 0x8b, 0xe5, 0x3b, 0xaf, 0x72, 0xb7, 0xda, 0xf1, 0x89, + 0xca, 0xc9, 0xb1, 0x00, 0x71, 0xfa, 0x81, 0x10, 0x6e, 0xb9, 0x44, 0x77, 0xe5, 0x15, 0xb9, 0xde, + 0x23, 0xa8, 0x0d, 0x05, 0x8f, 0xf5, 0x75, 0xbd, 0xa9, 0xc6, 0x19, 0x0e, 0x0b, 0x60, 0xe7, 0x1f, + 0x0a, 0x50, 0x7c, 0xc0, 0x5c, 0x1f, 0xdd, 0x82, 0x16, 0x8d, 0x1c, 0x62, 0x0d, 0x18, 0xb1, 0x42, + 0x7a, 0xee, 0x72, 0x71, 0xa3, 0x17, 0x5a, 0x15, 0xf0, 0xa2, 0x40, 0x3c, 0x64, 0x04, 0x6b, 0x30, + 0xba, 0x0d, 0x65, 0x7e, 0x6a, 0x87, 0x24, 0xbe, 0xcd, 0x5c, 0x4b, 0x82, 0x50, 0x88, 0x52, 0xc3, + 0x0b, 0xac, 0x49, 0xd0, 0x32, 0xd4, 0xe4, 0x93, 0x9e, 0x40, 0x14, 0xe4, 0x19, 0x83, 0x04, 0xa9, + 0xf9, 0xc3, 0x6d, 0x68, 0xc5, 0x43, 0x0a, 0xe2, 0x86, 0xd2, 0x4c, 0xa3, 0x78, 0xa6, 0xa5, 0x11, + 0xdb, 0x31, 0x1c, 0xfd, 0x08, 0x62, 0x98, 0x45, 0xb5, 0x0d, 0xe4, 0x81, 0x19, 0x78, 0x51, 0xc3, + 0x63, 0xd3, 0xa0, 0x8f, 0x60, 0xd1, 0x93, 0xd7, 0xff, 0x94, 0x52, 0x85, 0x45, 0x43, 0x81, 0x63, + 0xc2, 0xf6, 0xdf, 0xe7, 0xa0, 0x24, 0x75, 0x46, 0x0d, 0xc8, 0xbb, 0x44, 0x37, 0x0f, 0x79, 0x97, + 0xa0, 0x2e, 0x54, 0x3d, 0xfb, 0x98, 0x7a, 0xc2, 0x39, 0xf3, 0x3a, 0x1b, 0xcb, 0x8c, 0x28, 0xa8, + 0xf7, 0x35, 0x06, 0x27, 0x34, 0x68, 0x03, 0x2a, 0x21, 0xb5, 0x85, 0xa6, 0xda, 0xda, 0x66, 0x3a, + 0x92, 0x38, 0x08, 0x99, 0x43, 0x39, 0x3f, 0x0c, 0xa8, 0xd3, 0xdd, 0xdb, 0xc6, 0x31, 0x21, 0x5a, + 0x87, 0x25, 0x69, 0x78, 0x27, 0xa4, 0x76, 0x44, 0x53, 0xdb, 0xcb, 0xe1, 0x03, 0x46, 0x02, 0xd7, + 0x93, 0xa8, 0xd8, 0xfc, 0x9d, 0x4f, 0xa0, 0x2c, 0xec, 0x4c, 0x89, 0x38, 0x34, 0x51, 0x71, 0x25, + 0xff, 0xe4, 0xa1, 0x0d, 0xec, 0xe7, 0x3b, 0x91, 0x93, 0x1c, 0x5a, 0xe7, 0xd7, 0x39, 0x28, 0x1e, + 0xd9, 0xfc, 0x4c, 0xa4, 0x3d, 0x1e, 0x50, 0x47, 0x77, 0xc1, 0xf2, 0x59, 0x08, 0x62, 0x01, 0xb7, + 0x64, 0x8a, 0xb7, 0xe2, 0xd9, 0x9f, 0x72, 0xa3, 0x45, 0x16, 0x70, 0x59, 0x01, 0xf4, 0x6c, 0x4f, + 0x84, 0x65, 0x20, 0x5e, 0x46, 0x2f, 0x74, 0x14, 0xc5, 0x4b, 0x71, 0x38, 0x42, 0x9d, 0x28, 0xb4, + 0x7d, 0x6e, 0xeb, 0x7c, 0x59, 0x92, 0xe7, 0x2d, 0xb4, 0x39, 0xca, 0x80, 0x3b, 0x7f, 0x5d, 0x82, + 0x0a, 0xa6, 0x0e, 0x3b, 0x97, 0x25, 0xac, 0x66, 0x3b, 0x67, 0x96, 0xeb, 0x47, 0xd4, 0x8f, 0xe2, + 0xc4, 0xbe, 0x92, 0xd6, 0x54, 0x45, 0xd6, 0xdd, 0x74, 0xce, 0xf6, 0x14, 0x89, 0xba, 0xde, 0x82, + 0x9d, 0x00, 0xd0, 0x06, 0x5c, 0x57, 0x57, 0xbc, 0x88, 0x12, 0xd1, 0x80, 0x70, 0xaa, 0xdb, 0x90, + 0xbc, 0x6c, 0x43, 0xae, 0x25, 0xc8, 0x9e, 0xc0, 0xa9, 0x8e, 0xe4, 0x3e, 0xa0, 0x94, 0x47, 0x26, + 0x02, 0x97, 0xc6, 0xe7, 0xd6, 0xea, 0xc6, 0xb3, 0xdf, 0x5d, 0x8d, 0xc0, 0xad, 0x84, 0x38, 0x06, + 0xa1, 0x3b, 0xb0, 0xe4, 0xc4, 0x91, 0x2d, 0x6d, 0x47, 0x33, 0x99, 0x1e, 0x37, 0x12, 0x9c, 0x30, + 0x1f, 0x45, 0x77, 0x00, 0x9d, 0x8a, 0x3d, 0x8e, 0x2b, 0x58, 0x52, 0x23, 0x08, 0x85, 0xc9, 0x68, + 0xf7, 0x39, 0x2c, 0x6a, 0xea, 0x44, 0xb5, 0xf2, 0x3c, 0xd5, 0x1a, 0x8a, 0x32, 0xd1, 0xeb, 0x3d, + 0xa8, 0x7b, 0x36, 0x8f, 0x2c, 0x3b, 0x08, 0x3c, 0x97, 0x12, 0x39, 0x7e, 0xac, 0xe3, 0x9a, 0x80, + 0x6d, 0x2a, 0x10, 0xda, 0x84, 0x96, 0x47, 0xfb, 0xb6, 0x33, 0xca, 0x36, 0x7f, 0xd5, 0x4b, 0x9a, + 0xbf, 0xa6, 0x22, 0xcf, 0xdc, 0x7c, 0x3e, 0x05, 0xd1, 0xdd, 0x59, 0x67, 0x74, 0x14, 0x4f, 0x73, + 0xde, 0x9d, 0x3a, 0xb3, 0x87, 0xf6, 0xf3, 0x5f, 0xd2, 0x91, 0x3e, 0xb0, 0xca, 0x40, 0xad, 0xd0, + 0x2d, 0xb8, 0x16, 0x85, 0x6e, 0xbf, 0x2f, 0xaa, 0x9b, 0x1d, 0xda, 0x03, 0xae, 0xcc, 0x06, 0x52, + 0xcd, 0x05, 0x8d, 0x3a, 0x90, 0x98, 0xf6, 0x3d, 0x58, 0x9c, 0x38, 0xf8, 0xec, 0x3c, 0xc2, 0x98, + 0x31, 0x8f, 0xa8, 0x67, 0xe6, 0x11, 0xed, 0xcf, 0xa1, 0x9e, 0xd5, 0xe1, 0xdb, 0x66, 0x19, 0x59, + 0xde, 0xce, 0xbf, 0x96, 0xa1, 0x72, 0x40, 0x43, 0xee, 0xf2, 0x08, 0x5d, 0x87, 0x32, 0xa7, 0x5f, + 0x5b, 0x3e, 0x93, 0xac, 0x45, 0x5c, 0xe2, 0xf4, 0xeb, 0x47, 0x4c, 0x9c, 0xa9, 0xaa, 0x49, 0x56, + 0xd6, 0x83, 0x55, 0xb5, 0x6a, 0x2a, 0x4c, 0xaa, 0xfd, 0xa4, 0xa3, 0x17, 0x26, 0x1c, 0x5d, 0xbf, + 0xeb, 0x6a, 0x8e, 0x5e, 0x7c, 0x5d, 0x47, 0x2f, 0xbd, 0x86, 0xa3, 0xaf, 0xc3, 0x8d, 0xd4, 0xd1, + 0x03, 0x3b, 0x72, 0x4e, 0xa9, 0x3e, 0x33, 0x55, 0x0a, 0x9b, 0x09, 0xf6, 0x40, 0x21, 0xe7, 0x38, + 0x7b, 0x65, 0x8e, 0xb3, 0x7f, 0x02, 0x37, 0xb4, 0x19, 0x27, 0x7d, 0xbe, 0x2a, 0x4d, 0xb9, 0xa4, + 0xb0, 0x5f, 0x8d, 0xbb, 0xf9, 0x8c, 0x10, 0x31, 0xae, 0x1a, 0x22, 0xf0, 0x8a, 0x21, 0x52, 0xbb, + 0x72, 0x88, 0xd4, 0x27, 0x42, 0x24, 0x3e, 0xed, 0xd9, 0x21, 0xb2, 0x01, 0xd7, 0xb5, 0x45, 0xc6, + 0x23, 0xc5, 0x5c, 0x90, 0x06, 0xb9, 0xa6, 0x90, 0x47, 0xd9, 0x50, 0x99, 0x17, 0x56, 0x8d, 0x1f, + 0x58, 0x58, 0x75, 0xc0, 0xd0, 0x7b, 0xa7, 0x64, 0x4e, 0x5c, 0x75, 0xfe, 0x26, 0x07, 0x25, 0x71, + 0x0e, 0xa3, 0x99, 0xd5, 0xca, 0x84, 0xca, 0xb9, 0x90, 0xa0, 0x9b, 0x52, 0x03, 0xc7, 0x4b, 0x71, + 0x05, 0x95, 0xc7, 0x2a, 0x59, 0x54, 0x1a, 0xae, 0x0a, 0x80, 0xa8, 0xba, 0xc9, 0x99, 0xc7, 0xbc, + 0xaa, 0x6f, 0x90, 0x67, 0xfe, 0x54, 0xf3, 0xaf, 0xcf, 0xc9, 0xe8, 0xca, 0xcd, 0xd1, 0x78, 0x46, + 0x17, 0x1d, 0x65, 0xe7, 0x19, 0x54, 0x62, 0x87, 0xb9, 0x0b, 0x48, 0x95, 0xb7, 0xe4, 0x86, 0x18, + 0x97, 0x63, 0x03, 0xb7, 0x14, 0x66, 0x3b, 0x45, 0x5c, 0x12, 0x54, 0xf9, 0xd9, 0x41, 0xd5, 0xf9, + 0x5d, 0x4e, 0xdf, 0x83, 0x5e, 0xcf, 0x28, 0x1f, 0xc6, 0x5f, 0xae, 0x0a, 0x33, 0xbf, 0x5c, 0xc5, + 0xdf, 0xac, 0xde, 0xbf, 0xb4, 0x9a, 0xc9, 0xeb, 0x1f, 0x45, 0x3f, 0xc9, 0x78, 0x74, 0x49, 0x7a, + 0x74, 0x7a, 0xf9, 0x95, 0x57, 0xae, 0x99, 0xee, 0xfc, 0x9d, 0xfc, 0x05, 0xa0, 0x2a, 0x53, 0xc5, + 0x23, 0x76, 0xd1, 0x29, 0x43, 0xf1, 0x30, 0x62, 0x41, 0xc7, 0x80, 0x8a, 0xf8, 0x0d, 0x28, 0xe9, + 0xfc, 0x11, 0xd4, 0x0e, 0x29, 0x17, 0x1b, 0xdd, 0x67, 0x2c, 0x98, 0x73, 0x4f, 0xcf, 0x5d, 0xe5, + 0x9e, 0xfe, 0x77, 0x37, 0xa0, 0x96, 0xb9, 0x43, 0xa3, 0xbb, 0x19, 0xab, 0xd7, 0x36, 0xde, 0xea, + 0x66, 0x3f, 0x31, 0xc7, 0xd7, 0x4e, 0x69, 0x50, 0x75, 0x20, 0x5f, 0xc0, 0x82, 0xf8, 0xb5, 0x42, + 0xdd, 0xee, 0x27, 0xc3, 0x82, 0x71, 0x3e, 0x85, 0x54, 0x8c, 0x75, 0xc1, 0x90, 0x5c, 0x0f, 0x3e, + 0x83, 0xea, 0xb9, 0xed, 0xb9, 0xc4, 0x8e, 0xe2, 0xa3, 0x7b, 0x77, 0xe6, 0x3b, 0x9f, 0x6a, 0x22, + 0x9c, 0x90, 0xa3, 0x7b, 0x60, 0xc4, 0xcf, 0xf1, 0x98, 0x68, 0x79, 0xf6, 0x7b, 0x63, 0x66, 0x82, + 0x53, 0x0e, 0xf4, 0x33, 0xa8, 0x73, 0x65, 0x53, 0xcb, 0x63, 0x2c, 0x30, 0x97, 0x74, 0xd6, 0x8b, + 0xcd, 0x97, 0x31, 0x38, 0xae, 0xf1, 0x8c, 0xf5, 0xdf, 0x83, 0xe2, 0x33, 0xe6, 0xfa, 0xe6, 0x75, + 0xc9, 0xb0, 0x30, 0x76, 0x2f, 0xc0, 0x12, 0x85, 0x3e, 0x82, 0xf2, 0x33, 0xd9, 0xbd, 0x9a, 0x37, + 0xb4, 0x3b, 0x66, 0x89, 0x28, 0xc1, 0x1a, 0x2d, 0x64, 0x45, 0x36, 0x3f, 0x33, 0xdf, 0x9c, 0x90, + 0x25, 0x9a, 0x58, 0x2c, 0x51, 0xe8, 0x96, 0xe8, 0xb7, 0x65, 0xa7, 0x61, 0x9a, 0xfa, 0x13, 0xe0, + 0x44, 0x07, 0x82, 0x63, 0x02, 0xf4, 0x01, 0x94, 0x44, 0xb2, 0x1f, 0x99, 0x6f, 0x49, 0xca, 0xf4, + 0x6b, 0xb8, 0xcc, 0x33, 0x58, 0x21, 0x85, 0xc4, 0xb8, 0x24, 0xb4, 0x27, 0x24, 0xea, 0x30, 0xc7, + 0x31, 0x81, 0x50, 0x50, 0xce, 0x3d, 0xde, 0x9e, 0x50, 0x30, 0x33, 0xe8, 0xf8, 0x38, 0x99, 0x15, + 0xbe, 0x33, 0x31, 0xeb, 0xc8, 0x38, 0xd6, 0xe4, 0x94, 0xf0, 0x2e, 0x14, 0x3d, 0x66, 0x13, 0x73, + 0x55, 0xfb, 0xd9, 0x2c, 0x96, 0x7d, 0x66, 0x13, 0x2c, 0xc9, 0xc4, 0x3b, 0xc4, 0x2f, 0x25, 0xe6, + 0x8f, 0x2e, 0x79, 0xc7, 0xbe, 0x24, 0xc1, 0x9a, 0x14, 0xad, 0x43, 0x49, 0x8e, 0x7f, 0xcc, 0x5b, + 0x13, 0x91, 0x91, 0xe5, 0x91, 0x53, 0x21, 0xac, 0x08, 0xd1, 0x4f, 0xd3, 0x41, 0xd3, 0xed, 0x89, + 0x41, 0xcf, 0x14, 0x4f, 0x66, 0xba, 0x24, 0xde, 0xc4, 0x23, 0x16, 0x52, 0xf3, 0xce, 0x25, 0x6f, + 0x3a, 0x14, 0x14, 0x58, 0x11, 0x8a, 0x0d, 0xc9, 0x07, 0x62, 0xde, 0xbd, 0x64, 0x43, 0x92, 0x45, + 0x5c, 0x33, 0xe5, 0x2f, 0xea, 0x4d, 0x7c, 0xea, 0xe9, 0x4a, 0xd6, 0x95, 0x39, 0xac, 0xb3, 0x3f, + 0xf2, 0xa0, 0x3d, 0x68, 0xc8, 0xa5, 0x68, 0x5b, 0x94, 0x98, 0xb5, 0x89, 0x11, 0xeb, 0x94, 0x18, + 0x4a, 0xb4, 0xa0, 0x05, 0x9e, 0x5d, 0xa2, 0x2d, 0xd9, 0xeb, 0xf9, 0xec, 0xc2, 0xa3, 0xa4, 0x4f, + 0xcd, 0xf5, 0x4b, 0xd4, 0xd9, 0x4c, 0xe9, 0x70, 0x96, 0x09, 0xed, 0x40, 0x3d, 0xb3, 0x24, 0xe6, + 0x8f, 0x27, 0xe6, 0xcd, 0x73, 0x84, 0x10, 0x3c, 0xc6, 0x26, 0x7c, 0x3a, 0x50, 0x05, 0xd7, 0xdc, + 0x98, 0xf0, 0x69, 0x5d, 0x88, 0x71, 0x4c, 0x80, 0xd6, 0xc1, 0x08, 0xe2, 0xe2, 0x6c, 0x7e, 0x3c, + 0x31, 0x80, 0x4c, 0xca, 0x36, 0x4e, 0x89, 0x50, 0x17, 0x0c, 0xd5, 0xe2, 0xf9, 0xec, 0xc2, 0xfc, + 0x44, 0xf7, 0x5f, 0xc9, 0x50, 0x4e, 0x27, 0x6e, 0x5c, 0x75, 0xf4, 0x93, 0x88, 0x1a, 0x1e, 0xb1, + 0xc0, 0xfc, 0xf9, 0x44, 0xd4, 0x88, 0x7c, 0x8e, 0x25, 0x4a, 0x28, 0xcc, 0x55, 0x76, 0x37, 0xef, + 0x4d, 0x28, 0xac, 0xb3, 0x3e, 0x8e, 0x09, 0xda, 0x7f, 0x9a, 0x4b, 0x26, 0xda, 0xaf, 0x3f, 0x3c, + 0xfd, 0x32, 0x5b, 0xf2, 0x32, 0x5d, 0x5e, 0xfe, 0x92, 0x2e, 0xef, 0x5a, 0xc2, 0x91, 0x99, 0x83, + 0xff, 0x04, 0x8a, 0x22, 0xc0, 0xd0, 0x5d, 0xa8, 0x26, 0xbd, 0x68, 0x6e, 0x5e, 0x2f, 0x9a, 0x90, + 0xb4, 0x7f, 0x97, 0x87, 0xb2, 0x0a, 0x4c, 0xf4, 0xc5, 0xd4, 0x68, 0xf3, 0xfd, 0x4b, 0xe2, 0x78, + 0x7a, 0xb2, 0xa9, 0x5a, 0x17, 0x39, 0x5a, 0x0b, 0x2d, 0xf5, 0x95, 0xf7, 0x78, 0x14, 0x51, 0x75, + 0x19, 0x29, 0x8a, 0xd6, 0x45, 0xe1, 0x9e, 0x08, 0xd4, 0x96, 0xc0, 0xb4, 0xff, 0x27, 0x97, 0xce, + 0x42, 0x97, 0xa0, 0xa4, 0xe6, 0x33, 0xaa, 0x24, 0xab, 0x05, 0x5a, 0x85, 0xe6, 0xc0, 0xf5, 0x2d, + 0xce, 0x86, 0xa1, 0x33, 0x7e, 0xa3, 0x6e, 0x0c, 0x5c, 0xff, 0x50, 0x82, 0x55, 0x07, 0xbf, 0xaa, + 0xae, 0xfe, 0x63, 0x94, 0x05, 0x4d, 0x69, 0x3f, 0xcf, 0x52, 0xde, 0x01, 0xa4, 0xa8, 0x88, 0x45, + 0x98, 0xc3, 0xad, 0x88, 0x45, 0xb6, 0x27, 0x6b, 0x54, 0x11, 0x37, 0x35, 0x66, 0x9b, 0x39, 0xfc, + 0x48, 0xc0, 0x51, 0x17, 0xae, 0xc5, 0xd4, 0x72, 0x3b, 0x9a, 0xbc, 0x24, 0xc9, 0x5b, 0x1a, 0x25, + 0xb7, 0xa3, 0xe8, 0x3b, 0xb0, 0xa0, 0xfb, 0x13, 0x8b, 0x50, 0x2f, 0xd2, 0x7f, 0x28, 0x81, 0x6b, + 0xaa, 0x11, 0xd9, 0x16, 0xa0, 0xf6, 0x67, 0x50, 0x92, 0x59, 0xea, 0x92, 0x0e, 0x2c, 0x37, 0xbb, + 0x03, 0x6b, 0xff, 0x6f, 0x2e, 0x9d, 0x95, 0x5f, 0x36, 0x8c, 0x9e, 0x91, 0x11, 0x67, 0x1e, 0xd9, + 0x6b, 0x76, 0x80, 0xed, 0xd1, 0xb7, 0x9d, 0xd8, 0x2d, 0x68, 0xa9, 0x0c, 0x9f, 0x35, 0xae, 0x72, + 0x81, 0x45, 0x85, 0x48, 0x6d, 0x7b, 0x07, 0x90, 0xa6, 0xcd, 0x9a, 0xb6, 0xa0, 0x4e, 0x42, 0x61, + 0x52, 0xcb, 0xb6, 0x2b, 0x50, 0x92, 0x29, 0xb7, 0xfd, 0x4f, 0x39, 0x28, 0xab, 0xe4, 0xfb, 0xca, + 0x4e, 0xab, 0xc8, 0x67, 0x8c, 0xe3, 0x5f, 0x65, 0x3f, 0x2a, 0xc1, 0xcf, 0xd8, 0x8f, 0x42, 0x8c, + 0xed, 0x47, 0xd3, 0xce, 0xd8, 0x8f, 0xc2, 0x64, 0xf6, 0xf3, 0x67, 0xb9, 0xf1, 0x2f, 0xfa, 0xaf, + 0xed, 0x0c, 0xdf, 0x5f, 0xf6, 0xd8, 0x84, 0x85, 0xb1, 0x5a, 0x72, 0x05, 0xc7, 0xfc, 0x02, 0x6a, + 0x99, 0x0a, 0x70, 0x05, 0x01, 0xf7, 0xa1, 0x9e, 0x2d, 0x21, 0xaf, 0x2f, 0xe1, 0xd6, 0x6d, 0x28, + 0xc9, 0x3f, 0x2c, 0x44, 0x00, 0xe5, 0x83, 0x27, 0x5b, 0xfb, 0x7b, 0xbd, 0xe6, 0x1b, 0xa8, 0x06, + 0x95, 0x03, 0xbc, 0xf7, 0x74, 0xf3, 0x68, 0xa7, 0x99, 0x43, 0x06, 0x94, 0xf6, 0x1f, 0xf7, 0x36, + 0xf7, 0x9b, 0xf9, 0x8d, 0x07, 0x50, 0xd5, 0x7f, 0xf8, 0x15, 0xa2, 0x5f, 0x40, 0x45, 0x3f, 0xa3, + 0x37, 0xd3, 0x44, 0x3f, 0xf6, 0x27, 0x89, 0x6d, 0x73, 0x1a, 0xa1, 0x1a, 0xd8, 0xf5, 0xdc, 0xc6, + 0x3e, 0x54, 0xf5, 0x47, 0x85, 0x10, 0xdd, 0x87, 0x8a, 0x7e, 0xce, 0xc8, 0x1a, 0xff, 0x34, 0x94, + 0x91, 0x35, 0xf1, 0x2d, 0x62, 0x35, 0xb7, 0x9e, 0xdb, 0x38, 0x85, 0xc6, 0xf8, 0xb8, 0x1e, 0x3d, + 0x85, 0x45, 0xf9, 0x90, 0x80, 0x39, 0xba, 0x99, 0xad, 0x2b, 0xd3, 0x43, 0xff, 0xf6, 0xf2, 0x5c, + 0x7c, 0xe6, 0x4d, 0x5f, 0x42, 0x79, 0x5f, 0xfd, 0x7d, 0xda, 0xbd, 0xf1, 0x8b, 0xc6, 0xd2, 0xac, + 0x20, 0x6a, 0xcf, 0x84, 0x4a, 0x41, 0xbb, 0xf1, 0xfc, 0xfa, 0xbb, 0xc9, 0xd9, 0xfa, 0xc5, 0x37, + 0xff, 0x71, 0xf3, 0x8d, 0x6f, 0x7e, 0x7b, 0x33, 0xf7, 0xcf, 0xbf, 0xbd, 0x99, 0xfb, 0xab, 0xff, + 0xbc, 0x99, 0xfb, 0x93, 0x3b, 0xaf, 0xf4, 0xc7, 0x65, 0x5a, 0xde, 0x71, 0x59, 0x82, 0x3e, 0xfe, + 0x7d, 0x00, 0x00, 0x00, 0xff, 0xff, 0x8b, 0x5e, 0x96, 0xbe, 0x45, 0x2c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -5417,18 +5416,6 @@ func (m *Task) MarshalToSizedBuffer(dAtA []byte) (int, error) { 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) @@ -7843,10 +7830,6 @@ func (m *Task) ProtoSize() (n int) { 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 } @@ -13180,42 +13163,6 @@ func (m *Task) Unmarshal(dAtA []byte) error { } 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) diff --git a/go/protocols/runtime/runtime.proto b/go/protocols/runtime/runtime.proto index 1a008f6eade..d74367c713b 100644 --- a/go/protocols/runtime/runtime.proto +++ b/go/protocols/runtime/runtime.proto @@ -470,10 +470,10 @@ message Joined { message Task { // Task specification (protobuf-encoded bytes). bytes spec = 1; - // Collection journal partition to which task states are written. + // Collection journal partition to which task stats are written. + // The journal is pre-created by activate; the runtime appends directly + // without consulting the partitions watch or creating new partitions. 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. diff --git a/go/runtime/materialize_v2.go b/go/runtime/materialize_v2.go index 0ec106382e9..0d694997d1b 100644 --- a/go/runtime/materialize_v2.go +++ b/go/runtime/materialize_v2.go @@ -1,7 +1,6 @@ package runtime import ( - "database/sql" "fmt" "io" "os" @@ -10,7 +9,6 @@ import ( "github.com/estuary/flow/go/bindings" "github.com/estuary/flow/go/flow" "github.com/estuary/flow/go/labels" - "github.com/estuary/flow/go/protocols/catalog" pf "github.com/estuary/flow/go/protocols/flow" pr "github.com/estuary/flow/go/protocols/runtime" "github.com/estuary/flow/go/shuffle" @@ -170,10 +168,6 @@ func (m *materializeAppV2) runOneSession(shard consumer.Shard, ch chan<- consume // 1. Load and marshal everything that doesn't depend on Etcd topology. var opsStatsJournal = m.term.labels.StatsJournal - var opsStatsSpec *pf.CollectionSpec - if opsStatsSpec, err = m.loadOpsCollectionSpec(opsStatsJournal); err != nil { - return err - } var specBytes []byte if specBytes, err = m.term.taskSpec.Marshal(); err != nil { return fmt.Errorf("marshaling MaterializationSpec: %w", err) @@ -219,7 +213,6 @@ func (m *materializeAppV2) runOneSession(shard consumer.Shard, ch chan<- consume Task: &pr.Task{ Spec: specBytes, OpsStatsJournal: string(opsStatsJournal), - OpsStatsSpec: opsStatsSpec, Preview: false, MaxTransactions: 0, }, @@ -268,29 +261,6 @@ func (m *materializeAppV2) runOneSession(shard consumer.Shard, ch chan<- consume } } -// loadOpsCollectionSpec resolves an ops CollectionSpec for `journal` from -// the current term's build DB. catalog.LoadCollectionForJournal inverts the -// `/=/...` shape produced by -// activate::ops_partition_spec via SQL prefix match against -// built_collections. -func (m *materializeAppV2) loadOpsCollectionSpec(journal pb.Journal) (*pf.CollectionSpec, error) { - var spec *pf.CollectionSpec - var build = m.host.builds.Open(m.term.labels.Build) - defer build.Close() - - if err := build.Extract(func(db *sql.DB) error { - var s, err = catalog.LoadCollectionForJournal(db, string(journal)) - if err != nil { - return err - } - spec = s - return nil - }); err != nil { - return nil, fmt.Errorf("loading ops CollectionSpec for %q: %w", journal, err) - } - return spec, nil -} - func (m *materializeAppV2) Destroy() { if m.client != nil { _ = m.client.CloseSend() From 3158644f926958c2e69c6e90ae5a8298b0ac91f4 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 22:00:57 +0000 Subject: [PATCH 12/22] auth: grant V2 journal clients a joint APPEND|APPLY|LIST capability The V2 publisher creates destination partitions on demand, so it needs APPLY in addition to APPEND, plus LIST to watch journals. Have the runtime-next task service and the runtime-sidecar publisher factory request `APPEND | APPLY | LIST` jointly, and teach `authorize_task` to accept that combined capability as `models::Capability::Write`. Update the `TaskCollectionAuth` doc comment to reflect the broadened set. --- .../src/server/authorize_task.rs | 20 +++++++------------ .../src/workflows/task_collection_auth.rs | 5 ++--- crates/runtime-next/src/task_service.rs | 3 ++- crates/runtime-sidecar/src/lib.rs | 14 +++++++------ 4 files changed, 19 insertions(+), 23 deletions(-) diff --git a/crates/control-plane-api/src/server/authorize_task.rs b/crates/control-plane-api/src/server/authorize_task.rs index f062914f886..deb4f1be21f 100644 --- a/crates/control-plane-api/src/server/authorize_task.rs +++ b/crates/control-plane-api/src/server/authorize_task.rs @@ -39,20 +39,14 @@ pub async fn authorize_task( // checking that the requested capability contains a particular grant isn't enough. // For example, we wouldn't want to allow a request for `REPLICATE` just // because it also requests `READ`. + use proto_gazette::capability::{APPEND, APPLY, LIST, READ}; let required_role = match cap { - cap if (cap == proto_gazette::capability::LIST) - || (cap == proto_gazette::capability::READ) - || (cap == (proto_gazette::capability::LIST | proto_gazette::capability::READ)) => - { - models::Capability::Read - } - // We're intentionally rejecting requests for both APPLY and APPEND, as those two - // grants authorize wildly different capabilities, and no sane logic should - // need both at the same time. So as a sanity check/defense-in-depth measure - // we won't grant you a token that has both, even if we technically could. - cap if (cap == proto_gazette::capability::APPLY) - || (cap == proto_gazette::capability::APPEND) => - { + // Legacy go shuffle module obtains LIST and READ separately. + // The `shuffle` crate and `dekaf` obtain LIST | READ jointly. + cap if (cap == LIST) || (cap == READ) || (cap == (LIST | READ)) => models::Capability::Read, + // Legacy publisher separately obtains APPLY vs APPEND tokens. + // The `publisher` crates obtains APPEND | APPLY | LIST jointly. + cap if (cap == APPLY) || (cap == APPEND) || (cap == (APPEND | APPLY | LIST)) => { models::Capability::Write } cap => { diff --git a/crates/flow-client-next/src/workflows/task_collection_auth.rs b/crates/flow-client-next/src/workflows/task_collection_auth.rs index 196a451f714..d53a4b96fea 100644 --- a/crates/flow-client-next/src/workflows/task_collection_auth.rs +++ b/crates/flow-client-next/src/workflows/task_collection_auth.rs @@ -28,9 +28,8 @@ pub struct TaskCollectionAuth { /// /// `capability` is the requested capability level of the authorization. /// This is NOT a models::Capability. Rather, it's a bit-mask in the u32 -/// Gazette capability namespace and is restricted to: -/// - proto_gazette::capability::READ -/// - proto_gazette::capability::APPEND +/// Gazette capability namespace and is restricted to +/// proto_gazette::capability::{APPEND, APPLY, LIST, READ}. /// /// `data_plane_fqdn` is the FQDN of the data-plane hosting the task. /// diff --git a/crates/runtime-next/src/task_service.rs b/crates/runtime-next/src/task_service.rs index b2be5214e4a..8ef98572f8f 100644 --- a/crates/runtime-next/src/task_service.rs +++ b/crates/runtime-next/src/task_service.rs @@ -47,10 +47,11 @@ impl TaskService { let control_api_endpoint: url::Url = url::Url::parse(&control_api_endpoint).context("invalid control API endpoint URL")?; + use proto_gazette::capability::{APPEND, APPLY, LIST}; 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, + APPEND | APPLY | LIST, gazette::Router::new(&availability_zone), data_plane_fqdn, tokens::jwt::EncodingKey::from_secret(&data_plane_signing_key), diff --git a/crates/runtime-sidecar/src/lib.rs b/crates/runtime-sidecar/src/lib.rs index e8d2a086409..af3885e317a 100644 --- a/crates/runtime-sidecar/src/lib.rs +++ b/crates/runtime-sidecar/src/lib.rs @@ -86,20 +86,22 @@ pub async fn run(args: Args) -> anyhow::Result<()> { let api_client = flow_client_next::rest::Client::new(&args.agent_endpoint, "runtime-sidecar"); let router = gazette::Router::new(&args.gazette_zone); - // Two journal client factories: LIST|READ (shuffle reads source journals) - // and LIST|APPEND (leader publishes stats and ACK intents). + use proto_gazette::capability::{APPEND, APPLY, LIST, READ}; + // Shuffle read factory watches (LIST) and reads (READ) source journals. let read_factory = flow_client_next::workflows::task_collection_auth::new_journal_client_factory( api_client.clone(), - proto_gazette::capability::LIST | proto_gazette::capability::READ, + LIST | READ, router.clone(), args.data_plane_fqdn.clone(), signing_key.clone(), ); - let append_factory = + // Publisher factory watches (LIST), creates partitions (APPLY), + // and appends (APPEND) to dest journals. + let publisher_factory = flow_client_next::workflows::task_collection_auth::new_journal_client_factory( api_client, - proto_gazette::capability::LIST | proto_gazette::capability::APPEND, + APPEND | APPLY | LIST, router, args.data_plane_fqdn, signing_key, @@ -110,7 +112,7 @@ pub async fn run(args: Args) -> anyhow::Result<()> { read_factory, args.disk_backlog_threshold, ); - let runtime_svc = runtime_next::Service::new(shuffle_svc.clone(), append_factory); + let runtime_svc = runtime_next::Service::new(shuffle_svc.clone(), publisher_factory); // Build a TLS identity if both files were given. // clap `requires` enforces both-or-neither. From 1a95eaec10a105994c508703b6bf4d94435e0a7f Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 22:02:49 +0000 Subject: [PATCH 13/22] bindings: initialize rustls with aws_lc_rs as the default provider `bindings` is linked into Go binaries through CGO, so there is no Rust binary entrypoint to install rustls' process-wide CryptoProvider. Install the `aws-lc-rs` provider lazily (once) when a task service is created, and enable rustls' `aws_lc_rs` feature. --- Cargo.lock | 1 + Cargo.toml | 2 +- crates/bindings/Cargo.toml | 1 + crates/bindings/src/lib.rs | 10 ++++++++++ crates/bindings/src/task_service.rs | 2 ++ crates/bindings/src/task_service_v2.rs | 2 ++ 6 files changed, 17 insertions(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index 0a928d9ec38..f32be10df48 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1413,6 +1413,7 @@ dependencies = [ "proto-flow", "runtime", "runtime-next", + "rustls 0.23.32", "serde", "serde_json", "thiserror 2.0.17", diff --git a/Cargo.toml b/Cargo.toml index 255691eed56..7fcfc282ac2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -169,7 +169,7 @@ sqlx = { version = "0", features = [ ] } tokio-rustls = "0" -rustls = { version = "0" } +rustls = { version = "0", features = ["aws_lc_rs"] } rustls-pemfile = "2" webpki = "0" tempfile = "3" diff --git a/crates/bindings/Cargo.toml b/crates/bindings/Cargo.toml index 0e21d9e49a9..d49061f0cce 100644 --- a/crates/bindings/Cargo.toml +++ b/crates/bindings/Cargo.toml @@ -26,6 +26,7 @@ futures = { workspace = true } prost = { workspace = true } serde = { workspace = true } serde_json = { workspace = true } +rustls = { workspace = true } thiserror = { workspace = true } time = { workspace = true } tracing = { workspace = true } diff --git a/crates/bindings/src/lib.rs b/crates/bindings/src/lib.rs index af6336dd159..d98f78e6446 100644 --- a/crates/bindings/src/lib.rs +++ b/crates/bindings/src/lib.rs @@ -4,3 +4,13 @@ mod service; mod task_service; mod task_service_v2; mod upper_case; + +fn install_crypto_provider() { + static ONCE: std::sync::Once = std::sync::Once::new(); + + ONCE.call_once(|| { + // `bindings` is linked into Go binaries through CGO, so Rust binary + // entrypoints cannot install rustls' process-wide provider for us. + let _ = rustls::crypto::aws_lc_rs::default_provider().install_default(); + }); +} diff --git a/crates/bindings/src/task_service.rs b/crates/bindings/src/task_service.rs index 31107336766..771a2d95a79 100644 --- a/crates/bindings/src/task_service.rs +++ b/crates/bindings/src/task_service.rs @@ -21,6 +21,8 @@ pub struct TaskService { #[unsafe(no_mangle)] pub extern "C" fn new_task_service(config_ptr: *const u8, config_len: u32) -> *mut TaskService { + crate::install_crypto_provider(); + let config = unsafe { std::slice::from_raw_parts(config_ptr, config_len as usize) }; let config = proto_flow::runtime::TaskServiceConfig::decode(config).unwrap(); diff --git a/crates/bindings/src/task_service_v2.rs b/crates/bindings/src/task_service_v2.rs index fb5708eb6e6..5fcd9f9bc58 100644 --- a/crates/bindings/src/task_service_v2.rs +++ b/crates/bindings/src/task_service_v2.rs @@ -22,6 +22,8 @@ pub extern "C" fn new_task_service_v2( config_ptr: *const u8, config_len: u32, ) -> *mut TaskServiceV2 { + crate::install_crypto_provider(); + let config = unsafe { std::slice::from_raw_parts(config_ptr, config_len as usize) }; let config = proto_flow::runtime::TaskServiceConfig::decode(config).unwrap(); From ab4b45ea499fce0ec22e23365a5a75d7b6bf82fb Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 22:06:32 +0000 Subject: [PATCH 14/22] runtime-next: source stats journal from labeling; route log level via Materialize Drop `ops_stats_journal` from the `Task` proto: both the leader and the shard already receive it via shard labeling at Join time, so passing it through `Task` was redundant. Add `log_level` to the top-level `Materialize` message so the controller can supply it on unary `spec` / `validate` requests, which never see the Join-time labeling that carries log level for session-bound work. Session paths continue to read log level from labeling. --- crates/flowctl/src/raw/preview_next/driver.rs | 1 - crates/proto-flow/src/runtime.rs | 15 +- crates/proto-flow/src/runtime.serde.rs | 38 +- .../src/leader/materialize/handler.rs | 4 + .../src/leader/materialize/startup.rs | 2 +- .../src/shard/materialize/connector.rs | 2 +- .../src/shard/materialize/handler.rs | 23 +- .../src/shard/materialize/startup.rs | 7 +- go/protocols/runtime/runtime.pb.go | 554 +++++++++--------- go/protocols/runtime/runtime.proto | 14 +- go/runtime/materialize_v2.go | 46 +- 11 files changed, 355 insertions(+), 351 deletions(-) diff --git a/crates/flowctl/src/raw/preview_next/driver.rs b/crates/flowctl/src/raw/preview_next/driver.rs index 48fafe5e431..29daf7cc25d 100644 --- a/crates/flowctl/src/raw/preview_next/driver.rs +++ b/crates/flowctl/src/raw/preview_next/driver.rs @@ -161,7 +161,6 @@ async fn drive_one_shard( .send(Ok(proto::Materialize { task: Some(proto::Task { spec: spec_bytes.clone(), - ops_stats_journal: String::new(), preview: true, max_transactions: target_txns, }), diff --git a/crates/proto-flow/src/runtime.rs b/crates/proto-flow/src/runtime.rs index 7f9c4956840..eda143cd55b 100644 --- a/crates/proto-flow/src/runtime.rs +++ b/crates/proto-flow/src/runtime.rs @@ -490,16 +490,11 @@ pub struct Task { /// Task specification (protobuf-encoded bytes). #[prost(bytes = "bytes", tag = "1")] pub spec: ::prost::bytes::Bytes, - /// Collection journal partition to which task stats are written. - /// The journal is pre-created by activate; the runtime appends directly - /// without consulting the partitions watch or creating new partitions. - #[prost(string, tag = "2")] - pub ops_stats_journal: ::prost::alloc::string::String, /// When true, documents and stats are written to output and not directed to collections. - #[prost(bool, tag = "4")] + #[prost(bool, tag = "2")] pub preview: bool, /// Preview / harness control. Zero means unlimited. - #[prost(uint32, tag = "5")] + #[prost(uint32, tag = "3")] pub max_transactions: u32, } /// Recover is sent by each shard to the leader after Joined, and carries @@ -743,6 +738,12 @@ pub struct Materialize { /// Shard → Controller. Connector's reply to `validate`. #[prost(message, optional, tag = "4")] pub validated: ::core::option::Option, + /// Controller → Shard. Effective only on unary `spec` / `validate` + /// messages, which never see the Join-time labeling that supplies the + /// log level for session-bound work. Ignored on all other variants + /// (the leader → shard messages MUST NOT set this). + #[prost(enumeration = "super::ops::log::Level", tag = "5")] + pub log_level: i32, /// Controller → Shard. First message of a session-loop stream; /// never sent to the Leader. #[prost(message, optional, tag = "20")] diff --git a/crates/proto-flow/src/runtime.serde.rs b/crates/proto-flow/src/runtime.serde.rs index 99a5cacfde3..2ee5352df9c 100644 --- a/crates/proto-flow/src/runtime.serde.rs +++ b/crates/proto-flow/src/runtime.serde.rs @@ -3330,6 +3330,9 @@ impl serde::Serialize for Materialize { if self.validated.is_some() { len += 1; } + if self.log_level != 0 { + len += 1; + } if self.session_loop.is_some() { len += 1; } @@ -3415,6 +3418,11 @@ impl serde::Serialize for Materialize { if let Some(v) = self.validated.as_ref() { struct_ser.serialize_field("validated", v)?; } + 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.session_loop.as_ref() { struct_ser.serialize_field("sessionLoop", v)?; } @@ -3502,6 +3510,8 @@ impl<'de> serde::Deserialize<'de> for Materialize { "specResponse", "validate", "validated", + "log_level", + "logLevel", "session_loop", "sessionLoop", "join", @@ -3538,6 +3548,7 @@ impl<'de> serde::Deserialize<'de> for Materialize { SpecResponse, Validate, Validated, + LogLevel, SessionLoop, Join, Joined, @@ -3587,6 +3598,7 @@ impl<'de> serde::Deserialize<'de> for Materialize { "specResponse" | "spec_response" => Ok(GeneratedField::SpecResponse), "validate" => Ok(GeneratedField::Validate), "validated" => Ok(GeneratedField::Validated), + "logLevel" | "log_level" => Ok(GeneratedField::LogLevel), "sessionLoop" | "session_loop" => Ok(GeneratedField::SessionLoop), "join" => Ok(GeneratedField::Join), "joined" => Ok(GeneratedField::Joined), @@ -3634,6 +3646,7 @@ impl<'de> serde::Deserialize<'de> for Materialize { let mut spec_response__ = None; let mut validate__ = None; let mut validated__ = None; + let mut log_level__ = None; let mut session_loop__ = None; let mut join__ = None; let mut joined__ = None; @@ -3684,6 +3697,12 @@ impl<'de> serde::Deserialize<'de> for Materialize { } validated__ = map_.next_value()?; } + GeneratedField::LogLevel => { + if log_level__.is_some() { + return Err(serde::de::Error::duplicate_field("logLevel")); + } + log_level__ = Some(map_.next_value::()? as i32); + } GeneratedField::SessionLoop => { if session_loop__.is_some() { return Err(serde::de::Error::duplicate_field("sessionLoop")); @@ -3835,6 +3854,7 @@ impl<'de> serde::Deserialize<'de> for Materialize { spec_response: spec_response__, validate: validate__, validated: validated__, + log_level: log_level__.unwrap_or_default(), session_loop: session_loop__, join: join__, joined: joined__, @@ -7626,9 +7646,6 @@ impl serde::Serialize for Task { if !self.spec.is_empty() { len += 1; } - if !self.ops_stats_journal.is_empty() { - len += 1; - } if self.preview { len += 1; } @@ -7641,9 +7658,6 @@ impl serde::Serialize for Task { #[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 self.preview { struct_ser.serialize_field("preview", &self.preview)?; } @@ -7661,8 +7675,6 @@ impl<'de> serde::Deserialize<'de> for Task { { const FIELDS: &[&str] = &[ "spec", - "ops_stats_journal", - "opsStatsJournal", "preview", "max_transactions", "maxTransactions", @@ -7671,7 +7683,6 @@ impl<'de> serde::Deserialize<'de> for Task { #[allow(clippy::enum_variant_names)] enum GeneratedField { Spec, - OpsStatsJournal, Preview, MaxTransactions, } @@ -7696,7 +7707,6 @@ impl<'de> serde::Deserialize<'de> for Task { { match value { "spec" => Ok(GeneratedField::Spec), - "opsStatsJournal" | "ops_stats_journal" => Ok(GeneratedField::OpsStatsJournal), "preview" => Ok(GeneratedField::Preview), "maxTransactions" | "max_transactions" => Ok(GeneratedField::MaxTransactions), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), @@ -7719,7 +7729,6 @@ impl<'de> serde::Deserialize<'de> for Task { V: serde::de::MapAccess<'de>, { let mut spec__ = None; - let mut ops_stats_journal__ = None; let mut preview__ = None; let mut max_transactions__ = None; while let Some(k) = map_.next_key()? { @@ -7732,12 +7741,6 @@ impl<'de> serde::Deserialize<'de> for Task { 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::Preview => { if preview__.is_some() { return Err(serde::de::Error::duplicate_field("preview")); @@ -7756,7 +7759,6 @@ impl<'de> serde::Deserialize<'de> for Task { } Ok(Task { spec: spec__.unwrap_or_default(), - ops_stats_journal: ops_stats_journal__.unwrap_or_default(), preview: preview__.unwrap_or_default(), max_transactions: max_transactions__.unwrap_or_default(), }) diff --git a/crates/runtime-next/src/leader/materialize/handler.rs b/crates/runtime-next/src/leader/materialize/handler.rs index c6795da175f..ba90d8d447a 100644 --- a/crates/runtime-next/src/leader/materialize/handler.rs +++ b/crates/runtime-next/src/leader/materialize/handler.rs @@ -89,6 +89,7 @@ where ); let mut build = String::new(); + let mut ops_stats_journal = 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()); @@ -128,7 +129,9 @@ where directory, endpoint, }); + build = labeling.build; + ops_stats_journal = labeling.stats_journal; } let error_tx = shard_tx.clone(); @@ -146,6 +149,7 @@ where task, } = startup::run( build, + ops_stats_journal, reactors, &mut shard_rx, &mut shard_tx, diff --git a/crates/runtime-next/src/leader/materialize/startup.rs b/crates/runtime-next/src/leader/materialize/startup.rs index 5d09efb6c24..f144d9e7a18 100644 --- a/crates/runtime-next/src/leader/materialize/startup.rs +++ b/crates/runtime-next/src/leader/materialize/startup.rs @@ -37,6 +37,7 @@ pub(super) struct Startup { )] pub(super) async fn run( build: String, + ops_stats_journal: String, reactors: Vec, shard_rx: &mut Vec>>, shard_tx: &Vec>>, @@ -77,7 +78,6 @@ pub(super) async fn run( // Build task definition. let proto::Task { - ops_stats_journal, preview, max_transactions, spec: spec_bytes, diff --git a/crates/runtime-next/src/shard/materialize/connector.rs b/crates/runtime-next/src/shard/materialize/connector.rs index 8e82716c109..06cff02c0b0 100644 --- a/crates/runtime-next/src/shard/materialize/connector.rs +++ b/crates/runtime-next/src/shard/materialize/connector.rs @@ -11,13 +11,13 @@ use zeroize::Zeroize; // plus OpenExtras with decrypted trigger configs and connector metadata. pub async fn start( service: &crate::shard::Service, + log_level: ops::LogLevel, 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); diff --git a/crates/runtime-next/src/shard/materialize/handler.rs b/crates/runtime-next/src/shard/materialize/handler.rs index 5d06ee2fc2b..97cbe57f637 100644 --- a/crates/runtime-next/src/shard/materialize/handler.rs +++ b/crates/runtime-next/src/shard/materialize/handler.rs @@ -34,25 +34,34 @@ where } proto::Materialize { - spec: Some(spec), .. + spec: Some(spec), + log_level, + .. } => { + let log_level = + ops::LogLevel::try_from(log_level).unwrap_or(ops::LogLevel::UndefinedLevel); + service.set_log_level(log_level); let request = materialize::Request { spec: Some(spec), ..Default::default() }; - let response = serve_unary(&service, request).await?; + let response = serve_unary(&service, request, log_level).await?; _ = controller_tx.send(Ok(response)); } proto::Materialize { validate: Some(validate), + log_level, .. } => { + let log_level = + ops::LogLevel::try_from(log_level).unwrap_or(ops::LogLevel::UndefinedLevel); + service.set_log_level(log_level); let request = materialize::Request { validate: Some(validate), ..Default::default() }; - let response = serve_unary(&service, request).await?; + let response = serve_unary(&service, request, log_level).await?; _ = controller_tx.send(Ok(response)); } @@ -65,12 +74,14 @@ where pub async fn serve_unary( service: &crate::shard::Service, request: materialize::Request, + log_level: ops::LogLevel, ) -> 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?; + let (connector_tx, mut connector_rx, _container) = + connector::start(service, log_level, request).await?; std::mem::drop(connector_tx); // Send EOF. // Read connector response, and verify it matches the request type. @@ -160,10 +171,13 @@ where .context("missing shard for shard index")?; let labeling = labeling.as_ref().context("missing shard labeling")?.clone(); + let log_level = labeling.log_level(); let shard_id = shard_id.clone(); let shard_index = join.shard_index; let shuffle_directory = join.shuffle_directory.clone(); + service.set_log_level(log_level); + let (joined, leader_stream) = startup::dial_and_join(join).await?; // Forward Joined to controller. @@ -194,6 +208,7 @@ where labeling, leader_rx, leader_tx, + log_level, service, shard_id, shard_index, diff --git a/crates/runtime-next/src/shard/materialize/startup.rs b/crates/runtime-next/src/shard/materialize/startup.rs index 88b82af7281..b649f0fd725 100644 --- a/crates/runtime-next/src/shard/materialize/startup.rs +++ b/crates/runtime-next/src/shard/materialize/startup.rs @@ -81,6 +81,7 @@ pub(super) async fn run( labeling: ops::proto::ShardLabeling, mut leader_rx: tonic::Streaming, leader_tx: mpsc::UnboundedSender, + log_level: ops::LogLevel, service: &crate::shard::Service, shard_id: String, shard_index: u32, @@ -107,7 +108,6 @@ where let proto::Task { max_transactions: _, - ops_stats_journal, preview, spec: spec_bytes, } = l_task; @@ -128,7 +128,7 @@ where crate::Publisher::new_real( shard_id, // Shard ID is AuthZ subject. &service.publisher_factory, - &ops_stats_journal, + &labeling.stats_journal, [], // No additional bindings. ) .context("creating publisher")? @@ -200,6 +200,7 @@ where apply: Some(apply), ..Default::default() }, + log_level, ) .await?, ); @@ -250,7 +251,7 @@ where ..Default::default() }; let (connector_tx, mut connector_rx, container) = - super::connector::start(service, initial).await?; + super::connector::start(service, log_level, initial).await?; // Read C:Opened from the connector. let verify = crate::verify("Materialize", "Opened", "connector"); diff --git a/go/protocols/runtime/runtime.pb.go b/go/protocols/runtime/runtime.pb.go index ce416e7fc5f..cf8d509a1a5 100644 --- a/go/protocols/runtime/runtime.pb.go +++ b/go/protocols/runtime/runtime.pb.go @@ -1514,14 +1514,10 @@ var xxx_messageInfo_Joined proto.InternalMessageInfo 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 stats are written. - // The journal is pre-created by activate; the runtime appends directly - // without consulting the partitions watch or creating new partitions. - OpsStatsJournal string `protobuf:"bytes,2,opt,name=ops_stats_journal,json=opsStatsJournal,proto3" json:"ops_stats_journal,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 bool `protobuf:"varint,2,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"` + MaxTransactions uint32 `protobuf:"varint,3,opt,name=max_transactions,json=maxTransactions,proto3" json:"max_transactions,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2115,6 +2111,11 @@ type Materialize struct { 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. Effective only on unary `spec` / `validate` + // messages, which never see the Join-time labeling that supplies the + // log level for session-bound work. Ignored on all other variants + // (the leader → shard messages MUST NOT set this). + LogLevel ops.Log_Level `protobuf:"varint,5,opt,name=log_level,json=logLevel,proto3,enum=ops.Log_Level" json:"log_level,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"` @@ -2920,243 +2921,242 @@ func init() { } var fileDescriptor_73af6e0737ce390c = []byte{ - // 3774 bytes of a gzipped FileDescriptorProto + // 3757 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x3a, 0x3b, 0x70, 0x1c, 0x47, 0x76, 0xda, 0xff, 0xce, 0xdb, 0xc5, 0x62, 0xb7, 0x09, 0x52, 0xa3, 0x95, 0x44, 0x40, 0x2b, 0xc9, 0xc2, 0xf1, 0xb3, 0xc0, 0x41, 0xba, 0x3b, 0x49, 0x75, 0x3c, 0x11, 0x58, 0x00, 0x12, 0x78, 0x20, 0x09, 0x37, 0x40, 0x96, 0xed, 0x64, 0x6a, 0x30, 0xdd, 0x58, 0x0c, 0x31, 0x3b, 0x3d, 0x9a, 0x9e, - 0x05, 0xb8, 0x97, 0x3a, 0x71, 0xd5, 0x05, 0x4e, 0x9c, 0x39, 0xb1, 0x23, 0x57, 0xb9, 0xec, 0xc0, - 0xe9, 0x05, 0x8e, 0x1c, 0x28, 0x74, 0xb9, 0x5c, 0x2e, 0x47, 0xac, 0xf2, 0x39, 0x75, 0x76, 0x76, - 0x95, 0xcd, 0x72, 0x70, 0xd5, 0x9f, 0xf9, 0xec, 0x0f, 0x22, 0x21, 0x05, 0x0a, 0xc8, 0x9d, 0x7e, - 0xbf, 0x79, 0xfd, 0xfa, 0xfd, 0xfa, 0x0d, 0xa0, 0xd3, 0x67, 0x6b, 0x41, 0xc8, 0x22, 0xe6, 0x30, - 0x8f, 0xaf, 0x85, 0x43, 0x3f, 0x72, 0x07, 0x34, 0xfe, 0xed, 0x4a, 0x0c, 0xaa, 0xe8, 0x65, 0xfb, - 0xe6, 0x71, 0xc8, 0xce, 0x68, 0x98, 0x30, 0x24, 0x0f, 0x8a, 0xb0, 0xbd, 0xe2, 0x30, 0x9f, 0x0f, - 0x07, 0x97, 0x51, 0x8c, 0xbd, 0x8e, 0xd0, 0xd0, 0x3d, 0xa7, 0xfa, 0x47, 0x53, 0xbc, 0x33, 0x46, - 0x71, 0xe2, 0xb1, 0x0b, 0xf9, 0x9f, 0xc6, 0xde, 0x1a, 0xc3, 0x0e, 0xec, 0x88, 0x86, 0xae, 0xed, - 0xb9, 0xbf, 0xa2, 0xd9, 0x67, 0x4d, 0xdb, 0x1e, 0xa3, 0x65, 0x81, 0xfc, 0xa7, 0x71, 0xe3, 0xdb, - 0xe6, 0xa7, 0xc3, 0x93, 0x13, 0x8f, 0xc6, 0xbf, 0x9a, 0x66, 0xa9, 0xcf, 0xfa, 0x4c, 0x3e, 0xae, - 0x89, 0x27, 0x05, 0xed, 0xfc, 0x63, 0x0e, 0x5a, 0x47, 0x36, 0x3f, 0x3b, 0xa4, 0xe1, 0xb9, 0xeb, - 0xd0, 0x1e, 0xf3, 0x4f, 0xdc, 0x3e, 0xba, 0x09, 0x35, 0x8f, 0xf5, 0xad, 0x13, 0xd7, 0xa3, 0xd6, - 0x09, 0x31, 0x73, 0x2b, 0xb9, 0xd5, 0x12, 0x36, 0x3c, 0xd6, 0xdf, 0x75, 0x3d, 0xba, 0x4b, 0xd0, - 0xdb, 0x60, 0x44, 0x36, 0x3f, 0xb3, 0x7c, 0x7b, 0x40, 0xcd, 0xfc, 0x4a, 0x6e, 0xd5, 0xc0, 0x55, - 0x01, 0x78, 0x64, 0x0f, 0x28, 0x7a, 0x0b, 0xaa, 0x43, 0xc2, 0xad, 0xc0, 0x8e, 0x4e, 0xcd, 0x82, - 0xc4, 0x55, 0x86, 0x84, 0x1f, 0xd8, 0xd1, 0x29, 0xba, 0x0d, 0x2d, 0x87, 0xf9, 0x91, 0xed, 0xfa, - 0x34, 0xb4, 0x7c, 0x1a, 0x5d, 0xb0, 0xf0, 0xcc, 0x2c, 0x4a, 0x9a, 0x66, 0x82, 0x78, 0xa4, 0xe0, - 0xe8, 0x03, 0x28, 0x05, 0x9e, 0xed, 0x53, 0xb3, 0xbc, 0x92, 0x5b, 0x6d, 0x6c, 0x34, 0xba, 0xf1, - 0x31, 0x1e, 0x08, 0x28, 0x56, 0xc8, 0xce, 0xff, 0x15, 0xa1, 0x71, 0xa8, 0x36, 0x8a, 0xe9, 0xd7, - 0x43, 0xca, 0x23, 0xb4, 0x07, 0x95, 0x67, 0x6c, 0x18, 0xfa, 0xb6, 0x27, 0x35, 0x37, 0xb6, 0xd6, - 0x5e, 0xbe, 0x58, 0xbe, 0xdd, 0x67, 0xdd, 0xbe, 0xfd, 0x2b, 0x1a, 0x45, 0xb4, 0x4b, 0xe8, 0xf9, - 0x9a, 0xc3, 0x42, 0xba, 0x36, 0xe1, 0x00, 0xdd, 0x07, 0x8a, 0x0d, 0xc7, 0xfc, 0xe8, 0x06, 0x94, - 0x43, 0x1a, 0x78, 0xf6, 0x48, 0xee, 0xb2, 0x8a, 0xf5, 0x4a, 0xec, 0xf1, 0x78, 0xe8, 0x7a, 0xc4, - 0x72, 0x49, 0xbc, 0x47, 0xb9, 0xde, 0x23, 0x68, 0x17, 0xca, 0xec, 0xe4, 0x84, 0xd3, 0x48, 0x6e, - 0xac, 0xb0, 0xd5, 0x7d, 0xf9, 0x62, 0xf9, 0xd6, 0xab, 0xbc, 0xfc, 0xb1, 0xe4, 0xc2, 0x9a, 0x1b, - 0x3d, 0x04, 0xa0, 0x3e, 0xb1, 0xb4, 0xac, 0xd2, 0x95, 0x64, 0x19, 0xd4, 0x27, 0xea, 0x11, 0xdd, - 0x86, 0x52, 0x68, 0xfb, 0x7d, 0x65, 0xcd, 0xda, 0xc6, 0x62, 0x57, 0xba, 0x21, 0x16, 0xa0, 0xc3, - 0x80, 0x3a, 0x5b, 0xc5, 0x6f, 0x5e, 0x2c, 0xbf, 0x81, 0x15, 0x0d, 0x3a, 0x84, 0x9a, 0xc3, 0x58, - 0x48, 0x5c, 0xdf, 0x8e, 0x58, 0x68, 0x56, 0xa4, 0x15, 0x7f, 0xfc, 0xf2, 0xc5, 0xf2, 0xdd, 0x59, - 0x2f, 0x9f, 0x0a, 0x93, 0xee, 0xe1, 0xa9, 0x1d, 0x92, 0xbd, 0x6d, 0x9c, 0x95, 0x82, 0xd6, 0x01, - 0x42, 0xca, 0x99, 0x37, 0x8c, 0x5c, 0xe6, 0x9b, 0x55, 0xa9, 0x46, 0xb3, 0x9b, 0xf0, 0x7c, 0x45, - 0x6d, 0x42, 0x43, 0x9c, 0xa1, 0x41, 0xef, 0xc3, 0x82, 0xf6, 0x61, 0xcb, 0xf5, 0x09, 0x7d, 0x6e, - 0x1a, 0x2b, 0xb9, 0xd5, 0x05, 0x5c, 0xd7, 0xc0, 0x3d, 0x01, 0x43, 0x9f, 0x00, 0xc8, 0x88, 0xb3, - 0xa5, 0x58, 0x90, 0x62, 0x97, 0xd4, 0xee, 0x7a, 0xcc, 0xf3, 0xa8, 0x23, 0xe0, 0x62, 0x8b, 0x38, - 0x43, 0x87, 0x7a, 0xb0, 0x98, 0x86, 0x98, 0x62, 0xad, 0x49, 0xd6, 0xb7, 0x14, 0xeb, 0xc3, 0x71, - 0xa4, 0xe4, 0x9f, 0xe4, 0xe8, 0xfc, 0x4b, 0x11, 0x16, 0x13, 0xdf, 0xe3, 0x01, 0xf3, 0x39, 0x45, - 0xab, 0x50, 0xe6, 0x91, 0x1d, 0x0d, 0xb9, 0xf4, 0xbd, 0xc6, 0x46, 0xb3, 0x1b, 0x9b, 0xa7, 0x7b, - 0x28, 0xe1, 0x58, 0xe3, 0x05, 0xe5, 0xa9, 0xdc, 0xb3, 0xf4, 0xad, 0x59, 0xb6, 0xd0, 0x78, 0xf4, - 0x21, 0x34, 0x22, 0x1a, 0x0e, 0x5c, 0xdf, 0xf6, 0x2c, 0x1a, 0x86, 0x2c, 0xd4, 0x3e, 0xb7, 0x10, - 0x43, 0x77, 0x04, 0x10, 0xfd, 0x21, 0xd4, 0x43, 0x6a, 0x13, 0x2b, 0x3a, 0x0d, 0xd9, 0xb0, 0x7f, - 0x7a, 0x45, 0xff, 0xab, 0x09, 0x19, 0x47, 0x4a, 0x84, 0x70, 0xc2, 0x8b, 0xd0, 0x8d, 0xa8, 0x25, - 0x34, 0xb9, 0xaa, 0x13, 0x4a, 0x09, 0x62, 0x4b, 0x68, 0x0f, 0x4a, 0x76, 0x48, 0x7d, 0x5b, 0x3a, - 0x61, 0x7d, 0xeb, 0xe3, 0x97, 0x2f, 0x96, 0xd7, 0xfa, 0x6e, 0x74, 0x3a, 0x3c, 0xee, 0x3a, 0x6c, - 0xb0, 0x46, 0x79, 0x34, 0xb4, 0xc3, 0x91, 0x4a, 0x93, 0x53, 0x89, 0xb3, 0xbb, 0x29, 0x58, 0xb1, - 0x92, 0x80, 0x3e, 0x84, 0x22, 0x61, 0x0e, 0x37, 0x2b, 0x2b, 0x85, 0xd5, 0xda, 0x46, 0x4d, 0x9d, - 0xda, 0xa1, 0xe7, 0x3a, 0x54, 0xbb, 0xb2, 0x44, 0xa3, 0xaf, 0xa0, 0xa2, 0x22, 0x88, 0x9b, 0xd5, - 0x95, 0xc2, 0x15, 0xb4, 0x8f, 0xd9, 0x85, 0x9f, 0x0d, 0x87, 0x2e, 0xb1, 0x02, 0x3b, 0x8c, 0xb8, - 0x69, 0xc8, 0xd7, 0xea, 0x28, 0x7a, 0xf2, 0x64, 0x6f, 0xfb, 0x40, 0x80, 0xf5, 0xab, 0x0d, 0x41, - 0x28, 0x01, 0xc2, 0xe9, 0x03, 0xdb, 0x39, 0xa3, 0xc4, 0x3a, 0xa3, 0x23, 0x13, 0xe6, 0x29, 0x6b, - 0x28, 0xa2, 0x5f, 0xd2, 0x51, 0x87, 0x40, 0x0b, 0x33, 0xe7, 0x8c, 0x6f, 0x6f, 0x6d, 0x53, 0xee, - 0x84, 0x6e, 0x20, 0x62, 0xe7, 0x0e, 0xa0, 0x50, 0x00, 0xc9, 0xb1, 0x45, 0xfd, 0x73, 0x6b, 0x40, - 0x07, 0x41, 0x14, 0x4a, 0x0f, 0x2b, 0xe3, 0xa6, 0xc6, 0xec, 0xf8, 0xe7, 0x0f, 0x25, 0x1c, 0xbd, - 0x07, 0xf5, 0x98, 0x5a, 0x66, 0x61, 0x95, 0xa1, 0x6b, 0x1a, 0x26, 0x32, 0x71, 0xe7, 0x2f, 0xf2, - 0x60, 0xf4, 0xe2, 0x8c, 0x8b, 0xde, 0x84, 0x8a, 0x1b, 0x58, 0x36, 0x21, 0x4a, 0xa6, 0x81, 0xcb, - 0x6e, 0xb0, 0x49, 0x48, 0x88, 0x7e, 0x0a, 0x0b, 0x3a, 0x4d, 0x5b, 0x01, 0x13, 0xfb, 0xce, 0xcb, - 0x1d, 0xb4, 0xd4, 0x0e, 0x74, 0xa6, 0x3e, 0x60, 0x61, 0x84, 0xeb, 0x7e, 0xba, 0xe0, 0xe8, 0x10, - 0x5a, 0x03, 0x3b, 0x08, 0x28, 0xb1, 0x4e, 0x19, 0x8f, 0x34, 0x6f, 0x41, 0xf2, 0x7e, 0x94, 0xe4, - 0xf1, 0xe4, 0xfd, 0xdd, 0x87, 0x92, 0xf6, 0x2b, 0xc6, 0x23, 0xc9, 0xbe, 0xe3, 0x47, 0xe1, 0x48, - 0x84, 0xdb, 0x18, 0x14, 0xbd, 0x0b, 0x30, 0xe4, 0x76, 0x9f, 0x5a, 0xa1, 0x1d, 0x51, 0xe9, 0xdd, - 0x79, 0x6c, 0x48, 0x08, 0xb6, 0x23, 0xda, 0xde, 0x82, 0xa5, 0x59, 0x72, 0x50, 0x13, 0x0a, 0xc2, - 0xf6, 0x39, 0x99, 0x3b, 0xc4, 0x23, 0x5a, 0x82, 0xd2, 0xb9, 0xed, 0x0d, 0xe3, 0xd2, 0xa5, 0x16, - 0x9f, 0xe7, 0x3f, 0xcd, 0x75, 0xfe, 0x36, 0x0f, 0xad, 0x9e, 0x1d, 0x44, 0xc3, 0x30, 0xae, 0x26, - 0x3b, 0xcf, 0x45, 0xee, 0x14, 0xb5, 0xcf, 0xf2, 0xe8, 0x39, 0xf5, 0x74, 0x58, 0x37, 0xba, 0xa2, - 0xfa, 0xee, 0xb3, 0x7e, 0x77, 0x5f, 0x40, 0x71, 0xd5, 0x63, 0x7d, 0xf9, 0x84, 0xf6, 0xd2, 0xa3, - 0x22, 0xc9, 0x01, 0xea, 0x10, 0x6f, 0x27, 0x7b, 0x9f, 0x3a, 0x62, 0xdc, 0xd2, 0x5c, 0x99, 0x53, - 0xdf, 0x83, 0x3a, 0x8f, 0xec, 0x30, 0xb2, 0x1c, 0x36, 0x18, 0xb8, 0x91, 0x8c, 0xfa, 0xda, 0xc6, - 0x1f, 0xa4, 0x06, 0x9c, 0xd4, 0x54, 0xa4, 0x98, 0x30, 0xea, 0x49, 0x6a, 0x5c, 0xe3, 0xe9, 0xa2, - 0x8d, 0xa1, 0x96, 0xc1, 0xa1, 0x1e, 0x20, 0x2d, 0xc4, 0x72, 0x4e, 0xa9, 0x73, 0x16, 0x30, 0xd7, - 0x8f, 0xe4, 0xd6, 0x44, 0xf2, 0x4c, 0x32, 0x56, 0x2f, 0xc1, 0xe1, 0x96, 0xa6, 0x4f, 0x41, 0x9d, - 0xff, 0x2f, 0x02, 0x4a, 0x54, 0x50, 0xe9, 0x4f, 0x58, 0x6b, 0x1d, 0x8c, 0xa4, 0x96, 0x6b, 0x91, - 0x68, 0xfa, 0xcc, 0x71, 0x4a, 0x84, 0x3e, 0x87, 0x32, 0x0b, 0xa8, 0x4f, 0x89, 0x36, 0x53, 0x67, - 0x7a, 0x87, 0x89, 0xf8, 0xee, 0x63, 0x49, 0x89, 0x35, 0x07, 0xba, 0x0f, 0x55, 0x47, 0x11, 0x11, - 0x6d, 0x9f, 0x0f, 0x2e, 0xe3, 0xd6, 0x20, 0x82, 0x13, 0x2e, 0xb4, 0x0b, 0x90, 0xb1, 0x41, 0x71, - 0x9e, 0x8d, 0x33, 0x32, 0x52, 0xab, 0x64, 0x38, 0xdb, 0x0f, 0xa1, 0xac, 0x74, 0xfb, 0x5e, 0xac, - 0xdb, 0x7e, 0x0a, 0xd5, 0x58, 0x59, 0xe1, 0xf9, 0x67, 0x74, 0x64, 0xa9, 0x24, 0x21, 0x05, 0xd5, - 0xb1, 0x71, 0x46, 0x47, 0x07, 0x12, 0x20, 0xda, 0x2a, 0x91, 0x95, 0x5c, 0x51, 0x94, 0x78, 0x4c, - 0x95, 0x97, 0x54, 0xcd, 0x14, 0xa1, 0x88, 0xdb, 0x17, 0x00, 0xe9, 0x5b, 0xd0, 0x0a, 0x94, 0x44, - 0x39, 0xe2, 0x5a, 0x3b, 0x90, 0x6e, 0x2d, 0x0a, 0x15, 0xc7, 0x0a, 0x81, 0xbe, 0x84, 0x5a, 0xc0, - 0x3c, 0xcf, 0x0a, 0x29, 0x1f, 0x7a, 0x91, 0x14, 0xdb, 0xb8, 0xdc, 0x3e, 0x07, 0xcc, 0xf3, 0xb0, - 0xa4, 0xc6, 0x10, 0x24, 0xcf, 0x9d, 0x47, 0x00, 0x29, 0x06, 0xd5, 0xa0, 0xb2, 0xf7, 0xe8, 0xe9, - 0xe6, 0xfe, 0xde, 0x76, 0xf3, 0x0d, 0x64, 0x40, 0x09, 0xef, 0x6c, 0x6e, 0xff, 0x71, 0x33, 0x87, - 0x16, 0xc0, 0x78, 0xf4, 0xf8, 0xc8, 0x52, 0xcb, 0x3c, 0xaa, 0x43, 0xb5, 0xf7, 0xf8, 0xf1, 0xbe, - 0xf5, 0x78, 0x77, 0xb7, 0x59, 0x10, 0x4c, 0x78, 0xe7, 0xf0, 0x68, 0x13, 0x1f, 0x35, 0x8b, 0x9d, - 0xff, 0xca, 0x41, 0x73, 0x5b, 0xf6, 0xda, 0x3f, 0x80, 0x50, 0xdd, 0x80, 0xa2, 0x70, 0x48, 0xed, - 0x82, 0x37, 0x13, 0xe6, 0x49, 0x05, 0xa5, 0xfb, 0x62, 0x49, 0xdb, 0xbe, 0x03, 0x45, 0xb1, 0x42, - 0x1f, 0x40, 0x83, 0x7f, 0xed, 0x89, 0x2a, 0x7b, 0x7e, 0xc2, 0xad, 0x61, 0xe8, 0xea, 0x24, 0x5c, - 0x57, 0xd0, 0xa7, 0x27, 0xfc, 0x49, 0xe8, 0x76, 0xfe, 0xbb, 0x00, 0xad, 0x58, 0xda, 0x77, 0x09, - 0xb6, 0xcf, 0x26, 0x82, 0xed, 0xbd, 0x29, 0x5d, 0xe7, 0xc6, 0xda, 0x16, 0x18, 0xc1, 0xf0, 0xd8, - 0x73, 0xf9, 0xe9, 0x8c, 0x60, 0x9b, 0xe6, 0x3e, 0x88, 0x69, 0x71, 0xca, 0x86, 0x7e, 0x0e, 0x95, - 0x13, 0x6f, 0x28, 0x25, 0x14, 0x27, 0x82, 0x7d, 0x5a, 0xc2, 0xae, 0xa2, 0xc4, 0x31, 0xcb, 0xf7, - 0x1d, 0x63, 0x11, 0x18, 0x89, 0x92, 0xe2, 0x52, 0x33, 0xb0, 0x9f, 0x5b, 0x8e, 0xc7, 0x9c, 0x33, - 0x5d, 0x5a, 0xab, 0x03, 0xfb, 0x79, 0x4f, 0xac, 0x27, 0x22, 0x30, 0xff, 0x4a, 0x11, 0x58, 0x98, - 0x13, 0x81, 0xb7, 0xa1, 0xa2, 0x37, 0xf6, 0xed, 0xe1, 0xd7, 0xf9, 0xf3, 0x1c, 0x5c, 0x4f, 0x9b, - 0xd1, 0x1f, 0x80, 0xab, 0x77, 0x7e, 0x93, 0x83, 0x1b, 0x63, 0x1a, 0x7d, 0x17, 0x6f, 0xdc, 0x4c, - 0xdd, 0x41, 0x29, 0x93, 0xb6, 0x07, 0xb3, 0xdf, 0x31, 0xed, 0x13, 0xaf, 0x65, 0xce, 0xdf, 0x14, - 0xa1, 0xd1, 0x63, 0x83, 0x63, 0xd7, 0x4f, 0xae, 0x8b, 0xeb, 0x3a, 0x74, 0x15, 0xcf, 0x3b, 0x19, - 0x7d, 0xb3, 0x64, 0x99, 0xc0, 0x45, 0x77, 0xa1, 0x60, 0x93, 0x58, 0xe1, 0xb7, 0xe7, 0x31, 0x6c, - 0x12, 0x82, 0x05, 0x5d, 0xfb, 0xdf, 0xf2, 0x3a, 0xd0, 0xef, 0x43, 0xf5, 0xd8, 0xf5, 0x89, 0xeb, - 0xf7, 0x85, 0x86, 0x85, 0xf1, 0x5a, 0x35, 0xfd, 0xb6, 0xee, 0x96, 0x22, 0xc6, 0x09, 0x57, 0xfb, - 0xd7, 0x79, 0xa8, 0x68, 0x28, 0x42, 0x50, 0x3c, 0x19, 0x7a, 0xea, 0xe8, 0xab, 0x58, 0x3e, 0xc7, - 0xbd, 0x8e, 0xe8, 0xd2, 0x0c, 0xd5, 0xeb, 0x7c, 0x0a, 0xb5, 0x20, 0x64, 0xcf, 0xd4, 0x35, 0x28, - 0xee, 0xc1, 0x9a, 0xaa, 0x7f, 0x3b, 0x48, 0x10, 0xba, 0x0d, 0xcd, 0x92, 0xa2, 0x7b, 0x50, 0xe3, - 0xce, 0x29, 0x1d, 0xd8, 0xd6, 0x33, 0xce, 0x7c, 0x19, 0xad, 0xf5, 0xad, 0x77, 0x5e, 0xbe, 0x58, - 0x36, 0xa9, 0xef, 0x30, 0xa1, 0xc2, 0x9a, 0x40, 0x74, 0xb1, 0x7d, 0xf1, 0x90, 0x72, 0xd9, 0x86, - 0x81, 0x62, 0x78, 0xc0, 0x99, 0x8f, 0xba, 0x00, 0x9c, 0x86, 0x56, 0xc0, 0x3c, 0xd7, 0x19, 0xc9, - 0xab, 0x43, 0xd2, 0x2f, 0x1f, 0xd2, 0xf0, 0x40, 0x82, 0xb1, 0xc1, 0xe3, 0x47, 0x39, 0x36, 0x90, - 0xfd, 0x75, 0x14, 0xca, 0xeb, 0x81, 0x81, 0x2b, 0xb2, 0x8d, 0x8e, 0x42, 0x71, 0x0b, 0x97, 0x2d, - 0x9a, 0xea, 0xf6, 0x0d, 0xac, 0x57, 0x6d, 0x1f, 0x0a, 0x9b, 0x84, 0x20, 0x13, 0x2a, 0xda, 0x40, - 0xba, 0xc9, 0x8b, 0x97, 0xe8, 0x67, 0x50, 0x25, 0xcc, 0x51, 0xfa, 0xe7, 0x5f, 0x41, 0xff, 0x0a, - 0x61, 0x8e, 0x54, 0x7e, 0x09, 0x4a, 0x27, 0x21, 0xf3, 0x55, 0xcb, 0x55, 0xc5, 0x6a, 0xd1, 0xf9, - 0xf7, 0x1c, 0x2c, 0x26, 0xe7, 0xa4, 0xef, 0x7b, 0xf3, 0x5f, 0x6e, 0x42, 0x85, 0x50, 0x8f, 0x46, - 0xda, 0xb5, 0xab, 0x38, 0x5e, 0x8e, 0xa9, 0x55, 0xb8, 0x92, 0x5a, 0xc5, 0x8c, 0x5a, 0x13, 0xb9, - 0xa9, 0x34, 0x99, 0x9b, 0xde, 0x87, 0x05, 0x65, 0xaf, 0x98, 0x42, 0x5e, 0xbe, 0x70, 0x5d, 0x01, - 0x15, 0x51, 0xe7, 0x4d, 0xb8, 0xde, 0x63, 0xbe, 0x4f, 0x9d, 0x88, 0x85, 0x07, 0x21, 0x7b, 0x3e, - 0xd2, 0x8e, 0xd8, 0xf9, 0xcb, 0x1c, 0xdc, 0x98, 0xc4, 0xe8, 0xad, 0x3f, 0x80, 0x8a, 0xb8, 0x32, - 0x50, 0xce, 0xf5, 0x9c, 0x65, 0xfd, 0xe5, 0x8b, 0xe5, 0x3b, 0xaf, 0x72, 0xb7, 0xda, 0xf1, 0x89, - 0xca, 0xc9, 0xb1, 0x00, 0x71, 0xfa, 0x81, 0x10, 0x6e, 0xb9, 0x44, 0x77, 0xe5, 0x15, 0xb9, 0xde, - 0x23, 0xa8, 0x0d, 0x05, 0x8f, 0xf5, 0x75, 0xbd, 0xa9, 0xc6, 0x19, 0x0e, 0x0b, 0x60, 0xe7, 0x1f, - 0x0a, 0x50, 0x7c, 0xc0, 0x5c, 0x1f, 0xdd, 0x82, 0x16, 0x8d, 0x1c, 0x62, 0x0d, 0x18, 0xb1, 0x42, - 0x7a, 0xee, 0x72, 0x71, 0xa3, 0x17, 0x5a, 0x15, 0xf0, 0xa2, 0x40, 0x3c, 0x64, 0x04, 0x6b, 0x30, - 0xba, 0x0d, 0x65, 0x7e, 0x6a, 0x87, 0x24, 0xbe, 0xcd, 0x5c, 0x4b, 0x82, 0x50, 0x88, 0x52, 0xc3, - 0x0b, 0xac, 0x49, 0xd0, 0x32, 0xd4, 0xe4, 0x93, 0x9e, 0x40, 0x14, 0xe4, 0x19, 0x83, 0x04, 0xa9, - 0xf9, 0xc3, 0x6d, 0x68, 0xc5, 0x43, 0x0a, 0xe2, 0x86, 0xd2, 0x4c, 0xa3, 0x78, 0xa6, 0xa5, 0x11, - 0xdb, 0x31, 0x1c, 0xfd, 0x08, 0x62, 0x98, 0x45, 0xb5, 0x0d, 0xe4, 0x81, 0x19, 0x78, 0x51, 0xc3, - 0x63, 0xd3, 0xa0, 0x8f, 0x60, 0xd1, 0x93, 0xd7, 0xff, 0x94, 0x52, 0x85, 0x45, 0x43, 0x81, 0x63, - 0xc2, 0xf6, 0xdf, 0xe7, 0xa0, 0x24, 0x75, 0x46, 0x0d, 0xc8, 0xbb, 0x44, 0x37, 0x0f, 0x79, 0x97, - 0xa0, 0x2e, 0x54, 0x3d, 0xfb, 0x98, 0x7a, 0xc2, 0x39, 0xf3, 0x3a, 0x1b, 0xcb, 0x8c, 0x28, 0xa8, - 0xf7, 0x35, 0x06, 0x27, 0x34, 0x68, 0x03, 0x2a, 0x21, 0xb5, 0x85, 0xa6, 0xda, 0xda, 0x66, 0x3a, - 0x92, 0x38, 0x08, 0x99, 0x43, 0x39, 0x3f, 0x0c, 0xa8, 0xd3, 0xdd, 0xdb, 0xc6, 0x31, 0x21, 0x5a, - 0x87, 0x25, 0x69, 0x78, 0x27, 0xa4, 0x76, 0x44, 0x53, 0xdb, 0xcb, 0xe1, 0x03, 0x46, 0x02, 0xd7, - 0x93, 0xa8, 0xd8, 0xfc, 0x9d, 0x4f, 0xa0, 0x2c, 0xec, 0x4c, 0x89, 0x38, 0x34, 0x51, 0x71, 0x25, - 0xff, 0xe4, 0xa1, 0x0d, 0xec, 0xe7, 0x3b, 0x91, 0x93, 0x1c, 0x5a, 0xe7, 0xd7, 0x39, 0x28, 0x1e, - 0xd9, 0xfc, 0x4c, 0xa4, 0x3d, 0x1e, 0x50, 0x47, 0x77, 0xc1, 0xf2, 0x59, 0x08, 0x62, 0x01, 0xb7, - 0x64, 0x8a, 0xb7, 0xe2, 0xd9, 0x9f, 0x72, 0xa3, 0x45, 0x16, 0x70, 0x59, 0x01, 0xf4, 0x6c, 0x4f, - 0x84, 0x65, 0x20, 0x5e, 0x46, 0x2f, 0x74, 0x14, 0xc5, 0x4b, 0x71, 0x38, 0x42, 0x9d, 0x28, 0xb4, - 0x7d, 0x6e, 0xeb, 0x7c, 0x59, 0x92, 0xe7, 0x2d, 0xb4, 0x39, 0xca, 0x80, 0x3b, 0x7f, 0x5d, 0x82, - 0x0a, 0xa6, 0x0e, 0x3b, 0x97, 0x25, 0xac, 0x66, 0x3b, 0x67, 0x96, 0xeb, 0x47, 0xd4, 0x8f, 0xe2, - 0xc4, 0xbe, 0x92, 0xd6, 0x54, 0x45, 0xd6, 0xdd, 0x74, 0xce, 0xf6, 0x14, 0x89, 0xba, 0xde, 0x82, - 0x9d, 0x00, 0xd0, 0x06, 0x5c, 0x57, 0x57, 0xbc, 0x88, 0x12, 0xd1, 0x80, 0x70, 0xaa, 0xdb, 0x90, - 0xbc, 0x6c, 0x43, 0xae, 0x25, 0xc8, 0x9e, 0xc0, 0xa9, 0x8e, 0xe4, 0x3e, 0xa0, 0x94, 0x47, 0x26, - 0x02, 0x97, 0xc6, 0xe7, 0xd6, 0xea, 0xc6, 0xb3, 0xdf, 0x5d, 0x8d, 0xc0, 0xad, 0x84, 0x38, 0x06, - 0xa1, 0x3b, 0xb0, 0xe4, 0xc4, 0x91, 0x2d, 0x6d, 0x47, 0x33, 0x99, 0x1e, 0x37, 0x12, 0x9c, 0x30, - 0x1f, 0x45, 0x77, 0x00, 0x9d, 0x8a, 0x3d, 0x8e, 0x2b, 0x58, 0x52, 0x23, 0x08, 0x85, 0xc9, 0x68, - 0xf7, 0x39, 0x2c, 0x6a, 0xea, 0x44, 0xb5, 0xf2, 0x3c, 0xd5, 0x1a, 0x8a, 0x32, 0xd1, 0xeb, 0x3d, - 0xa8, 0x7b, 0x36, 0x8f, 0x2c, 0x3b, 0x08, 0x3c, 0x97, 0x12, 0x39, 0x7e, 0xac, 0xe3, 0x9a, 0x80, - 0x6d, 0x2a, 0x10, 0xda, 0x84, 0x96, 0x47, 0xfb, 0xb6, 0x33, 0xca, 0x36, 0x7f, 0xd5, 0x4b, 0x9a, - 0xbf, 0xa6, 0x22, 0xcf, 0xdc, 0x7c, 0x3e, 0x05, 0xd1, 0xdd, 0x59, 0x67, 0x74, 0x14, 0x4f, 0x73, - 0xde, 0x9d, 0x3a, 0xb3, 0x87, 0xf6, 0xf3, 0x5f, 0xd2, 0x91, 0x3e, 0xb0, 0xca, 0x40, 0xad, 0xd0, - 0x2d, 0xb8, 0x16, 0x85, 0x6e, 0xbf, 0x2f, 0xaa, 0x9b, 0x1d, 0xda, 0x03, 0xae, 0xcc, 0x06, 0x52, - 0xcd, 0x05, 0x8d, 0x3a, 0x90, 0x98, 0xf6, 0x3d, 0x58, 0x9c, 0x38, 0xf8, 0xec, 0x3c, 0xc2, 0x98, - 0x31, 0x8f, 0xa8, 0x67, 0xe6, 0x11, 0xed, 0xcf, 0xa1, 0x9e, 0xd5, 0xe1, 0xdb, 0x66, 0x19, 0x59, - 0xde, 0xce, 0xbf, 0x96, 0xa1, 0x72, 0x40, 0x43, 0xee, 0xf2, 0x08, 0x5d, 0x87, 0x32, 0xa7, 0x5f, - 0x5b, 0x3e, 0x93, 0xac, 0x45, 0x5c, 0xe2, 0xf4, 0xeb, 0x47, 0x4c, 0x9c, 0xa9, 0xaa, 0x49, 0x56, - 0xd6, 0x83, 0x55, 0xb5, 0x6a, 0x2a, 0x4c, 0xaa, 0xfd, 0xa4, 0xa3, 0x17, 0x26, 0x1c, 0x5d, 0xbf, - 0xeb, 0x6a, 0x8e, 0x5e, 0x7c, 0x5d, 0x47, 0x2f, 0xbd, 0x86, 0xa3, 0xaf, 0xc3, 0x8d, 0xd4, 0xd1, - 0x03, 0x3b, 0x72, 0x4e, 0xa9, 0x3e, 0x33, 0x55, 0x0a, 0x9b, 0x09, 0xf6, 0x40, 0x21, 0xe7, 0x38, - 0x7b, 0x65, 0x8e, 0xb3, 0x7f, 0x02, 0x37, 0xb4, 0x19, 0x27, 0x7d, 0xbe, 0x2a, 0x4d, 0xb9, 0xa4, - 0xb0, 0x5f, 0x8d, 0xbb, 0xf9, 0x8c, 0x10, 0x31, 0xae, 0x1a, 0x22, 0xf0, 0x8a, 0x21, 0x52, 0xbb, - 0x72, 0x88, 0xd4, 0x27, 0x42, 0x24, 0x3e, 0xed, 0xd9, 0x21, 0xb2, 0x01, 0xd7, 0xb5, 0x45, 0xc6, - 0x23, 0xc5, 0x5c, 0x90, 0x06, 0xb9, 0xa6, 0x90, 0x47, 0xd9, 0x50, 0x99, 0x17, 0x56, 0x8d, 0x1f, - 0x58, 0x58, 0x75, 0xc0, 0xd0, 0x7b, 0xa7, 0x64, 0x4e, 0x5c, 0x75, 0xfe, 0x26, 0x07, 0x25, 0x71, - 0x0e, 0xa3, 0x99, 0xd5, 0xca, 0x84, 0xca, 0xb9, 0x90, 0xa0, 0x9b, 0x52, 0x03, 0xc7, 0x4b, 0x71, - 0x05, 0x95, 0xc7, 0x2a, 0x59, 0x54, 0x1a, 0xae, 0x0a, 0x80, 0xa8, 0xba, 0xc9, 0x99, 0xc7, 0xbc, - 0xaa, 0x6f, 0x90, 0x67, 0xfe, 0x54, 0xf3, 0xaf, 0xcf, 0xc9, 0xe8, 0xca, 0xcd, 0xd1, 0x78, 0x46, - 0x17, 0x1d, 0x65, 0xe7, 0x19, 0x54, 0x62, 0x87, 0xb9, 0x0b, 0x48, 0x95, 0xb7, 0xe4, 0x86, 0x18, - 0x97, 0x63, 0x03, 0xb7, 0x14, 0x66, 0x3b, 0x45, 0x5c, 0x12, 0x54, 0xf9, 0xd9, 0x41, 0xd5, 0xf9, - 0x5d, 0x4e, 0xdf, 0x83, 0x5e, 0xcf, 0x28, 0x1f, 0xc6, 0x5f, 0xae, 0x0a, 0x33, 0xbf, 0x5c, 0xc5, - 0xdf, 0xac, 0xde, 0xbf, 0xb4, 0x9a, 0xc9, 0xeb, 0x1f, 0x45, 0x3f, 0xc9, 0x78, 0x74, 0x49, 0x7a, - 0x74, 0x7a, 0xf9, 0x95, 0x57, 0xae, 0x99, 0xee, 0xfc, 0x9d, 0xfc, 0x05, 0xa0, 0x2a, 0x53, 0xc5, - 0x23, 0x76, 0xd1, 0x29, 0x43, 0xf1, 0x30, 0x62, 0x41, 0xc7, 0x80, 0x8a, 0xf8, 0x0d, 0x28, 0xe9, - 0xfc, 0x11, 0xd4, 0x0e, 0x29, 0x17, 0x1b, 0xdd, 0x67, 0x2c, 0x98, 0x73, 0x4f, 0xcf, 0x5d, 0xe5, - 0x9e, 0xfe, 0x77, 0x37, 0xa0, 0x96, 0xb9, 0x43, 0xa3, 0xbb, 0x19, 0xab, 0xd7, 0x36, 0xde, 0xea, - 0x66, 0x3f, 0x31, 0xc7, 0xd7, 0x4e, 0x69, 0x50, 0x75, 0x20, 0x5f, 0xc0, 0x82, 0xf8, 0xb5, 0x42, - 0xdd, 0xee, 0x27, 0xc3, 0x82, 0x71, 0x3e, 0x85, 0x54, 0x8c, 0x75, 0xc1, 0x90, 0x5c, 0x0f, 0x3e, - 0x83, 0xea, 0xb9, 0xed, 0xb9, 0xc4, 0x8e, 0xe2, 0xa3, 0x7b, 0x77, 0xe6, 0x3b, 0x9f, 0x6a, 0x22, - 0x9c, 0x90, 0xa3, 0x7b, 0x60, 0xc4, 0xcf, 0xf1, 0x98, 0x68, 0x79, 0xf6, 0x7b, 0x63, 0x66, 0x82, - 0x53, 0x0e, 0xf4, 0x33, 0xa8, 0x73, 0x65, 0x53, 0xcb, 0x63, 0x2c, 0x30, 0x97, 0x74, 0xd6, 0x8b, - 0xcd, 0x97, 0x31, 0x38, 0xae, 0xf1, 0x8c, 0xf5, 0xdf, 0x83, 0xe2, 0x33, 0xe6, 0xfa, 0xe6, 0x75, - 0xc9, 0xb0, 0x30, 0x76, 0x2f, 0xc0, 0x12, 0x85, 0x3e, 0x82, 0xf2, 0x33, 0xd9, 0xbd, 0x9a, 0x37, - 0xb4, 0x3b, 0x66, 0x89, 0x28, 0xc1, 0x1a, 0x2d, 0x64, 0x45, 0x36, 0x3f, 0x33, 0xdf, 0x9c, 0x90, - 0x25, 0x9a, 0x58, 0x2c, 0x51, 0xe8, 0x96, 0xe8, 0xb7, 0x65, 0xa7, 0x61, 0x9a, 0xfa, 0x13, 0xe0, - 0x44, 0x07, 0x82, 0x63, 0x02, 0xf4, 0x01, 0x94, 0x44, 0xb2, 0x1f, 0x99, 0x6f, 0x49, 0xca, 0xf4, - 0x6b, 0xb8, 0xcc, 0x33, 0x58, 0x21, 0x85, 0xc4, 0xb8, 0x24, 0xb4, 0x27, 0x24, 0xea, 0x30, 0xc7, - 0x31, 0x81, 0x50, 0x50, 0xce, 0x3d, 0xde, 0x9e, 0x50, 0x30, 0x33, 0xe8, 0xf8, 0x38, 0x99, 0x15, - 0xbe, 0x33, 0x31, 0xeb, 0xc8, 0x38, 0xd6, 0xe4, 0x94, 0xf0, 0x2e, 0x14, 0x3d, 0x66, 0x13, 0x73, - 0x55, 0xfb, 0xd9, 0x2c, 0x96, 0x7d, 0x66, 0x13, 0x2c, 0xc9, 0xc4, 0x3b, 0xc4, 0x2f, 0x25, 0xe6, - 0x8f, 0x2e, 0x79, 0xc7, 0xbe, 0x24, 0xc1, 0x9a, 0x14, 0xad, 0x43, 0x49, 0x8e, 0x7f, 0xcc, 0x5b, - 0x13, 0x91, 0x91, 0xe5, 0x91, 0x53, 0x21, 0xac, 0x08, 0xd1, 0x4f, 0xd3, 0x41, 0xd3, 0xed, 0x89, - 0x41, 0xcf, 0x14, 0x4f, 0x66, 0xba, 0x24, 0xde, 0xc4, 0x23, 0x16, 0x52, 0xf3, 0xce, 0x25, 0x6f, - 0x3a, 0x14, 0x14, 0x58, 0x11, 0x8a, 0x0d, 0xc9, 0x07, 0x62, 0xde, 0xbd, 0x64, 0x43, 0x92, 0x45, - 0x5c, 0x33, 0xe5, 0x2f, 0xea, 0x4d, 0x7c, 0xea, 0xe9, 0x4a, 0xd6, 0x95, 0x39, 0xac, 0xb3, 0x3f, - 0xf2, 0xa0, 0x3d, 0x68, 0xc8, 0xa5, 0x68, 0x5b, 0x94, 0x98, 0xb5, 0x89, 0x11, 0xeb, 0x94, 0x18, - 0x4a, 0xb4, 0xa0, 0x05, 0x9e, 0x5d, 0xa2, 0x2d, 0xd9, 0xeb, 0xf9, 0xec, 0xc2, 0xa3, 0xa4, 0x4f, - 0xcd, 0xf5, 0x4b, 0xd4, 0xd9, 0x4c, 0xe9, 0x70, 0x96, 0x09, 0xed, 0x40, 0x3d, 0xb3, 0x24, 0xe6, - 0x8f, 0x27, 0xe6, 0xcd, 0x73, 0x84, 0x10, 0x3c, 0xc6, 0x26, 0x7c, 0x3a, 0x50, 0x05, 0xd7, 0xdc, - 0x98, 0xf0, 0x69, 0x5d, 0x88, 0x71, 0x4c, 0x80, 0xd6, 0xc1, 0x08, 0xe2, 0xe2, 0x6c, 0x7e, 0x3c, - 0x31, 0x80, 0x4c, 0xca, 0x36, 0x4e, 0x89, 0x50, 0x17, 0x0c, 0xd5, 0xe2, 0xf9, 0xec, 0xc2, 0xfc, - 0x44, 0xf7, 0x5f, 0xc9, 0x50, 0x4e, 0x27, 0x6e, 0x5c, 0x75, 0xf4, 0x93, 0x88, 0x1a, 0x1e, 0xb1, - 0xc0, 0xfc, 0xf9, 0x44, 0xd4, 0x88, 0x7c, 0x8e, 0x25, 0x4a, 0x28, 0xcc, 0x55, 0x76, 0x37, 0xef, - 0x4d, 0x28, 0xac, 0xb3, 0x3e, 0x8e, 0x09, 0xda, 0x7f, 0x9a, 0x4b, 0x26, 0xda, 0xaf, 0x3f, 0x3c, - 0xfd, 0x32, 0x5b, 0xf2, 0x32, 0x5d, 0x5e, 0xfe, 0x92, 0x2e, 0xef, 0x5a, 0xc2, 0x91, 0x99, 0x83, - 0xff, 0x04, 0x8a, 0x22, 0xc0, 0xd0, 0x5d, 0xa8, 0x26, 0xbd, 0x68, 0x6e, 0x5e, 0x2f, 0x9a, 0x90, - 0xb4, 0x7f, 0x97, 0x87, 0xb2, 0x0a, 0x4c, 0xf4, 0xc5, 0xd4, 0x68, 0xf3, 0xfd, 0x4b, 0xe2, 0x78, - 0x7a, 0xb2, 0xa9, 0x5a, 0x17, 0x39, 0x5a, 0x0b, 0x2d, 0xf5, 0x95, 0xf7, 0x78, 0x14, 0x51, 0x75, - 0x19, 0x29, 0x8a, 0xd6, 0x45, 0xe1, 0x9e, 0x08, 0xd4, 0x96, 0xc0, 0xb4, 0xff, 0x27, 0x97, 0xce, - 0x42, 0x97, 0xa0, 0xa4, 0xe6, 0x33, 0xaa, 0x24, 0xab, 0x05, 0x5a, 0x85, 0xe6, 0xc0, 0xf5, 0x2d, - 0xce, 0x86, 0xa1, 0x33, 0x7e, 0xa3, 0x6e, 0x0c, 0x5c, 0xff, 0x50, 0x82, 0x55, 0x07, 0xbf, 0xaa, - 0xae, 0xfe, 0x63, 0x94, 0x05, 0x4d, 0x69, 0x3f, 0xcf, 0x52, 0xde, 0x01, 0xa4, 0xa8, 0x88, 0x45, - 0x98, 0xc3, 0xad, 0x88, 0x45, 0xb6, 0x27, 0x6b, 0x54, 0x11, 0x37, 0x35, 0x66, 0x9b, 0x39, 0xfc, - 0x48, 0xc0, 0x51, 0x17, 0xae, 0xc5, 0xd4, 0x72, 0x3b, 0x9a, 0xbc, 0x24, 0xc9, 0x5b, 0x1a, 0x25, - 0xb7, 0xa3, 0xe8, 0x3b, 0xb0, 0xa0, 0xfb, 0x13, 0x8b, 0x50, 0x2f, 0xd2, 0x7f, 0x28, 0x81, 0x6b, - 0xaa, 0x11, 0xd9, 0x16, 0xa0, 0xf6, 0x67, 0x50, 0x92, 0x59, 0xea, 0x92, 0x0e, 0x2c, 0x37, 0xbb, - 0x03, 0x6b, 0xff, 0x6f, 0x2e, 0x9d, 0x95, 0x5f, 0x36, 0x8c, 0x9e, 0x91, 0x11, 0x67, 0x1e, 0xd9, - 0x6b, 0x76, 0x80, 0xed, 0xd1, 0xb7, 0x9d, 0xd8, 0x2d, 0x68, 0xa9, 0x0c, 0x9f, 0x35, 0xae, 0x72, - 0x81, 0x45, 0x85, 0x48, 0x6d, 0x7b, 0x07, 0x90, 0xa6, 0xcd, 0x9a, 0xb6, 0xa0, 0x4e, 0x42, 0x61, - 0x52, 0xcb, 0xb6, 0x2b, 0x50, 0x92, 0x29, 0xb7, 0xfd, 0x4f, 0x39, 0x28, 0xab, 0xe4, 0xfb, 0xca, - 0x4e, 0xab, 0xc8, 0x67, 0x8c, 0xe3, 0x5f, 0x65, 0x3f, 0x2a, 0xc1, 0xcf, 0xd8, 0x8f, 0x42, 0x8c, - 0xed, 0x47, 0xd3, 0xce, 0xd8, 0x8f, 0xc2, 0x64, 0xf6, 0xf3, 0x67, 0xb9, 0xf1, 0x2f, 0xfa, 0xaf, - 0xed, 0x0c, 0xdf, 0x5f, 0xf6, 0xd8, 0x84, 0x85, 0xb1, 0x5a, 0x72, 0x05, 0xc7, 0xfc, 0x02, 0x6a, - 0x99, 0x0a, 0x70, 0x05, 0x01, 0xf7, 0xa1, 0x9e, 0x2d, 0x21, 0xaf, 0x2f, 0xe1, 0xd6, 0x6d, 0x28, - 0xc9, 0x3f, 0x2c, 0x44, 0x00, 0xe5, 0x83, 0x27, 0x5b, 0xfb, 0x7b, 0xbd, 0xe6, 0x1b, 0xa8, 0x06, - 0x95, 0x03, 0xbc, 0xf7, 0x74, 0xf3, 0x68, 0xa7, 0x99, 0x43, 0x06, 0x94, 0xf6, 0x1f, 0xf7, 0x36, - 0xf7, 0x9b, 0xf9, 0x8d, 0x07, 0x50, 0xd5, 0x7f, 0xf8, 0x15, 0xa2, 0x5f, 0x40, 0x45, 0x3f, 0xa3, - 0x37, 0xd3, 0x44, 0x3f, 0xf6, 0x27, 0x89, 0x6d, 0x73, 0x1a, 0xa1, 0x1a, 0xd8, 0xf5, 0xdc, 0xc6, - 0x3e, 0x54, 0xf5, 0x47, 0x85, 0x10, 0xdd, 0x87, 0x8a, 0x7e, 0xce, 0xc8, 0x1a, 0xff, 0x34, 0x94, - 0x91, 0x35, 0xf1, 0x2d, 0x62, 0x35, 0xb7, 0x9e, 0xdb, 0x38, 0x85, 0xc6, 0xf8, 0xb8, 0x1e, 0x3d, - 0x85, 0x45, 0xf9, 0x90, 0x80, 0x39, 0xba, 0x99, 0xad, 0x2b, 0xd3, 0x43, 0xff, 0xf6, 0xf2, 0x5c, - 0x7c, 0xe6, 0x4d, 0x5f, 0x42, 0x79, 0x5f, 0xfd, 0x7d, 0xda, 0xbd, 0xf1, 0x8b, 0xc6, 0xd2, 0xac, - 0x20, 0x6a, 0xcf, 0x84, 0x4a, 0x41, 0xbb, 0xf1, 0xfc, 0xfa, 0xbb, 0xc9, 0xd9, 0xfa, 0xc5, 0x37, - 0xff, 0x71, 0xf3, 0x8d, 0x6f, 0x7e, 0x7b, 0x33, 0xf7, 0xcf, 0xbf, 0xbd, 0x99, 0xfb, 0xab, 0xff, - 0xbc, 0x99, 0xfb, 0x93, 0x3b, 0xaf, 0xf4, 0xc7, 0x65, 0x5a, 0xde, 0x71, 0x59, 0x82, 0x3e, 0xfe, - 0x7d, 0x00, 0x00, 0x00, 0xff, 0xff, 0x8b, 0x5e, 0x96, 0xbe, 0x45, 0x2c, 0x00, 0x00, + 0x05, 0xb8, 0x97, 0x3a, 0x71, 0x95, 0x03, 0x27, 0xce, 0x9c, 0xd8, 0x91, 0xab, 0x5c, 0xe5, 0xc0, + 0xe9, 0x05, 0x8e, 0x1c, 0x28, 0x74, 0xb9, 0x5c, 0x2e, 0x3b, 0x61, 0x95, 0xcf, 0xa9, 0x33, 0xd9, + 0x55, 0x36, 0xcb, 0x81, 0xab, 0x3f, 0xf3, 0xd9, 0x1f, 0x44, 0x42, 0x0a, 0x14, 0x90, 0x3b, 0xfd, + 0x7e, 0xf3, 0xfa, 0xf5, 0xfb, 0xf5, 0x1b, 0x40, 0xa7, 0xcf, 0xd6, 0x82, 0x90, 0x45, 0xcc, 0x61, + 0x1e, 0x5f, 0x0b, 0x87, 0x7e, 0xe4, 0x0e, 0x68, 0xfc, 0xdb, 0x95, 0x18, 0x54, 0xd1, 0xcb, 0xf6, + 0xcd, 0xe3, 0x90, 0x9d, 0xd1, 0x30, 0x61, 0x48, 0x1e, 0x14, 0x61, 0x7b, 0xc5, 0x61, 0x3e, 0x1f, + 0x0e, 0x2e, 0xa3, 0x18, 0x7b, 0x1d, 0xa1, 0xa1, 0x7b, 0x4e, 0xf5, 0x8f, 0xa6, 0x78, 0x67, 0x8c, + 0xe2, 0xc4, 0x63, 0x17, 0xf2, 0x3f, 0x8d, 0xbd, 0x35, 0x86, 0x1d, 0xd8, 0x11, 0x0d, 0x5d, 0xdb, + 0x73, 0x7f, 0x43, 0xb3, 0xcf, 0x9a, 0xb6, 0x3d, 0x46, 0xcb, 0x02, 0xf9, 0x4f, 0xe3, 0xc6, 0xb7, + 0xcd, 0x4f, 0x87, 0x27, 0x27, 0x1e, 0x8d, 0x7f, 0x35, 0xcd, 0x52, 0x9f, 0xf5, 0x99, 0x7c, 0x5c, + 0x13, 0x4f, 0x0a, 0xda, 0xf9, 0xfb, 0x1c, 0xb4, 0x8e, 0x6c, 0x7e, 0x76, 0x48, 0xc3, 0x73, 0xd7, + 0xa1, 0x3d, 0xe6, 0x9f, 0xb8, 0x7d, 0x74, 0x13, 0x6a, 0x1e, 0xeb, 0x5b, 0x27, 0xae, 0x47, 0xad, + 0x13, 0x62, 0xe6, 0x56, 0x72, 0xab, 0x25, 0x6c, 0x78, 0xac, 0xbf, 0xeb, 0x7a, 0x74, 0x97, 0xa0, + 0xb7, 0xc1, 0x88, 0x6c, 0x7e, 0x66, 0xf9, 0xf6, 0x80, 0x9a, 0xf9, 0x95, 0xdc, 0xaa, 0x81, 0xab, + 0x02, 0xf0, 0xc8, 0x1e, 0x50, 0xf4, 0x16, 0x54, 0x87, 0x84, 0x5b, 0x81, 0x1d, 0x9d, 0x9a, 0x05, + 0x89, 0xab, 0x0c, 0x09, 0x3f, 0xb0, 0xa3, 0x53, 0x74, 0x1b, 0x5a, 0x0e, 0xf3, 0x23, 0xdb, 0xf5, + 0x69, 0x68, 0xf9, 0x34, 0xba, 0x60, 0xe1, 0x99, 0x59, 0x94, 0x34, 0xcd, 0x04, 0xf1, 0x48, 0xc1, + 0xd1, 0x07, 0x50, 0x0a, 0x3c, 0xdb, 0xa7, 0x66, 0x79, 0x25, 0xb7, 0xda, 0xd8, 0x68, 0x74, 0xe3, + 0x63, 0x3c, 0x10, 0x50, 0xac, 0x90, 0x9d, 0xff, 0x2d, 0x42, 0xe3, 0x50, 0x6d, 0x14, 0xd3, 0xaf, + 0x87, 0x94, 0x47, 0x68, 0x0f, 0x2a, 0xcf, 0xd8, 0x30, 0xf4, 0x6d, 0x4f, 0x6a, 0x6e, 0x6c, 0xad, + 0xbd, 0x7c, 0xb1, 0x7c, 0xbb, 0xcf, 0xba, 0x7d, 0xfb, 0x37, 0x34, 0x8a, 0x68, 0x97, 0xd0, 0xf3, + 0x35, 0x87, 0x85, 0x74, 0x6d, 0xc2, 0x01, 0xba, 0x0f, 0x14, 0x1b, 0x8e, 0xf9, 0xd1, 0x0d, 0x28, + 0x87, 0x34, 0xf0, 0xec, 0x91, 0xdc, 0x65, 0x15, 0xeb, 0x95, 0xd8, 0xe3, 0xf1, 0xd0, 0xf5, 0x88, + 0xe5, 0x92, 0x78, 0x8f, 0x72, 0xbd, 0x47, 0xd0, 0x2e, 0x94, 0xd9, 0xc9, 0x09, 0xa7, 0x91, 0xdc, + 0x58, 0x61, 0xab, 0xfb, 0xf2, 0xc5, 0xf2, 0xad, 0x57, 0x79, 0xf9, 0x63, 0xc9, 0x85, 0x35, 0x37, + 0x7a, 0x08, 0x40, 0x7d, 0x62, 0x69, 0x59, 0xa5, 0x2b, 0xc9, 0x32, 0xa8, 0x4f, 0xd4, 0x23, 0xba, + 0x0d, 0xa5, 0xd0, 0xf6, 0xfb, 0xca, 0x9a, 0xb5, 0x8d, 0xc5, 0xae, 0x74, 0x43, 0x2c, 0x40, 0x87, + 0x01, 0x75, 0xb6, 0x8a, 0xdf, 0xbc, 0x58, 0x7e, 0x03, 0x2b, 0x1a, 0x74, 0x08, 0x35, 0x87, 0xb1, + 0x90, 0xb8, 0xbe, 0x1d, 0xb1, 0xd0, 0xac, 0x48, 0x2b, 0xfe, 0xf4, 0xe5, 0x8b, 0xe5, 0xbb, 0xb3, + 0x5e, 0x3e, 0x15, 0x26, 0xdd, 0xc3, 0x53, 0x3b, 0x24, 0x7b, 0xdb, 0x38, 0x2b, 0x05, 0xad, 0x03, + 0x84, 0x94, 0x33, 0x6f, 0x18, 0xb9, 0xcc, 0x37, 0xab, 0x52, 0x8d, 0x66, 0x37, 0xe1, 0xf9, 0x8a, + 0xda, 0x84, 0x86, 0x38, 0x43, 0x83, 0xde, 0x87, 0x05, 0xed, 0xc3, 0x96, 0xeb, 0x13, 0xfa, 0xdc, + 0x34, 0x56, 0x72, 0xab, 0x0b, 0xb8, 0xae, 0x81, 0x7b, 0x02, 0x86, 0x3e, 0x01, 0x90, 0x11, 0x67, + 0x4b, 0xb1, 0x20, 0xc5, 0x2e, 0xa9, 0xdd, 0xf5, 0x98, 0xe7, 0x51, 0x47, 0xc0, 0xc5, 0x16, 0x71, + 0x86, 0x0e, 0xf5, 0x60, 0x31, 0x0d, 0x31, 0xc5, 0x5a, 0x93, 0xac, 0x6f, 0x29, 0xd6, 0x87, 0xe3, + 0x48, 0xc9, 0x3f, 0xc9, 0xd1, 0xf9, 0xa7, 0x22, 0x2c, 0x26, 0xbe, 0xc7, 0x03, 0xe6, 0x73, 0x8a, + 0x56, 0xa1, 0xcc, 0x23, 0x3b, 0x1a, 0x72, 0xe9, 0x7b, 0x8d, 0x8d, 0x66, 0x37, 0x36, 0x4f, 0xf7, + 0x50, 0xc2, 0xb1, 0xc6, 0x0b, 0xca, 0x53, 0xb9, 0x67, 0xe9, 0x5b, 0xb3, 0x6c, 0xa1, 0xf1, 0xe8, + 0x43, 0x68, 0x44, 0x34, 0x1c, 0xb8, 0xbe, 0xed, 0x59, 0x34, 0x0c, 0x59, 0xa8, 0x7d, 0x6e, 0x21, + 0x86, 0xee, 0x08, 0x20, 0xfa, 0x7d, 0xa8, 0x87, 0xd4, 0x26, 0x56, 0x74, 0x1a, 0xb2, 0x61, 0xff, + 0xf4, 0x8a, 0xfe, 0x57, 0x13, 0x32, 0x8e, 0x94, 0x08, 0xe1, 0x84, 0x17, 0xa1, 0x1b, 0x51, 0x4b, + 0x68, 0x72, 0x55, 0x27, 0x94, 0x12, 0xc4, 0x96, 0xd0, 0x1e, 0x94, 0xec, 0x90, 0xfa, 0xb6, 0x74, + 0xc2, 0xfa, 0xd6, 0xc7, 0x2f, 0x5f, 0x2c, 0xaf, 0xf5, 0xdd, 0xe8, 0x74, 0x78, 0xdc, 0x75, 0xd8, + 0x60, 0x8d, 0xf2, 0x68, 0x68, 0x87, 0x23, 0x95, 0x26, 0xa7, 0x12, 0x67, 0x77, 0x53, 0xb0, 0x62, + 0x25, 0x01, 0x7d, 0x08, 0x45, 0xc2, 0x1c, 0x6e, 0x56, 0x56, 0x0a, 0xab, 0xb5, 0x8d, 0x9a, 0x3a, + 0xb5, 0x43, 0xcf, 0x75, 0xa8, 0x76, 0x65, 0x89, 0x46, 0x5f, 0x41, 0x45, 0x45, 0x10, 0x37, 0xab, + 0x2b, 0x85, 0x2b, 0x68, 0x1f, 0xb3, 0x0b, 0x3f, 0x1b, 0x0e, 0x5d, 0x62, 0x05, 0x76, 0x18, 0x71, + 0xd3, 0x90, 0xaf, 0xd5, 0x51, 0xf4, 0xe4, 0xc9, 0xde, 0xf6, 0x81, 0x00, 0xeb, 0x57, 0x1b, 0x82, + 0x50, 0x02, 0x84, 0xd3, 0x07, 0xb6, 0x73, 0x46, 0x89, 0x75, 0x46, 0x47, 0x26, 0xcc, 0x53, 0xd6, + 0x50, 0x44, 0xbf, 0xa6, 0xa3, 0x0e, 0x81, 0x16, 0x66, 0xce, 0x19, 0xdf, 0xde, 0xda, 0xa6, 0xdc, + 0x09, 0xdd, 0x40, 0xc4, 0xce, 0x1d, 0x40, 0xa1, 0x00, 0x92, 0x63, 0x8b, 0xfa, 0xe7, 0xd6, 0x80, + 0x0e, 0x82, 0x28, 0x94, 0x1e, 0x56, 0xc6, 0x4d, 0x8d, 0xd9, 0xf1, 0xcf, 0x1f, 0x4a, 0x38, 0x7a, + 0x0f, 0xea, 0x31, 0xb5, 0xcc, 0xc2, 0x2a, 0x43, 0xd7, 0x34, 0x4c, 0x64, 0xe2, 0xce, 0x9f, 0xe7, + 0xc1, 0xe8, 0xc5, 0x19, 0x17, 0xbd, 0x09, 0x15, 0x37, 0xb0, 0x6c, 0x42, 0x94, 0x4c, 0x03, 0x97, + 0xdd, 0x60, 0x93, 0x90, 0x10, 0xfd, 0x1c, 0x16, 0x74, 0x9a, 0xb6, 0x02, 0x26, 0xf6, 0x9d, 0x97, + 0x3b, 0x68, 0xa9, 0x1d, 0xe8, 0x4c, 0x7d, 0xc0, 0xc2, 0x08, 0xd7, 0xfd, 0x74, 0xc1, 0xd1, 0x21, + 0xb4, 0x06, 0x76, 0x10, 0x50, 0x62, 0x9d, 0x32, 0x1e, 0x69, 0xde, 0x82, 0xe4, 0xfd, 0x28, 0xc9, + 0xe3, 0xc9, 0xfb, 0xbb, 0x0f, 0x25, 0xed, 0x57, 0x8c, 0x47, 0x92, 0x7d, 0xc7, 0x8f, 0xc2, 0x91, + 0x08, 0xb7, 0x31, 0x28, 0x7a, 0x17, 0x60, 0xc8, 0xed, 0x3e, 0xb5, 0x42, 0x3b, 0xa2, 0xd2, 0xbb, + 0xf3, 0xd8, 0x90, 0x10, 0x6c, 0x47, 0xb4, 0xbd, 0x05, 0x4b, 0xb3, 0xe4, 0xa0, 0x26, 0x14, 0x84, + 0xed, 0x73, 0x32, 0x77, 0x88, 0x47, 0xb4, 0x04, 0xa5, 0x73, 0xdb, 0x1b, 0xc6, 0xa5, 0x4b, 0x2d, + 0x3e, 0xcf, 0x7f, 0x9a, 0xeb, 0xfc, 0x4d, 0x1e, 0x5a, 0x3d, 0x3b, 0x88, 0x86, 0x61, 0x5c, 0x4d, + 0x76, 0x9e, 0x8b, 0xdc, 0x29, 0x6a, 0x9f, 0xe5, 0xd1, 0x73, 0xea, 0xe9, 0xb0, 0x6e, 0x74, 0x45, + 0xf5, 0xdd, 0x67, 0xfd, 0xee, 0xbe, 0x80, 0xe2, 0xaa, 0xc7, 0xfa, 0xf2, 0x09, 0xed, 0xa5, 0x47, + 0x45, 0x92, 0x03, 0xd4, 0x21, 0xde, 0x4e, 0xf6, 0x3e, 0x75, 0xc4, 0xb8, 0xa5, 0xb9, 0x32, 0xa7, + 0xbe, 0x07, 0x75, 0x1e, 0xd9, 0x61, 0x64, 0x39, 0x6c, 0x30, 0x70, 0x23, 0x19, 0xf5, 0xb5, 0x8d, + 0xdf, 0x4b, 0x0d, 0x38, 0xa9, 0xa9, 0x48, 0x31, 0x61, 0xd4, 0x93, 0xd4, 0xb8, 0xc6, 0xd3, 0x45, + 0x1b, 0x43, 0x2d, 0x83, 0x43, 0x3d, 0x40, 0x5a, 0x88, 0xe5, 0x9c, 0x52, 0xe7, 0x2c, 0x60, 0xae, + 0x1f, 0xc9, 0xad, 0x89, 0xe4, 0x99, 0x64, 0xac, 0x5e, 0x82, 0xc3, 0x2d, 0x4d, 0x9f, 0x82, 0x3a, + 0xff, 0x57, 0x04, 0x94, 0xa8, 0xa0, 0xd2, 0x9f, 0xb0, 0xd6, 0x3a, 0x18, 0x49, 0x2d, 0xd7, 0x22, + 0xd1, 0xf4, 0x99, 0xe3, 0x94, 0x08, 0x7d, 0x0e, 0x65, 0x16, 0x50, 0x9f, 0x12, 0x6d, 0xa6, 0xce, + 0xf4, 0x0e, 0x13, 0xf1, 0xdd, 0xc7, 0x92, 0x12, 0x6b, 0x0e, 0x74, 0x1f, 0xaa, 0x8e, 0x22, 0x22, + 0xda, 0x3e, 0x1f, 0x5c, 0xc6, 0xad, 0x41, 0x04, 0x27, 0x5c, 0x68, 0x17, 0x20, 0x63, 0x83, 0xe2, + 0x3c, 0x1b, 0x67, 0x64, 0xa4, 0x56, 0xc9, 0x70, 0xb6, 0x1f, 0x42, 0x59, 0xe9, 0xf6, 0x83, 0x58, + 0xb7, 0xfd, 0x14, 0xaa, 0xb1, 0xb2, 0xc2, 0xf3, 0xcf, 0xe8, 0xc8, 0x52, 0x49, 0x42, 0x0a, 0xaa, + 0x63, 0xe3, 0x8c, 0x8e, 0x0e, 0x24, 0x40, 0xb4, 0x55, 0x22, 0x2b, 0xb9, 0xa2, 0x28, 0xf1, 0x98, + 0x2a, 0x2f, 0xa9, 0x9a, 0x29, 0x42, 0x11, 0xb7, 0x2f, 0x00, 0xd2, 0xb7, 0xa0, 0x15, 0x28, 0x89, + 0x72, 0xc4, 0xb5, 0x76, 0x20, 0xdd, 0x5a, 0x14, 0x2a, 0x8e, 0x15, 0x02, 0x7d, 0x09, 0xb5, 0x80, + 0x79, 0x9e, 0x15, 0x52, 0x3e, 0xf4, 0x22, 0x29, 0xb6, 0x71, 0xb9, 0x7d, 0x0e, 0x98, 0xe7, 0x61, + 0x49, 0x8d, 0x21, 0x48, 0x9e, 0x3b, 0x8f, 0x00, 0x52, 0x0c, 0xaa, 0x41, 0x65, 0xef, 0xd1, 0xd3, + 0xcd, 0xfd, 0xbd, 0xed, 0xe6, 0x1b, 0xc8, 0x80, 0x12, 0xde, 0xd9, 0xdc, 0xfe, 0xc3, 0x66, 0x0e, + 0x2d, 0x80, 0xf1, 0xe8, 0xf1, 0x91, 0xa5, 0x96, 0x79, 0x54, 0x87, 0x6a, 0xef, 0xf1, 0xe3, 0x7d, + 0xeb, 0xf1, 0xee, 0x6e, 0xb3, 0x20, 0x98, 0xf0, 0xce, 0xe1, 0xd1, 0x26, 0x3e, 0x6a, 0x16, 0x3b, + 0xff, 0x99, 0x83, 0xe6, 0xb6, 0xec, 0xb5, 0x7f, 0x04, 0xa1, 0xba, 0x01, 0x45, 0xe1, 0x90, 0xda, + 0x05, 0x6f, 0x26, 0xcc, 0x93, 0x0a, 0x4a, 0xf7, 0xc5, 0x92, 0xb6, 0x7d, 0x07, 0x8a, 0x62, 0x85, + 0x3e, 0x80, 0x06, 0xff, 0xda, 0x13, 0x55, 0xf6, 0xfc, 0x84, 0x5b, 0xc3, 0xd0, 0xd5, 0x49, 0xb8, + 0xae, 0xa0, 0x4f, 0x4f, 0xf8, 0x93, 0xd0, 0xed, 0xfc, 0x57, 0x01, 0x5a, 0xb1, 0xb4, 0xef, 0x13, + 0x6c, 0x9f, 0x4d, 0x04, 0xdb, 0x7b, 0x53, 0xba, 0xce, 0x8d, 0xb5, 0x2d, 0x30, 0x82, 0xe1, 0xb1, + 0xe7, 0xf2, 0xd3, 0x19, 0xc1, 0x36, 0xcd, 0x7d, 0x10, 0xd3, 0xe2, 0x94, 0x0d, 0xfd, 0x12, 0x2a, + 0x27, 0xde, 0x50, 0x4a, 0x28, 0x4e, 0x04, 0xfb, 0xb4, 0x84, 0x5d, 0x45, 0x89, 0x63, 0x96, 0x1f, + 0x3a, 0xc6, 0x22, 0x30, 0x12, 0x25, 0xc5, 0xa5, 0x66, 0x60, 0x3f, 0xb7, 0x1c, 0x8f, 0x39, 0x67, + 0xba, 0xb4, 0x56, 0x07, 0xf6, 0xf3, 0x9e, 0x58, 0x4f, 0x44, 0x60, 0xfe, 0x95, 0x22, 0xb0, 0x30, + 0x27, 0x02, 0x6f, 0x43, 0x45, 0x6f, 0xec, 0xbb, 0xc3, 0xaf, 0xf3, 0x67, 0x39, 0xb8, 0x9e, 0x36, + 0xa3, 0x3f, 0x02, 0x57, 0xef, 0xfc, 0x36, 0x07, 0x37, 0xc6, 0x34, 0xfa, 0x3e, 0xde, 0xb8, 0x99, + 0xba, 0x83, 0x52, 0x26, 0x6d, 0x0f, 0x66, 0xbf, 0x63, 0xda, 0x27, 0x5e, 0xcb, 0x9c, 0xbf, 0x2d, + 0x42, 0xa3, 0xc7, 0x06, 0xc7, 0xae, 0x9f, 0x5c, 0x17, 0xd7, 0x75, 0xe8, 0x2a, 0x9e, 0x77, 0x32, + 0xfa, 0x66, 0xc9, 0x32, 0x81, 0x8b, 0xee, 0x42, 0xc1, 0x26, 0xb1, 0xc2, 0x6f, 0xcf, 0x63, 0xd8, + 0x24, 0x04, 0x0b, 0xba, 0xf6, 0xbf, 0xe4, 0x75, 0xa0, 0xdf, 0x87, 0xea, 0xb1, 0xeb, 0x13, 0xd7, + 0xef, 0x0b, 0x0d, 0x0b, 0xe3, 0xb5, 0x6a, 0xfa, 0x6d, 0xdd, 0x2d, 0x45, 0x8c, 0x13, 0xae, 0xf6, + 0x9f, 0xe6, 0xa1, 0xa2, 0xa1, 0x08, 0x41, 0xf1, 0x64, 0xe8, 0xa9, 0xa3, 0xaf, 0x62, 0xf9, 0x1c, + 0xf7, 0x3a, 0xa2, 0x4b, 0x33, 0x54, 0xaf, 0xf3, 0x29, 0xd4, 0x82, 0x90, 0x3d, 0x53, 0xd7, 0xa0, + 0xb8, 0x07, 0x6b, 0xaa, 0xfe, 0xed, 0x20, 0x41, 0xe8, 0x36, 0x34, 0x4b, 0x8a, 0xee, 0x41, 0x8d, + 0x3b, 0xa7, 0x74, 0x60, 0x5b, 0xcf, 0x38, 0xf3, 0x65, 0xb4, 0xd6, 0xb7, 0xde, 0x79, 0xf9, 0x62, + 0xd9, 0xa4, 0xbe, 0xc3, 0x84, 0x0a, 0x6b, 0x02, 0xd1, 0xc5, 0xf6, 0xc5, 0x43, 0xca, 0x65, 0x1b, + 0x06, 0x8a, 0xe1, 0x01, 0x67, 0x3e, 0xea, 0x02, 0x70, 0x1a, 0x5a, 0x01, 0xf3, 0x5c, 0x67, 0x24, + 0xaf, 0x0e, 0x49, 0xbf, 0x7c, 0x48, 0xc3, 0x03, 0x09, 0xc6, 0x06, 0x8f, 0x1f, 0xe5, 0xd8, 0x40, + 0xf6, 0xd7, 0x51, 0x28, 0xaf, 0x07, 0x06, 0xae, 0xc8, 0x36, 0x3a, 0x0a, 0xc5, 0x2d, 0x5c, 0xb6, + 0x68, 0xaa, 0xdb, 0x37, 0xb0, 0x5e, 0xb5, 0x7d, 0x28, 0x6c, 0x12, 0x82, 0x4c, 0xa8, 0x68, 0x03, + 0xe9, 0x26, 0x2f, 0x5e, 0xa2, 0x5f, 0x40, 0x95, 0x30, 0x47, 0xe9, 0x9f, 0x7f, 0x05, 0xfd, 0x2b, + 0x84, 0x39, 0x52, 0xf9, 0x25, 0x28, 0x9d, 0x84, 0xcc, 0x57, 0x2d, 0x57, 0x15, 0xab, 0x45, 0xe7, + 0x5f, 0x73, 0xb0, 0x98, 0x9c, 0x93, 0xbe, 0xef, 0xcd, 0x7f, 0xb9, 0x09, 0x15, 0x42, 0x3d, 0x1a, + 0x69, 0xd7, 0xae, 0xe2, 0x78, 0x39, 0xa6, 0x56, 0xe1, 0x4a, 0x6a, 0x15, 0x33, 0x6a, 0x4d, 0xe4, + 0xa6, 0xd2, 0x64, 0x6e, 0x7a, 0x1f, 0x16, 0x94, 0xbd, 0x62, 0x0a, 0x79, 0xf9, 0xc2, 0x75, 0x05, + 0x54, 0x44, 0x9d, 0x37, 0xe1, 0x7a, 0x8f, 0xf9, 0x3e, 0x75, 0x22, 0x16, 0x1e, 0x84, 0xec, 0xf9, + 0x48, 0x3b, 0x62, 0xe7, 0x2f, 0x72, 0x70, 0x63, 0x12, 0xa3, 0xb7, 0xfe, 0x00, 0x2a, 0xe2, 0xca, + 0x40, 0x39, 0xd7, 0x73, 0x96, 0xf5, 0x97, 0x2f, 0x96, 0xef, 0xbc, 0xca, 0xdd, 0x6a, 0xc7, 0x27, + 0x2a, 0x27, 0xc7, 0x02, 0xc4, 0xe9, 0x07, 0x42, 0xb8, 0xe5, 0x12, 0xdd, 0x95, 0x57, 0xe4, 0x7a, + 0x8f, 0xa0, 0x36, 0x14, 0x3c, 0xd6, 0xd7, 0xf5, 0xa6, 0x1a, 0x67, 0x38, 0x2c, 0x80, 0x9d, 0xbf, + 0x2b, 0x40, 0xf1, 0x01, 0x73, 0x7d, 0x74, 0x0b, 0x5a, 0x34, 0x72, 0x88, 0x35, 0x60, 0xc4, 0x0a, + 0xe9, 0xb9, 0xcb, 0xc5, 0x8d, 0x5e, 0x68, 0x55, 0xc0, 0x8b, 0x02, 0xf1, 0x90, 0x11, 0xac, 0xc1, + 0xe8, 0x36, 0x94, 0xf9, 0xa9, 0x1d, 0x92, 0xf8, 0x36, 0x73, 0x2d, 0x09, 0x42, 0x21, 0x4a, 0x0d, + 0x2f, 0xb0, 0x26, 0x41, 0xcb, 0x50, 0x93, 0x4f, 0x7a, 0x02, 0x51, 0x90, 0x67, 0x0c, 0x12, 0xa4, + 0xe6, 0x0f, 0xb7, 0xa1, 0x15, 0x0f, 0x29, 0x88, 0x1b, 0x4a, 0x33, 0x8d, 0xe2, 0x99, 0x96, 0x46, + 0x6c, 0xc7, 0x70, 0xf4, 0x13, 0x88, 0x61, 0x16, 0xd5, 0x36, 0x90, 0x07, 0x66, 0xe0, 0x45, 0x0d, + 0x8f, 0x4d, 0x83, 0x3e, 0x82, 0x45, 0x4f, 0x5e, 0xff, 0x53, 0x4a, 0x15, 0x16, 0x0d, 0x05, 0x8e, + 0x09, 0xdb, 0x7f, 0x9b, 0x83, 0x92, 0xd4, 0x19, 0x35, 0x20, 0xef, 0x12, 0xdd, 0x3c, 0xe4, 0x5d, + 0x82, 0xba, 0x50, 0xf5, 0xec, 0x63, 0xea, 0x09, 0xe7, 0xcc, 0xeb, 0x6c, 0x2c, 0x33, 0xa2, 0xa0, + 0xde, 0xd7, 0x18, 0x9c, 0xd0, 0xa0, 0x0d, 0xa8, 0x84, 0xd4, 0x16, 0x9a, 0x6a, 0x6b, 0x9b, 0xe9, + 0x48, 0xe2, 0x20, 0x64, 0x0e, 0xe5, 0xfc, 0x30, 0xa0, 0x4e, 0x77, 0x6f, 0x1b, 0xc7, 0x84, 0x68, + 0x1d, 0x96, 0xa4, 0xe1, 0x9d, 0x90, 0xda, 0x11, 0x4d, 0x6d, 0x2f, 0x87, 0x0f, 0x18, 0x09, 0x5c, + 0x4f, 0xa2, 0x62, 0xf3, 0x77, 0x3e, 0x81, 0xb2, 0xb0, 0x33, 0x25, 0xe2, 0xd0, 0x44, 0xc5, 0x95, + 0xfc, 0x93, 0x87, 0x36, 0xb0, 0x9f, 0xef, 0x44, 0x4e, 0x72, 0x68, 0x1d, 0x0b, 0x8a, 0x47, 0x36, + 0x3f, 0x13, 0x59, 0x8f, 0x07, 0xd4, 0xd1, 0x4d, 0xb0, 0x7c, 0x16, 0x91, 0x16, 0x08, 0x7e, 0x7a, + 0x11, 0x47, 0x9a, 0x5e, 0x0a, 0x7b, 0x8b, 0x37, 0x44, 0xa1, 0xed, 0x73, 0x3b, 0x49, 0x81, 0xe2, + 0x08, 0xc5, 0x0b, 0x8e, 0x32, 0xe0, 0xce, 0x5f, 0x95, 0xa0, 0x82, 0xa9, 0xc3, 0xce, 0x65, 0x55, + 0xaa, 0xd9, 0xce, 0x99, 0xe5, 0xfa, 0x11, 0xf5, 0xa3, 0x38, 0x57, 0xaf, 0xa4, 0x65, 0x52, 0x91, + 0x75, 0x37, 0x9d, 0xb3, 0x3d, 0x45, 0xa2, 0x6e, 0xac, 0x60, 0x27, 0x00, 0xb4, 0x01, 0xd7, 0xd5, + 0xad, 0x2d, 0xa2, 0x44, 0xf4, 0x14, 0x9c, 0xea, 0xce, 0x22, 0x2f, 0x3b, 0x8b, 0x6b, 0x09, 0xb2, + 0x27, 0x70, 0xaa, 0xc9, 0xb8, 0x0f, 0x28, 0xe5, 0x91, 0xb1, 0xed, 0xd2, 0xf8, 0x28, 0x5a, 0xdd, + 0x78, 0x9c, 0xbb, 0xab, 0x11, 0xb8, 0x95, 0x10, 0xc7, 0x20, 0x74, 0x07, 0x96, 0x9c, 0x38, 0x58, + 0x2d, 0x51, 0xf1, 0x68, 0x26, 0x79, 0xe3, 0x46, 0x82, 0x13, 0x35, 0x91, 0xa2, 0x3b, 0x80, 0x4e, + 0xc5, 0x1e, 0xc7, 0x15, 0x2c, 0xa9, 0xa9, 0x82, 0xc2, 0x64, 0xb4, 0xfb, 0x1c, 0x16, 0x35, 0x75, + 0xa2, 0x5a, 0x79, 0x9e, 0x6a, 0x0d, 0x45, 0x99, 0xe8, 0xf5, 0x1e, 0xd4, 0x3d, 0x9b, 0x47, 0x96, + 0x1d, 0x04, 0x9e, 0x4b, 0x89, 0x9c, 0x28, 0xd6, 0x71, 0x4d, 0xc0, 0x36, 0x15, 0x08, 0x6d, 0x42, + 0xcb, 0xa3, 0x7d, 0xdb, 0x19, 0x65, 0xfb, 0xb9, 0xea, 0x25, 0xfd, 0x5c, 0x53, 0x91, 0x67, 0x2e, + 0x33, 0x9f, 0x82, 0x68, 0xd8, 0xac, 0x33, 0x3a, 0x8a, 0x07, 0x34, 0xef, 0x4e, 0x9d, 0xd9, 0x43, + 0xfb, 0xf9, 0xaf, 0xe9, 0x48, 0x1f, 0x58, 0x65, 0xa0, 0x56, 0xe8, 0x16, 0x5c, 0x8b, 0x42, 0xb7, + 0xdf, 0x17, 0x05, 0xcb, 0x0e, 0xed, 0x01, 0x57, 0x66, 0x03, 0xa9, 0xe6, 0x82, 0x46, 0x1d, 0x48, + 0x4c, 0xfb, 0x1e, 0x2c, 0x4e, 0x1c, 0x7c, 0x76, 0xc4, 0x60, 0xcc, 0x18, 0x31, 0xd4, 0x33, 0x23, + 0x86, 0xf6, 0xe7, 0x50, 0xcf, 0xea, 0xf0, 0x5d, 0xe3, 0x89, 0x2c, 0x6f, 0xe7, 0x9f, 0xcb, 0x50, + 0x39, 0xa0, 0x21, 0x77, 0x79, 0x84, 0xae, 0x43, 0x99, 0xd3, 0xaf, 0x2d, 0x9f, 0x49, 0xd6, 0x22, + 0x2e, 0x71, 0xfa, 0xf5, 0x23, 0x26, 0xce, 0x54, 0x95, 0x19, 0x2b, 0xeb, 0xc1, 0x2a, 0x2c, 0x9a, + 0x0a, 0x93, 0x6a, 0x3f, 0xe9, 0xe8, 0x85, 0x09, 0x47, 0xd7, 0xef, 0xba, 0x9a, 0xa3, 0x17, 0x5f, + 0xd7, 0xd1, 0x4b, 0xaf, 0xe1, 0xe8, 0xeb, 0x70, 0x23, 0x75, 0xf4, 0xc0, 0x8e, 0x9c, 0x53, 0xaa, + 0xcf, 0x4c, 0x55, 0xb7, 0x66, 0x82, 0x3d, 0x50, 0xc8, 0x39, 0xce, 0x5e, 0x99, 0xe3, 0xec, 0x9f, + 0xc0, 0x0d, 0x6d, 0xc6, 0x49, 0x9f, 0xaf, 0x4a, 0x53, 0x2e, 0x29, 0xec, 0x57, 0xe3, 0x6e, 0x3e, + 0x23, 0x44, 0x8c, 0xab, 0x86, 0x08, 0xbc, 0x62, 0x88, 0xd4, 0xae, 0x1c, 0x22, 0xf5, 0x89, 0x10, + 0x89, 0x4f, 0x7b, 0x76, 0x88, 0x6c, 0xc0, 0x75, 0x6d, 0x91, 0xf1, 0x48, 0x31, 0x17, 0xa4, 0x41, + 0xae, 0x29, 0xe4, 0x51, 0x36, 0x54, 0xe6, 0x85, 0x55, 0xe3, 0x47, 0x16, 0x56, 0x1d, 0x30, 0xf4, + 0xde, 0x29, 0x99, 0x13, 0x57, 0x9d, 0xbf, 0xce, 0x41, 0x49, 0x9c, 0xc3, 0x68, 0x5e, 0x05, 0x3a, + 0x17, 0x12, 0x74, 0x9f, 0x69, 0xe0, 0x78, 0x29, 0x6e, 0x95, 0xf2, 0x58, 0x25, 0x8b, 0x4a, 0xc3, + 0x55, 0x01, 0x10, 0x85, 0x34, 0x39, 0xf3, 0x98, 0x57, 0xb5, 0x02, 0xf2, 0xcc, 0x9f, 0x6a, 0xfe, + 0xf5, 0x39, 0x19, 0x5d, 0xb9, 0x39, 0x1a, 0xcf, 0xe8, 0xa2, 0x49, 0xec, 0x3c, 0x83, 0x4a, 0xec, + 0x30, 0x77, 0x01, 0xa9, 0xf2, 0x96, 0x5c, 0xfa, 0xe2, 0x0a, 0x6b, 0xe0, 0x96, 0xc2, 0x6c, 0xa7, + 0x88, 0x4b, 0x82, 0x2a, 0x3f, 0x3b, 0xa8, 0x3a, 0xdf, 0xe6, 0xf4, 0xd5, 0xe6, 0xf5, 0x8c, 0xf2, + 0x61, 0xfc, 0x31, 0xaa, 0x30, 0xf3, 0x63, 0x54, 0xfc, 0x19, 0xea, 0xfd, 0x4b, 0xab, 0x99, 0xbc, + 0xd1, 0x51, 0xf4, 0xb3, 0x8c, 0x47, 0x97, 0xa4, 0x47, 0xa7, 0xf7, 0x59, 0x79, 0x8b, 0x9a, 0xe9, + 0xce, 0xdf, 0xcb, 0x5f, 0x00, 0xaa, 0x32, 0x55, 0x3c, 0x62, 0x17, 0x9d, 0x32, 0x14, 0x0f, 0x23, + 0x16, 0x74, 0x0c, 0xa8, 0x88, 0xdf, 0x80, 0x92, 0xce, 0x1f, 0x40, 0xed, 0x90, 0x72, 0xb1, 0xd1, + 0x7d, 0xc6, 0x82, 0x39, 0x57, 0xef, 0xdc, 0x55, 0xae, 0xde, 0xff, 0x76, 0x03, 0x6a, 0x99, 0x6b, + 0x31, 0xba, 0x9b, 0xb1, 0x7a, 0x6d, 0xe3, 0xad, 0x6e, 0xf6, 0xab, 0x71, 0x7c, 0x93, 0x94, 0x06, + 0x55, 0x07, 0xf2, 0x05, 0x2c, 0x88, 0x5f, 0x2b, 0xd4, 0x1d, 0x7c, 0x72, 0xff, 0x1f, 0xe7, 0x53, + 0x48, 0xc5, 0x58, 0x17, 0x0c, 0x49, 0xc7, 0xff, 0x19, 0x54, 0xcf, 0x6d, 0xcf, 0x25, 0x76, 0x14, + 0x1f, 0xdd, 0xbb, 0x33, 0xdf, 0xf9, 0x54, 0x13, 0xe1, 0x84, 0x1c, 0xdd, 0x03, 0x23, 0x7e, 0x8e, + 0x27, 0x3f, 0xcb, 0xb3, 0xdf, 0x1b, 0x33, 0x13, 0x9c, 0x72, 0x8c, 0x0f, 0x3b, 0x4a, 0xdf, 0x31, + 0xec, 0xf8, 0x05, 0xd4, 0xb9, 0x3a, 0x00, 0xcb, 0x63, 0x2c, 0x30, 0x97, 0x74, 0x8a, 0x8c, 0x6d, + 0x9d, 0x39, 0x1d, 0x5c, 0xe3, 0x99, 0xa3, 0x7a, 0x0f, 0x8a, 0xcf, 0x98, 0xeb, 0x9b, 0xd7, 0x25, + 0xc3, 0xc2, 0xd8, 0xbd, 0x00, 0x4b, 0x14, 0xfa, 0x08, 0xca, 0xcf, 0x64, 0xf7, 0x6a, 0xde, 0xd0, + 0xbe, 0x9b, 0x25, 0xa2, 0x04, 0x6b, 0xb4, 0x90, 0x15, 0xd9, 0xfc, 0xcc, 0x7c, 0x73, 0x42, 0x96, + 0xe8, 0x62, 0xb1, 0x44, 0xa1, 0x5b, 0xa2, 0xdf, 0x96, 0x6d, 0x89, 0x69, 0xea, 0x4f, 0x80, 0x13, + 0xed, 0x0a, 0x8e, 0x09, 0xd0, 0x07, 0x50, 0x12, 0x95, 0x61, 0x64, 0xbe, 0x25, 0x29, 0xd3, 0xaf, + 0xe1, 0x32, 0x29, 0x61, 0x85, 0x14, 0x12, 0xe3, 0xfa, 0xd1, 0x9e, 0x90, 0xa8, 0x73, 0x02, 0x8e, + 0x09, 0x84, 0x82, 0x72, 0xee, 0xf1, 0xf6, 0x84, 0x82, 0x99, 0x41, 0xc7, 0xc7, 0xc9, 0xac, 0xf0, + 0x9d, 0x89, 0x59, 0x47, 0xc6, 0x0b, 0x27, 0xa7, 0x84, 0x77, 0xa1, 0xe8, 0x31, 0x9b, 0x98, 0xab, + 0xda, 0x29, 0x67, 0xb1, 0xec, 0x33, 0x9b, 0x60, 0x49, 0x26, 0xde, 0x21, 0x7e, 0x29, 0x31, 0x7f, + 0x72, 0xc9, 0x3b, 0xf6, 0x25, 0x09, 0xd6, 0xa4, 0x68, 0x1d, 0x4a, 0x72, 0xfc, 0x63, 0xde, 0x9a, + 0x08, 0xa3, 0x2c, 0x8f, 0x9c, 0x0a, 0x61, 0x45, 0x88, 0x7e, 0x9e, 0x0e, 0x9a, 0x6e, 0x4f, 0x0c, + 0x7a, 0xa6, 0x78, 0x32, 0xd3, 0x25, 0xf1, 0x26, 0x1e, 0xb1, 0x90, 0x9a, 0x77, 0x2e, 0x79, 0xd3, + 0xa1, 0xa0, 0xc0, 0x8a, 0x50, 0x6c, 0x48, 0x3e, 0x10, 0xf3, 0xee, 0x25, 0x1b, 0x92, 0x2c, 0xe2, + 0x9a, 0x29, 0x7f, 0x51, 0x6f, 0xe2, 0x53, 0x4f, 0x57, 0xb2, 0xae, 0xcc, 0x61, 0x9d, 0xfd, 0x91, + 0x07, 0xed, 0x41, 0x43, 0x2e, 0x45, 0x8f, 0xa3, 0xc4, 0xac, 0x4d, 0x8c, 0x58, 0xa7, 0xc4, 0x50, + 0xa2, 0x05, 0x2d, 0xf0, 0xec, 0x12, 0x6d, 0xc9, 0xc6, 0xd0, 0x67, 0x17, 0x1e, 0x25, 0x7d, 0x6a, + 0xae, 0x5f, 0xa2, 0xce, 0x66, 0x4a, 0x87, 0xb3, 0x4c, 0x68, 0x07, 0xea, 0x99, 0x25, 0x31, 0x7f, + 0x3a, 0x31, 0x6f, 0x9e, 0x23, 0x84, 0xe0, 0x31, 0x36, 0xe1, 0xd3, 0x81, 0xaa, 0xce, 0xe6, 0xc6, + 0x84, 0x4f, 0xeb, 0xaa, 0x8d, 0x63, 0x02, 0xb4, 0x0e, 0x46, 0x10, 0x57, 0x72, 0xf3, 0xe3, 0x89, + 0x01, 0x64, 0x52, 0xe3, 0x71, 0x4a, 0x84, 0xba, 0x60, 0xa8, 0x7e, 0xd0, 0x67, 0x17, 0xe6, 0x27, + 0xba, 0x59, 0x4b, 0x86, 0x72, 0x3a, 0xcb, 0xe3, 0xaa, 0xa3, 0x9f, 0x44, 0xd4, 0xf0, 0x88, 0x05, + 0xe6, 0x2f, 0x27, 0xa2, 0x46, 0x24, 0x7f, 0x2c, 0x51, 0x42, 0x61, 0xae, 0x4a, 0x81, 0x79, 0x6f, + 0x42, 0x61, 0x5d, 0x22, 0x70, 0x4c, 0xd0, 0xfe, 0xe3, 0x5c, 0x32, 0xd1, 0x7e, 0xfd, 0xe1, 0xe9, + 0x97, 0xd9, 0xfa, 0x98, 0x69, 0x09, 0xf3, 0x97, 0xb4, 0x84, 0xd7, 0x12, 0x8e, 0xcc, 0x1c, 0xfc, + 0x67, 0x50, 0x14, 0x01, 0x86, 0xee, 0x42, 0x35, 0x69, 0x5c, 0x73, 0xf3, 0x1a, 0xd7, 0x84, 0xa4, + 0xfd, 0x6d, 0x1e, 0xca, 0x2a, 0x30, 0xd1, 0x17, 0x53, 0xa3, 0xcd, 0xf7, 0x2f, 0x89, 0xe3, 0xe9, + 0xc9, 0xa6, 0xea, 0x73, 0xe4, 0x68, 0x2d, 0xb4, 0xd4, 0x57, 0xde, 0xe3, 0x51, 0x44, 0xd5, 0xcd, + 0xa5, 0x28, 0xfa, 0x1c, 0x85, 0x7b, 0x22, 0x50, 0x5b, 0x02, 0xd3, 0xfe, 0xef, 0x5c, 0x3a, 0x0b, + 0x5d, 0x82, 0x92, 0x9a, 0xcf, 0xa8, 0xfa, 0xad, 0x16, 0x68, 0x15, 0x9a, 0x03, 0xd7, 0xb7, 0x38, + 0x1b, 0x86, 0xce, 0xf8, 0xf5, 0xbb, 0x31, 0x70, 0xfd, 0x43, 0x09, 0x56, 0xed, 0xfe, 0xaa, 0x9a, + 0x13, 0x8c, 0x51, 0x16, 0x34, 0xa5, 0xfd, 0x3c, 0x4b, 0x79, 0x07, 0x90, 0xa2, 0x22, 0x16, 0x61, + 0x0e, 0xb7, 0x22, 0x16, 0xd9, 0x9e, 0x2c, 0x68, 0x45, 0xdc, 0xd4, 0x98, 0x6d, 0xe6, 0xf0, 0x23, + 0x01, 0x47, 0x5d, 0xb8, 0x16, 0x53, 0xcb, 0xed, 0x68, 0xf2, 0x92, 0x24, 0x6f, 0x69, 0x94, 0xdc, + 0x8e, 0xa2, 0xef, 0xc0, 0x82, 0x6e, 0x66, 0x2c, 0x42, 0xbd, 0x48, 0xff, 0xa1, 0x04, 0xae, 0xa9, + 0xae, 0x65, 0x5b, 0x80, 0xda, 0x9f, 0x41, 0x49, 0x66, 0xa9, 0x4b, 0xda, 0xb5, 0xdc, 0xec, 0x76, + 0xad, 0xfd, 0x3f, 0xb9, 0x74, 0x56, 0x7e, 0xd9, 0x30, 0x7a, 0x46, 0x46, 0x9c, 0x79, 0x64, 0xaf, + 0xd9, 0x2e, 0xb6, 0x47, 0xdf, 0x75, 0x62, 0xb7, 0xa0, 0xa5, 0x32, 0x7c, 0xd6, 0xb8, 0xca, 0x05, + 0x16, 0x15, 0x22, 0xb5, 0xed, 0x1d, 0x40, 0x9a, 0x36, 0x6b, 0xda, 0x82, 0x3a, 0x09, 0x85, 0x49, + 0x2d, 0xdb, 0xae, 0x40, 0x49, 0xa6, 0xdc, 0xf6, 0x3f, 0xe4, 0xa0, 0xac, 0x92, 0xef, 0x2b, 0x3b, + 0xad, 0x22, 0x9f, 0x31, 0x8e, 0x7f, 0x95, 0xfd, 0xa8, 0x04, 0x3f, 0x63, 0x3f, 0x0a, 0x31, 0xb6, + 0x1f, 0x4d, 0x3b, 0x63, 0x3f, 0x0a, 0x93, 0xd9, 0xcf, 0x9f, 0xe4, 0xc6, 0xbf, 0xe8, 0xbf, 0xb6, + 0x33, 0xfc, 0x70, 0xd9, 0x63, 0x13, 0x16, 0xc6, 0x6a, 0xc9, 0x15, 0x1c, 0xf3, 0x0b, 0xa8, 0x65, + 0x2a, 0xc0, 0x15, 0x04, 0xdc, 0x87, 0x7a, 0xb6, 0x84, 0xbc, 0xbe, 0x84, 0x5b, 0xb7, 0xa1, 0x24, + 0xff, 0xb0, 0x10, 0x01, 0x94, 0x0f, 0x9e, 0x6c, 0xed, 0xef, 0xf5, 0x9a, 0x6f, 0xa0, 0x1a, 0x54, + 0x0e, 0xf0, 0xde, 0xd3, 0xcd, 0xa3, 0x9d, 0x66, 0x0e, 0x19, 0x50, 0xda, 0x7f, 0xdc, 0xdb, 0xdc, + 0x6f, 0xe6, 0x37, 0x1e, 0x40, 0x55, 0xff, 0xe1, 0x57, 0x88, 0x7e, 0x05, 0x15, 0xfd, 0x8c, 0xde, + 0x4c, 0x13, 0xfd, 0xd8, 0x9f, 0x24, 0xb6, 0xcd, 0x69, 0x84, 0xea, 0x76, 0xd7, 0x73, 0x1b, 0xfb, + 0x50, 0xd5, 0x1f, 0x15, 0x42, 0x74, 0x1f, 0x2a, 0xfa, 0x39, 0x23, 0x6b, 0xfc, 0xd3, 0x50, 0x46, + 0xd6, 0xc4, 0xb7, 0x88, 0xd5, 0xdc, 0x7a, 0x6e, 0xe3, 0x14, 0x1a, 0xe3, 0xe3, 0x7a, 0xf4, 0x14, + 0x16, 0xe5, 0x43, 0x02, 0xe6, 0xe8, 0x66, 0xb6, 0xae, 0x4c, 0x0f, 0xfd, 0xdb, 0xcb, 0x73, 0xf1, + 0x99, 0x37, 0x7d, 0x09, 0xe5, 0x7d, 0xf5, 0xf7, 0x69, 0xf7, 0xc6, 0x6f, 0x25, 0x4b, 0xb3, 0x82, + 0xa8, 0x3d, 0x13, 0x2a, 0x05, 0xed, 0xc6, 0xf3, 0xeb, 0xef, 0x27, 0x67, 0xeb, 0x57, 0xdf, 0xfc, + 0xfb, 0xcd, 0x37, 0xbe, 0xf9, 0xdd, 0xcd, 0xdc, 0x3f, 0xfe, 0xee, 0x66, 0xee, 0x2f, 0xff, 0xe3, + 0x66, 0xee, 0x8f, 0xee, 0xbc, 0xd2, 0x1f, 0x97, 0x69, 0x79, 0xc7, 0x65, 0x09, 0xfa, 0xf8, 0xff, + 0x03, 0x00, 0x00, 0xff, 0xff, 0x08, 0x04, 0xd6, 0x5d, 0x45, 0x2c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -5404,7 +5404,7 @@ func (m *Task) MarshalToSizedBuffer(dAtA []byte) (int, error) { if m.MaxTransactions != 0 { i = encodeVarintRuntime(dAtA, i, uint64(m.MaxTransactions)) i-- - dAtA[i] = 0x28 + dAtA[i] = 0x18 } if m.Preview { i-- @@ -5414,14 +5414,7 @@ func (m *Task) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0 } i-- - dAtA[i] = 0x20 - } - 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 + dAtA[i] = 0x10 } if len(m.Spec) > 0 { i -= len(m.Spec) @@ -6434,6 +6427,11 @@ func (m *Materialize) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0xa2 } + if m.LogLevel != 0 { + i = encodeVarintRuntime(dAtA, i, uint64(m.LogLevel)) + i-- + dAtA[i] = 0x28 + } if m.Validated != nil { { size, err := m.Validated.MarshalToSizedBuffer(dAtA[:i]) @@ -7826,10 +7824,6 @@ func (m *Task) ProtoSize() (n int) { if l > 0 { n += 1 + l + sovRuntime(uint64(l)) } - l = len(m.OpsStatsJournal) - if l > 0 { - n += 1 + l + sovRuntime(uint64(l)) - } if m.Preview { n += 2 } @@ -8167,6 +8161,9 @@ func (m *Materialize) ProtoSize() (n int) { l = m.Validated.ProtoSize() n += 1 + l + sovRuntime(uint64(l)) } + if m.LogLevel != 0 { + n += 1 + sovRuntime(uint64(m.LogLevel)) + } if m.SessionLoop != nil { l = m.SessionLoop.ProtoSize() n += 2 + l + sovRuntime(uint64(l)) @@ -13132,38 +13129,6 @@ func (m *Task) Unmarshal(dAtA []byte) error { } 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 4: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field Preview", wireType) } @@ -13183,7 +13148,7 @@ func (m *Task) Unmarshal(dAtA []byte) error { } } m.Preview = bool(v != 0) - case 5: + case 3: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field MaxTransactions", wireType) } @@ -15467,6 +15432,25 @@ func (m *Materialize) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 5: + 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 20: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field SessionLoop", wireType) diff --git a/go/protocols/runtime/runtime.proto b/go/protocols/runtime/runtime.proto index d74367c713b..456247f3f6f 100644 --- a/go/protocols/runtime/runtime.proto +++ b/go/protocols/runtime/runtime.proto @@ -470,14 +470,10 @@ message Joined { message Task { // Task specification (protobuf-encoded bytes). bytes spec = 1; - // Collection journal partition to which task stats are written. - // The journal is pre-created by activate; the runtime appends directly - // without consulting the partitions watch or creating new partitions. - string ops_stats_journal = 2; // When true, documents and stats are written to output and not directed to collections. - bool preview = 4; + bool preview = 2; // Preview / harness control. Zero means unlimited. - uint32 max_transactions = 5; + uint32 max_transactions = 3; } // Recover is sent by each shard to the leader after Joined, and carries @@ -676,6 +672,12 @@ message Materialize { // Shard → Controller. Connector's reply to `validate`. materialize.Response.Validated validated = 4; + // Controller → Shard. Effective only on unary `spec` / `validate` + // messages, which never see the Join-time labeling that supplies the + // log level for session-bound work. Ignored on all other variants + // (the leader → shard messages MUST NOT set this). + ops.Log.Level log_level = 5; + // ----- STARTUP ----- // Controller → Shard. First message of a session-loop stream; diff --git a/go/runtime/materialize_v2.go b/go/runtime/materialize_v2.go index 0d694997d1b..fa6e3043142 100644 --- a/go/runtime/materialize_v2.go +++ b/go/runtime/materialize_v2.go @@ -166,14 +166,12 @@ func (m *materializeAppV2) runOneSession(shard consumer.Shard, ch chan<- consume close(ch) }() - // 1. Load and marshal everything that doesn't depend on Etcd topology. - var opsStatsJournal = m.term.labels.StatsJournal var specBytes []byte if specBytes, err = m.term.taskSpec.Marshal(); err != nil { return fmt.Errorf("marshaling MaterializationSpec: %w", err) } - // 2. Build & send Join, repeating on disagreement until consensus. + // Run the Join/Joined protocol loop until consensus is met. var waitForRevision int64 for { var ks = m.host.service.State.KS @@ -183,52 +181,50 @@ func (m *materializeAppV2) runOneSession(shard consumer.Shard, ch chan<- consume if err != nil { return fmt.Errorf("awaiting Etcd revision %d: %w", waitForRevision, err) } + + // Build Join from the current topology view, and send. var join, rev, err = m.buildJoin() if err != nil { return fmt.Errorf("building Join: %w", err) - } - if join == nil { + } else if join == nil { waitForRevision = rev + 1 continue } - if err := m.client.Send(&pr.Materialize{Join: join}); err != nil { - return fmt.Errorf("sending Join: %w", err) - } + _ = m.client.Send(&pr.Materialize{Join: join}) + + // Receive Joined and check consensus. var resp *pr.Materialize if resp, err = m.recv(); err != nil { return fmt.Errorf("receiving Joined: %w", pf.UnwrapGRPCError(err)) - } - if resp.Joined == nil { + } else if resp.Joined == nil { return fmt.Errorf("expected Joined, got %#v", resp) - } - if resp.Joined.MaxEtcdRevision == 0 { + } else if resp.Joined.MaxEtcdRevision == 0 { break // Consensus. + } else { + // Disagreement: await the indicated revision, then re-poll topology. + waitForRevision = resp.Joined.MaxEtcdRevision } - // Disagreement: await the indicated revision, then re-poll topology. - waitForRevision = resp.Joined.MaxEtcdRevision } - // 3. Send Task and wait for Opened. - if err := m.client.Send(&pr.Materialize{ + // Send Task. + _ = m.client.Send(&pr.Materialize{ Task: &pr.Task{ Spec: specBytes, - OpsStatsJournal: string(opsStatsJournal), Preview: false, MaxTransactions: 0, }, - }); err != nil { - return fmt.Errorf("sending Task: %w", err) - } - resp, err := m.recv() - if err != nil { + }) + + // Receive Opened. + var resp *pr.Materialize + if resp, err = m.recv(); err != nil { return fmt.Errorf("receiving Opened: %w", pf.UnwrapGRPCError(err)) - } - if resp.Opened == nil { + } else if resp.Opened == nil { return fmt.Errorf("expected Opened, got %#v", resp) } m.container.Store(resp.Opened.Container) - // 4. Steady-state: drive teardown signals and surface stream errors. + // Steady-state: drive teardown signals and surface stream errors. // termDone is nil-ed once we've sent Stop, so the case stops firing. // Future CloseNow plumbing slots in as another case alongside termDone. var termDone = m.term.ctx.Done() From 38516293010f504806054f556e634dbdc32c53b3 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 22:07:41 +0000 Subject: [PATCH 15/22] runtime-next: prune stale committed-frontier entries during recovery scan Long-lived tasks accumulate FC: entries for producers that stopped writing (including ones that wrote CONTINUE_TXN docs but never committed them), inflating startup cost, RocksDB size, and abandoned-transaction replay distance. Add `recovery::prune_committed_frontier`, a pure pass over the decoded per-(journal, binding) FC: chunks that drops a producer only when, within its group, it is not FH:-protected, trails the newest last_commit by at least FRONTIER_PRUNE_CLOCK_HORIZON, and trails the furthest read offset by at least FRONTIER_PRUNE_BYTE_HORIZON. The scan path then issues a small (non-synced; this is GC, not a commit) delete batch for the pruned FC: keys before returning Recover, so the leader never observes them. --- crates/runtime-next/README.md | 2 +- crates/runtime-next/src/shard/recovery.rs | 216 ++++++++++++++++++++-- crates/runtime-next/src/shard/rocksdb.rs | 112 +++++++++++ plans/runtime-v2/plan.md | 29 +++ 4 files changed, 344 insertions(+), 15 deletions(-) diff --git a/crates/runtime-next/README.md b/crates/runtime-next/README.md index 7a8e1a5840e..7dbf6d390ec 100644 --- a/crates/runtime-next/README.md +++ b/crates/runtime-next/README.md @@ -74,7 +74,7 @@ src/ │ └── shard/ # per-shard controller-facing service ├── service.rs # gRPC entry, dispatches by task type - ├── recovery.rs # Persist <-> RocksDB WriteBatch encode/decode + scan + ├── recovery.rs # Persist <-> RocksDB WriteBatch encode/decode + scan-time FC: pruning ├── rocksdb.rs # single Persist application path (capture will reuse) └── materialize/ ├── handler.rs # gRPC stream handler diff --git a/crates/runtime-next/src/shard/recovery.rs b/crates/runtime-next/src/shard/recovery.rs index fa61a288ffc..02165f74343 100644 --- a/crates/runtime-next/src/shard/recovery.rs +++ b/crates/runtime-next/src/shard/recovery.rs @@ -7,6 +7,9 @@ //! `(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. +//! [`prune_committed_frontier`] then drops stale `FC:` entries (conservatively; +//! see [`FRONTIER_PRUNE_CLOCK_HORIZON`] / [`FRONTIER_PRUNE_BYTE_HORIZON`]) so +//! `shard::rocksdb` can delete them from the DB before returning `Recover`. //! //! `{state_key}` below is the binding-stable `state_key` field of //! `flow::MaterializationSpec.Binding` — distinct from `journal_read_suffix`, @@ -60,6 +63,23 @@ pub const KEY_LAST_APPLIED: &[u8] = b"last-applied"; /// Trigger parameters (JSON `models::TriggerVariables`). pub const KEY_TRIGGER_PARAMS: &[u8] = b"trigger-params"; +/// Minimum clock distance between a committed-frontier producer and the most +/// recent committing producer of the same `(journal, state_key)` before the +/// stale producer becomes a pruning candidate. Time protects high-volume +/// journals from eager cleanup: when one producer quickly writes far ahead of +/// another, the byte-distance horizon alone would prune the laggard even though +/// little wall-clock time has actually passed. See [`prune_committed_frontier`]. +pub const FRONTIER_PRUNE_CLOCK_HORIZON: std::time::Duration = + std::time::Duration::from_secs(2 * 60 * 60); + +/// Minimum journal byte distance between a committed-frontier producer's read +/// offset and the furthest-along read offset of the same `(journal, state_key)` +/// before the stale producer becomes a pruning candidate. Byte distance is the +/// real operational cost of keeping an old pending span replayable: a retained +/// positive-offset entry forces the next session to re-read from that offset. +/// See [`prune_committed_frontier`]. +pub const FRONTIER_PRUNE_BYTE_HORIZON: i64 = 8 * 1024 * 1024 * 1024; + /// 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. @@ -304,6 +324,21 @@ fn append_frontier_key( out.extend_from_slice(producer); } +/// Build a `FC:{journal}\0{state_key}\0{producer}` committed-frontier key. +/// Used by `shard::rocksdb` to delete entries chosen by +/// [`prune_committed_frontier`]. +pub fn committed_frontier_key(journal: &str, state_key: &str, producer: &uuid::Producer) -> Bytes { + let mut buf = BytesMut::new(); + append_frontier_key( + &mut buf, + PREFIX_COMMITTED_FRONTIER, + journal, + state_key, + producer.as_bytes(), + ); + buf.freeze() +} + // --------------------------------------------------------------------------- // Decoder // --------------------------------------------------------------------------- @@ -480,6 +515,78 @@ fn decode_frontier_entry( Ok(()) } +// --------------------------------------------------------------------------- +// Frontier pruning +// --------------------------------------------------------------------------- + +/// Drop stale committed-frontier producers from `committed` in place, returning +/// the `(journal, binding, producer)` of each removed entry so the caller can +/// issue the matching RocksDB deletes. `JournalFrontier`s left without any +/// producers are removed from `committed`. +/// +/// Pruning is conservative: within a `(journal, binding)` group — equivalently a +/// `(journal, state_key)` group, since each state_key maps to a single binding — +/// a producer `P` is pruned only when **all** of: +/// +/// 1. No `FH:` (hinted) entry exists for the same `(journal, binding, producer)` +/// — a hinted producer's committed entry is its idempotent-replay baseline +/// and must be retained. +/// 2. `P.last_commit` trails the group's newest `last_commit` by at least +/// [`FRONTIER_PRUNE_CLOCK_HORIZON`]. +/// 3. `P`'s read offset (`offset.abs()`) trails the group's furthest-along read +/// offset by at least [`FRONTIER_PRUNE_BYTE_HORIZON`]. +/// +/// `committed` and `hinted` are the per-`(journal, binding)` chunks collected by +/// [`decode_recover_key_value`]; both are grouped (consecutive RocksDB key +/// order) but neither need be globally sorted. +pub fn prune_committed_frontier( + committed: &mut Vec, + hinted: &[shuffle::JournalFrontier], +) -> Vec<(Box, u16, uuid::Producer)> { + // Protected set: `(journal, binding, producer)` of every hinted entry. + let protected: std::collections::HashSet<(&str, u16, uuid::Producer)> = hinted + .iter() + .flat_map(|jf| { + jf.producers + .iter() + .map(move |p| (jf.journal.as_ref(), jf.binding, p.producer)) + }) + .collect(); + + let mut pruned = Vec::new(); + + committed.retain_mut(|jf| { + let group_clock = jf + .producers + .iter() + .map(|p| p.last_commit) + .max() + .unwrap_or_default(); + let group_offset = jf + .producers + .iter() + .map(|p| p.offset.unsigned_abs()) + .max() + .unwrap_or(0); + + jf.producers.retain(|p| { + let stale = !protected.contains(&(jf.journal.as_ref(), jf.binding, p.producer)) + && uuid::Clock::delta(group_clock, p.last_commit) >= FRONTIER_PRUNE_CLOCK_HORIZON + && group_offset.saturating_sub(p.offset.unsigned_abs()) + >= FRONTIER_PRUNE_BYTE_HORIZON as u64; + + if stale { + pruned.push((jf.journal.clone(), jf.binding, p.producer)); + } + !stale + }); + + !jf.producers.is_empty() + }); + + pruned +} + #[cfg(test)] mod test { use super::*; @@ -488,11 +595,7 @@ mod test { [0x01, tag, 0, 0, 0, 0] } - fn producer_frontier( - tag: u8, - last_commit: u64, - offset: i64, - ) -> shuffle::proto::ProducerFrontier { + fn proto_pf(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, @@ -509,17 +612,14 @@ mod test { shuffle::proto::JournalFrontier { journal_name_suffix: "acme/events/000".into(), binding: 0, - producers: vec![ - producer_frontier(0xaa, 100, 250), - producer_frontier(0xbb, 90, -300), - ], + producers: vec![proto_pf(0xaa, 100, 250), proto_pf(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)], + producers: vec![proto_pf(0xcc, 50, -50)], ..Default::default() }, ], @@ -756,8 +856,8 @@ mod test { #[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 fh_value = proto_pf(0xaa, 777, 12345).encode_to_vec(); + let fc_value = proto_pf(0xbb, 999, 4242).encode_to_vec(); let pairs = vec![ ( @@ -822,7 +922,7 @@ mod test { // 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 fh = proto_pf(0xaa, 1, 0).encode_to_vec(); let pairs = vec![ ( frontier_key( @@ -845,7 +945,7 @@ mod test { #[test] fn decode_recover_errors() { - let valid_value = Bytes::from(producer_frontier(0xaa, 1, 0).encode_to_vec()); + let valid_value = Bytes::from(proto_pf(0xaa, 1, 0).encode_to_vec()); // FH:/FC: layout: rest = journal \0 state_key \0 producer[6]. #[allow(clippy::type_complexity)] @@ -930,6 +1030,94 @@ mod test { } } + fn pf(tag: u8, last_commit_secs: u64, offset: i64) -> shuffle::ProducerFrontier { + shuffle::ProducerFrontier { + producer: uuid::Producer::from_bytes(producer_id(tag)), + last_commit: uuid::Clock::from_unix(last_commit_secs, 0), + hinted_commit: uuid::Clock::zero(), + offset, + } + } + + fn jf( + journal: &str, + binding: u16, + producers: Vec, + ) -> shuffle::JournalFrontier { + shuffle::JournalFrontier { + journal: journal.into(), + binding, + producers, + bytes_read_delta: 0, + bytes_behind_delta: 0, + } + } + + const GIB: i64 = 1024 * 1024 * 1024; + + #[test] + fn prune_committed_frontier_policy() { + // `j/old` exists only on binding 0; `j/two` exists on bindings 0 and 1. + // Within `(j/old, binding 0)`, 0xff is the "fresh" producer pinning the + // group's latest clock (~11.5 days) and read offset (20 GiB); the other + // producers each exercise one prune predicate. + let mut committed = vec![ + jf( + "j/old", + 0, + vec![ + pf(0xff, 1_000_000, -20 * GIB), // group clock/offset leader + pf(0xaa, 0, 5 * GIB), // old clock AND 15 GiB behind -> prune + pf(0xbb, 0, 8 * GIB), // old clock AND 12 GiB behind -> prune + pf(0xcc, 0, 13 * GIB), // old clock but only 7 GiB behind (< 8 GiB) -> retain + pf(0xdd, 999_000, 0), // 20 GiB behind but only 1000s old (< 2h) -> retain + pf(0xee, 0, GIB), // old + 19 GiB behind, but FH-protected -> retain + ], + ), + jf("j/two", 0, vec![pf(0xff, 5, -GIB)]), // group's only producer (the leader) -> retain + jf("j/two", 1, vec![pf(0xaa, 0, GIB)]), // group's only producer (the leader) -> retain + ]; + // 0xee on (j/old, binding 0) is hinted: its committed entry is the + // idempotent-replay baseline and must survive pruning. + let hinted = vec![jf("j/old", 0, vec![pf(0xee, 0, 0)])]; + + let pruned = prune_committed_frontier(&mut committed, &hinted); + + let mut pruned_tags: Vec<_> = pruned + .iter() + .map(|(j, b, p)| (j.to_string(), *b, p.as_bytes()[1])) + .collect(); + pruned_tags.sort(); + assert_eq!( + pruned_tags, + vec![ + ("j/old".to_string(), 0, 0xaa), + ("j/old".to_string(), 0, 0xbb), + ], + ); + + // Surviving producers of (j/old, binding 0): ff, cc, dd, ee. + let old0 = &committed[0]; + assert_eq!(old0.journal.as_ref(), "j/old"); + let mut survivors: Vec = old0 + .producers + .iter() + .map(|p| p.producer.as_bytes()[1]) + .collect(); + survivors.sort(); + assert_eq!(survivors, vec![0xcc, 0xdd, 0xee, 0xff]); + + // Both (j/two, _) journals retained intact. + assert_eq!(committed.len(), 3); + assert_eq!(committed[1].journal.as_ref(), "j/two"); + assert_eq!(committed[1].binding, 0); + assert_eq!(committed[2].journal.as_ref(), "j/two"); + assert_eq!(committed[2].binding, 1); + + // Pruning an already-pruned frontier is a no-op. + assert!(prune_committed_frontier(&mut committed, &hinted).is_empty()); + } + // 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. diff --git a/crates/runtime-next/src/shard/rocksdb.rs b/crates/runtime-next/src/shard/rocksdb.rs index b4502c64117..a2b45cf7d20 100644 --- a/crates/runtime-next/src/shard/rocksdb.rs +++ b/crates/runtime-next/src/shard/rocksdb.rs @@ -107,6 +107,10 @@ impl RocksDB { /// /// `binding_state_keys` is a sorted slice of `(state_key, binding_index)` /// tuples used to map from stable `state_key` to current binding index. + /// + /// As a side effect, stale committed-frontier (`FC:`) entries identified by + /// [`recovery::prune_committed_frontier`] are deleted from the DB before + /// `Recover` is returned, so the leader never observes them. pub async fn scan( self, binding_state_keys: Vec<(String, u32)>, @@ -139,6 +143,37 @@ impl RocksDB { () = it.status()?; std::mem::drop(it); + // Drop stale committed-frontier (`FC:`) entries: remove them + // from the recovered frontier and delete them from RocksDB so + // the leader never sees them and they stop costing scan time. + let pruned = + recovery::prune_committed_frontier(&mut committed_frontier, &hinted_frontier); + if !pruned.is_empty() { + // Invert `(state_key, binding)` → `binding → state_key`. + let state_key_of: std::collections::HashMap = binding_state_keys + .iter() + .map(|(sk, idx)| (*idx, sk.as_str())) + .collect(); + + let mut wb = rocksdb::WriteBatch::default(); + for (journal, binding, producer) in &pruned { + let state_key = state_key_of + .get(&(*binding as u32)) + .expect("pruned binding is present in the binding mapping"); + wb.delete(recovery::committed_frontier_key( + journal, state_key, producer, + )); + } + // `wo` is not sync because this is GC, not a commit. + let wo = rocksdb::WriteOptions::new(); + self.db.write_opt(wb, &wo)?; + + tracing::info!( + producers = pruned.len(), + "pruned stale committed-frontier entries during recovery scan" + ); + } + for (frontier, slot) in [ (&mut committed_frontier, &mut recover.committed_frontier), (&mut hinted_frontier, &mut recover.hinted_frontier), @@ -594,6 +629,83 @@ mod test { assert_eq!(hinted[1].binding, 1); } + /// `scan` drops stale `FC:` producers (old clock AND far behind in bytes) + /// from both the recovered frontier and the DB, but never an `FH:`-protected + /// producer's committed baseline. + #[tokio::test] + async fn scan_prunes_stale_committed_frontier() { + let db = RocksDB::open(None).await.unwrap(); + let producer = |tag: u8| proto_gazette::uuid::Producer::from_bytes([0x01, tag, 0, 0, 0, 0]); + let clock_secs = |s: u64| proto_gazette::uuid::Clock::from_unix(s, 0); + const GIB: i64 = 1024 * 1024 * 1024; + + let pf = |tag: u8, last_commit_secs: u64, offset: i64| shuffle::ProducerFrontier { + producer: producer(tag), + last_commit: clock_secs(last_commit_secs), + hinted_commit: proto_gazette::uuid::Clock::zero(), + offset, + }; + let committed = shuffle::JournalFrontier::encode(&[shuffle::JournalFrontier { + journal: "j/s".into(), + binding: 0, + producers: vec![ + pf(0x11, 1_000_000, -20 * GIB), // fresh leader: pins group clock + 20 GiB read offset + pf(0x22, 0, 0), // old clock + 20 GiB behind -> pruned + pf(0x33, 0, 0), // same, but FH-protected below -> retained + ], + bytes_read_delta: 0, + bytes_behind_delta: 0, + }]); + let hinted = shuffle::JournalFrontier::encode(&[shuffle::JournalFrontier { + journal: "j/s".into(), + binding: 0, + producers: vec![pf(0x33, 42, 7)], + bytes_read_delta: 0, + bytes_behind_delta: 0, + }]); + + let db = db + .persist( + &proto::Persist { + committed_frontier: Some(committed), + hinted_frontier: Some(hinted), + ..Default::default() + }, + &["sk0"], + ) + .await + .unwrap(); + + let mapping = vec![("sk0".to_string(), 0)]; + let (db, recover) = db.scan(mapping.clone()).await.unwrap(); + + let tags = |f: Option| -> Vec> { + f.into_iter() + .flat_map(shuffle::JournalFrontier::decode) + .map(|jf| { + jf.producers + .iter() + .map(|p| p.producer.as_bytes()[1]) + .collect() + }) + .collect() + }; + assert_eq!( + tags(recover.committed_frontier), + vec![vec![0x11_u8, 0x33]], + "0x22 pruned from recovered committed frontier" + ); + assert_eq!( + tags(recover.hinted_frontier), + vec![vec![0x33_u8]], + "hinted frontier untouched" + ); + + // A second scan confirms 0x22's FC: key was actually deleted. + let (_db, recover2) = db.scan(mapping).await.unwrap(); + assert_eq!(tags(recover2.committed_frontier), vec![vec![0x11_u8, 0x33]]); + } + /// 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 diff --git a/plans/runtime-v2/plan.md b/plans/runtime-v2/plan.md index 91ff8465b70..ed7ced9dea1 100644 --- a/plans/runtime-v2/plan.md +++ b/plans/runtime-v2/plan.md @@ -192,6 +192,35 @@ new code must continue to honor: the task still has per-shard committed state, and startup fails explicitly instead of proceeding with an unsafe consolidation. +## Frontier pruning + +Long-lived tasks can accumulate committed frontier entries for producers +that have stopped writing, including producers that wrote `CONTINUE_TXN` +documents but never later committed them. V2 should eventually retain +only enough producer frontier state to preserve exactly-once recovery for +plausibly active producers, while bounding startup cost, RocksDB size, and +the replay distance implied by abandoned transactions. + +Pruning should be conservative. A producer is a candidate only after both +the clock horizon and byte-distance horizon say that enough newer activity +has passed within the same source journal and binding. Time protects +high-volume journals from eager cleanup when another producer quickly +writes far ahead; byte distance captures the real operational cost of +keeping an old pending span replayable. Frontier +entries that participate in recovered hinted state are not ordinary stale +state and must not be silently removed. + +The cleanup point is recovery scan, not `Persist`. `Persist` writes +frontiers as fine-grained `FC:{journal}\0{state_key}\0{producer}` and +`FH:{journal}\0{state_key}\0{producer}` key updates, and the leader should +not need to hold or rewrite a fully reduced committed frontier after V1 +migration is complete. During RocksDB scan, shard zero can group recovered +`FC:` entries by raw `(journal, state_key)`, compute the latest producer +clock and read offset for the group, drop entries that satisfy the pruning +policy, and write a small delete batch for the pruned `FC:` keys before +returning `Recover`. `FH:` entries form a protected set: matching `FC:` +entries are retained so idempotent replay has its committed baseline. + ## Capture architecture Capture shards operate **independently** — each shard has its own From b02ffcc8a36a86b875065b164631d3c75188a8a5 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 22:08:36 +0000 Subject: [PATCH 16/22] runtime-next: fix leader close-policy thresholds and stats orientation The close-policy comparisons used a strict `>`, so a threshold of zero could never be satisfied; use `>=` so zero-valued thresholds fire. Widen the `last_close_age` placeholder ceiling from 300s to `Duration::MAX`. The materialize stats doc also reported the `sourced` and `loaded` document/byte tallies under swapped `left`/`right` keys; correct the orientation. --- .../src/leader/materialize/fsm.rs | 61 ++++++++++--------- .../src/leader/materialize/task.rs | 7 +-- 2 files changed, 34 insertions(+), 34 deletions(-) diff --git a/crates/runtime-next/src/leader/materialize/fsm.rs b/crates/runtime-next/src/leader/materialize/fsm.rs index 04d9f59bb30..44462fa048a 100644 --- a/crates/runtime-next/src/leader/materialize/fsm.rs +++ b/crates/runtime-next/src/leader/materialize/fsm.rs @@ -1160,11 +1160,11 @@ pub fn decide_close_policy(inputs: CloseInputs, task: &Task) -> CloseDecision { && 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); + 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; @@ -1215,8 +1215,8 @@ fn build_stats_doc( ); 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.sourced, &mut entry.right); + ops::merge_docs_and_bytes(&extents.loaded, &mut entry.left); ops::merge_docs_and_bytes(&extents.stored, &mut entry.out); } @@ -1637,32 +1637,33 @@ mod tests { 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"); - } + // Loaded(0) lands; HeadExtend still awaits Loaded(1) and rests. + ctx.shard_rx = Some(mk_loaded(0)); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; assert!(matches!(head, Head::Extend(_))); - // Extend the open transaction with a second ready Frontier. + // A second ready Frontier becomes available before Loaded(1) arrives — + // simulating the actor's loop pre-fetching the next frontier while + // awaiting the prior round's Loaded responses. ctx.ready_frontier = Some(shuffle::Frontier::default()); + ctx.shard_rx = Some(mk_loaded(1)); let (action, h) = ctx.step_head(head, &mut tail); head = h; + // With both inputs available the FSM extends rather than closes. assert!(matches!(action, Action::Load { .. })); + assert!(matches!(head, Head::Extend(_))); - for s in 0..2 { - ctx.shard_rx = Some(mk_loaded(s)); - let (_action, h) = ctx.step_head(head, &mut tail); - head = h; - } + // Second Load round: Loaded x2 arrive without another frontier queued. + // After the final Loaded the close-policy fires: ready_frontier is + // None and may_close is true (Tail::Done), so + // HeadExtend transitions straight into HeadFlush. + ctx.shard_rx = Some(mk_loaded(0)); + let (_action, h) = ctx.step_head(head, &mut tail); + head = h; + assert!(matches!(head, Head::Extend(_))); - // Close: with Tail::Done and no unresolved hints, `close_requested` - // forces may_close and we begin L:Flush. - ctx.close_requested = true; + ctx.shard_rx = Some(mk_loaded(1)); let (action, h) = ctx.step_head(head, &mut tail); head = h; assert!(matches!(action, Action::Flush { .. })); @@ -1721,18 +1722,18 @@ mod tests { "_meta": {}, "shard": {}, "ts": "2023-11-14T22:13:20.000000004+00:00", - "openSecondsTotal": 0.000000024, + "openSecondsTotal": 0.000000016, "txnCount": 1, "materialize": { "test/collection": { "left": { - "docsTotal": 12, - "bytesTotal": 1200 - }, - "right": { "docsTotal": 4, "bytesTotal": 400 }, + "right": { + "docsTotal": 12, + "bytesTotal": 1200 + }, "out": { "docsTotal": 8, "bytesTotal": 800 diff --git a/crates/runtime-next/src/leader/materialize/task.rs b/crates/runtime-next/src/leader/materialize/task.rs index 8db1c29b722..dcee0e583b8 100644 --- a/crates/runtime-next/src/leader/materialize/task.rs +++ b/crates/runtime-next/src/leader/materialize/task.rs @@ -80,12 +80,11 @@ impl Task { // 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 last_close_age = std::time::Duration::ZERO..std::time::Duration::MAX; + let open_duration = min_txn_duration..max_txn_duration; let read_bytes = 0..u64::MAX; + let read_docs = 0..u64::MAX; Ok(Self { binding_collection_names, From 0ec2cb700e101eec7e4ee5c46f2f5e6c6da72708 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 22:09:11 +0000 Subject: [PATCH 17/22] runtime: strip the V2 committed-close marker on rollback to V1 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The V2 leader stamps a synthetic "committed-close" source into the consumer.Checkpoint on each commit, recording the V2 RocksDB epoch. If a task is rolled back to the V1 runtime, V1 would otherwise carry that marker verbatim across its own commits; a later roll-forward to V2 would then mistake the stale marker for an in-sync RocksDB state, ignore the legacy_checkpoint, and resume from V2's stale frontier — reprocessing whatever V1 had advanced past. Strip the "committed-close" source on each V1 start-commit so a subsequent V2 startup treats V1's advanced sources as authoritative. --- crates/runtime/src/materialize/protocol.rs | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/crates/runtime/src/materialize/protocol.rs b/crates/runtime/src/materialize/protocol.rs index 3bfaa8d608f..238bb5a21cb 100644 --- a/crates/runtime/src/materialize/protocol.rs +++ b/crates/runtime/src/materialize/protocol.rs @@ -572,7 +572,7 @@ pub fn recv_client_start_commit( txn: &mut Transaction, ) -> anyhow::Result<(Request, rocksdb::WriteBatch)> { let verify = verify("client", "StartCommit with runtime_checkpoint"); - let request = verify.not_eof(request)?; + let mut request = verify.not_eof(request)?; let Request { start_commit: @@ -581,7 +581,7 @@ pub fn recv_client_start_commit( .. }), .. - } = &request + } = &mut request else { return verify.fail(request); }; @@ -590,6 +590,17 @@ pub fn recv_client_start_commit( // merge-able, incremental update that's written to the WriteBatch. let _last_checkpoint = last_checkpoint; + // The V2 leader stamps a synthetic "committed-close" source into the + // consumer.Checkpoint on each commit, recording the V2 RocksDB epoch. + // If V1 inherits a checkpoint from a prior V2 run, the marker is + // preserved verbatim across V1 commits. A subsequent V2 rollforward + // would then mistake the stale marker for an in-sync RocksDB state + // and ignore the legacy_checkpoint, resuming from V2's stale frontier + // and re-processing whatever V1 had advanced past. Strip the marker + // so V2 startup treats V1's advanced sources as authoritative. + runtime_checkpoint.sources.remove("committed-close"); + let runtime_checkpoint = runtime_checkpoint.clone(); + let mut wb = rocksdb::WriteBatch::default(); tracing::debug!( @@ -598,7 +609,7 @@ pub fn recv_client_start_commit( ); wb.put(RocksDB::CHECKPOINT_KEY, runtime_checkpoint.encode_to_vec()); - txn.checkpoint = runtime_checkpoint.clone(); + txn.checkpoint = runtime_checkpoint; Ok((request, wb)) } From bb52903439058e5747866ec13613b2b4541e6d48 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 22:11:24 +0000 Subject: [PATCH 18/22] go/runtime: fail the shard when the runtime-v2 feature flag is toggled `NewStore` is invoked only on the initial PRIMARY transition, so a publish that flips the `enable-runtime-v2` flag on a running shard cannot otherwise reroute it between the V1 and V2 materialize runtimes. Have each app's `RestoreCheckpoint` surface a functional error when its shard's flag no longer matches the running runtime, forcing the controller to restart the shard so `NewStore` re-evaluates the flag and selects the correct runtime. --- go/runtime/materialize.go | 10 ++++++++++ go/runtime/materialize_v2.go | 11 +++++++++++ 2 files changed, 21 insertions(+) diff --git a/go/runtime/materialize.go b/go/runtime/materialize.go index 62b8d0c402e..c2976cb6525 100644 --- a/go/runtime/materialize.go +++ b/go/runtime/materialize.go @@ -59,6 +59,16 @@ func (m *materializeApp) RestoreCheckpoint(shard consumer.Shard) (_ pf.Checkpoin return pf.Checkpoint{}, err } + // Fail the shard if its runtime-v2 flag has been turned on. NewStore is + // invoked only on the initial PRIMARY transition, so a publish that + // flips the flag on a running shard cannot otherwise reroute it. We + // surface a functional error so the controller restarts the shard, + // at which point NewStore re-evaluates the flag and selects V2. + if useRuntimeV2(shard.Spec().LabelSet) { + return pf.Checkpoint{}, fmt.Errorf( + "runtime-v2 feature flag is set but this shard is running the V1 materialize runtime; failing to force a restart") + } + var requestExt = &pr.MaterializeRequestExt{ LogLevel: m.term.labels.LogLevel, } diff --git a/go/runtime/materialize_v2.go b/go/runtime/materialize_v2.go index fa6e3043142..b7290eb52c4 100644 --- a/go/runtime/materialize_v2.go +++ b/go/runtime/materialize_v2.go @@ -136,6 +136,17 @@ func (m *materializeAppV2) RestoreCheckpoint(shard consumer.Shard) (pf.Checkpoin // STANDBY to PRIMARY: the shard is updated to PRIMARY only after its // return, and before StartReadingMessages. + // Fail the shard if its runtime-v2 flag has been turned off. NewStore is + // invoked only on the initial PRIMARY transition, so a publish that + // flips the flag on a running shard cannot otherwise reroute it. We + // surface a functional error so the controller restarts the shard, + // at which point NewStore re-evaluates the flag and selects V1. + if !useRuntimeV2(shard.Spec().LabelSet) { + m.term.cancel() + return pf.Checkpoint{}, fmt.Errorf( + "runtime-v2 feature flag is unset but this shard is running the V2 materialize runtime; failing to force a restart") + } + // The Rust runtime owns checkpoint persistence, recovery, ACK intent // publishing, and the core transaction loop. Nothing to recover here. return pf.Checkpoint{}, nil From d195be878466d19400622d5b824af34340d74e2c Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 22:11:56 +0000 Subject: [PATCH 19/22] mise: add runtime-sidecar systemd unit and local data-plane wiring Each local data plane now runs a dedicated runtime-v2 sidecar on base_port+60, advertising the same per-plane FQDN and HMAC key as its reactors. Cap brokers and reactors at 10 instances each so the +0..+9 and +90..+99 ranges stay clear of the +50/+51/+52 Dekaf and +60 sidecar reservations. Also set CONSUMER_ZONE on reactors so sidecar peering resolves, prefer the musl target dir ahead of glibc on PATH so an over-broad `cargo build` doesn't shadow the musl flow-connector-init, disable color in sidecar logs, and document the preview-harness scope and the Supabase Docker-network connector wiring. --- local/README.md | 39 +++++++------ local/systemd/flow-runtime-sidecar@.service | 16 ++++++ mise.toml | 2 +- mise/README.md | 1 + mise/tasks/local/common | 6 +- mise/tasks/local/data-plane | 16 ++++-- mise/tasks/local/gazette | 5 ++ mise/tasks/local/reactor | 15 ++++- mise/tasks/local/runtime-sidecar | 63 +++++++++++++++++++++ plans/runtime-v2/preview-harness.md | 12 ++++ 10 files changed, 148 insertions(+), 27 deletions(-) create mode 100644 local/systemd/flow-runtime-sidecar@.service create mode 100755 mise/tasks/local/runtime-sidecar diff --git a/local/README.md b/local/README.md index 7d70649bb4f..799b9f89c20 100644 --- a/local/README.md +++ b/local/README.md @@ -1,11 +1,11 @@ -# Local Stack +# Local Stack Architecture & Operations This directory holds the systemd units and helper scripts that `mise run local:*` glues together into a working Estuary stack. The interesting machinery is in `local/systemd/` (unit files) and `mise/tasks/local/` (drivers that emit env files and dropins, then `systemctl --user start`). -## Topology at a glance +## Topology ``` flow-supabase.service ──┐ @@ -17,7 +17,7 @@ flow-etcd.service (single-node, http://etcd.flow.localhost:2379) flow-plane@.target ─────────── one per data plane ├─ flow-gazette@- broker(s), base_port + 0..9 ├─ flow-reactor@- reactor(s), base_port + 99..90 - ├─ flow-runtime-sidecar@ Rust sidecar, base_port + 60 + ├─ flow-runtime-sidecar@ Rust sidecar, base_port + 60 (gRPC) / +61 (admin HTTP) ├─ flow-dekaf@ Kafka shim, base_port + 50/51/52 └─ flow-plane-link@ oneshot: POSTs to agent to register @@ -30,21 +30,16 @@ port-block 8000 with 4 brokers, 1 reactor, dekaf, and link, and then publishes ## The port scheme is load-bearing -Each data plane gets a 100-wide block starting on a multiple of 100. Within the -block: +Each data plane gets a 100-wide block starting on a multiple of 100; the +per-service offsets within it (annotated in the topology diagram above) are +fixed by the mise tasks, and the data-plane script rejects a base port that +isn't `% 100 == 0`. One surprising consequence worth internalizing: +reactors count *down* from the end of the block, so a "single reactor" +lands on `block+99` — `local-cluster`'s sole reactor is at **8099**, not 8004. -| Offset | Service | -| --- | --- | -| +0..+9 | brokers (count up from block start) | -| +50, +51, +52 | dekaf kafka / registry / metrics | -| +60 | runtime-sidecar | -| +90..+99 | reactors (count down from block end) | - -So a "single reactor" lands on `block+99` — `local-cluster` puts its sole -reactor at **8099**, not 8004. The data-plane script enforces `% 100 == 0` on -the base port. `mise/tasks/vm/port-forward` hardcodes three reserved blocks: -**8000–8099**, **10000–10099**, **10100–10199** — those are your three -forwardable data planes from a VM. +`mise/tasks/vm/port-forward` hardcodes three reserved blocks — **8000–8099**, +**10000–10099**, **10100–10199** — so those are your three forwardable data +planes from a VM. The dockerized Kafka that backs Dekaf consumer groups lives outside any block, on **:29092**, and is shared across all data planes. @@ -196,6 +191,16 @@ swaps every URL to a localhost equivalent: The `--profile` flag also controls which config file in `~/.config/flowctl/` is loaded. +## Connectors run on the Supabase Docker network + +Reactor and agent set `FLOW_NETWORK=supabase_network_flow` (see +`mise/tasks/local/reactor`), so connector containers — at build/discover time +and at runtime — run on that Docker network, not the default bridge. The handy +consequence: a connector endpoint config can point straight at the Supabase +Postgres, which sits on the same network as `supabase_db_flow` and answers to +`db:5432`. Any *other* service a connector needs has to be `docker network +connect supabase_network_flow`'d first, and then addressed by container name. + ## What `local:stack` is actually publishing After the data plane comes up, `local:stack` opens a psql session and writes a diff --git a/local/systemd/flow-runtime-sidecar@.service b/local/systemd/flow-runtime-sidecar@.service new file mode 100644 index 00000000000..b9030f2de64 --- /dev/null +++ b/local/systemd/flow-runtime-sidecar@.service @@ -0,0 +1,16 @@ +[Unit] +Description=Flow Runtime Sidecar (%i) +Documentation=https://github.com/estuary/flow + +[Service] +Type=simple +TimeoutStartSec=600 + +EnvironmentFile=%h/flow-local/env/common.env +EnvironmentFile=%h/flow-local/env/runtime-sidecar-%i.env + +ExecStartPre=sh -c 'cd ${FLOW_ROOT} && cargo build -p runtime-sidecar' +ExecStart=%h/cargo-target/debug/runtime-sidecar + +Restart=on-failure +RestartSec=5s diff --git a/mise.toml b/mise.toml index 145b0efe403..355d60d3a73 100644 --- a/mise.toml +++ b/mise.toml @@ -9,7 +9,7 @@ go = { version = "1.25" } "github:wasm-bindgen/wasm-pack" = { version = "0.13.1" } "github:etcd-io/etcd" = { version = "3.5.24" } "github:getsops/sops" = { version = "3.11.0", bin = "sops" } -"github:jdx/usage" = { version = "2.6.0" } +"github:jdx/usage" = { version = "2.11.0" } "github:mozilla/sccache" = { version = "0.12.0", platforms = { linux-arm64 = { asset_pattern = "sccache-v*-aarch64-unknown-linux-musl.tar.gz" }, linux-x64 = { asset_pattern = "sccache-v*-x86_64-unknown-linux-musl.tar.gz" }, macos-arm64 = { asset_pattern = "sccache-v*-aarch64-apple-darwin.tar.gz" } } } "github:nextest-rs/nextest" = { version = "cargo-nextest-0.9.114" } "github:rui314/mold" = { version = "2.40.4" } diff --git a/mise/README.md b/mise/README.md index e244745d5f2..68a920b4a3e 100644 --- a/mise/README.md +++ b/mise/README.md @@ -291,6 +291,7 @@ mise tasks | `local:control-plane` | Start control plane only | | `local:data-plane ` | Start a data plane | | `local:data-plane-controller` | Start data-plane-controller (service + job) in dry-run mode | +| `local:runtime-sidecar` | Start runtime-v2 sidecar for a local data plane | | `local:seed-controller-job` | Seed a controller job to trigger data plane converge | | `local:supabase` | Start Supabase only | | `local:stop` | Stop all services and clean up | diff --git a/mise/tasks/local/common b/mise/tasks/local/common index 9301bf855fd..41f667ad980 100755 --- a/mise/tasks/local/common +++ b/mise/tasks/local/common @@ -14,7 +14,9 @@ cat > "${FLOW_LOCAL}/env/common.env" <&2 +# Cap brokers and reactors at 10 each. Brokers occupy base_port+0..+9 and +# reactors occupy base_port+90..+99, leaving the +50/+51/+52 Dekaf ports +# and the +60 runtime-sidecar port reserved. +if [ "${N_BROKERS}" -gt 10 ]; then + echo "Error: --num-brokers ${N_BROKERS} exceeds the per-data-plane limit of 10" >&2 + exit 1 +fi +if [ "${N_REACTORS}" -gt 10 ]; then + echo "Error: --num-reactors ${N_REACTORS} exceeds the per-data-plane limit of 10" >&2 exit 1 fi FLOW_LOCAL="${HOME}/flow-local" HMAC_KEY=$(echo -n "key-${DATA_PLANE}" | base64 | tr -d '\n') -# Deterministic Dekaf port allocation within the 100-port block +# Deterministic port allocations within the 100-port block DEKAF_KAFKA_PORT=$((PORT_BLOCK + 50)) DEKAF_REGISTRY_PORT=$((PORT_BLOCK + 51)) DEKAF_METRICS_PORT=$((PORT_BLOCK + 52)) +SIDECAR_PORT=$((PORT_BLOCK + 60)) # Start broker instances if requested if [ "${N_BROKERS}" -gt 0 ] 2>/dev/null; then diff --git a/mise/tasks/local/gazette b/mise/tasks/local/gazette index 3fd883b5439..cf63e86d13c 100755 --- a/mise/tasks/local/gazette +++ b/mise/tasks/local/gazette @@ -10,6 +10,11 @@ DATA_PLANE="${usage_data_plane}" BASE_PORT="${usage_base_port}" COUNT="${usage_count}" +if [ "${COUNT}" -gt 10 ]; then + echo "Error: --count ${COUNT} exceeds the per-data-plane broker limit of 10" >&2 + exit 1 +fi + FLOW_LOCAL="${HOME}/flow-local" HMAC_KEY=$(echo -n "key-${DATA_PLANE}" | base64 | tr -d '\n') diff --git a/mise/tasks/local/reactor b/mise/tasks/local/reactor index 02081930329..e52317d66aa 100755 --- a/mise/tasks/local/reactor +++ b/mise/tasks/local/reactor @@ -12,8 +12,16 @@ BASE_PORT="${usage_base_port}" BROKER_PORT="${usage_broker_port}" COUNT="${usage_count}" +if [ "${COUNT}" -gt 10 ]; then + echo "Error: --count ${COUNT} exceeds the per-data-plane reactor limit of 10" >&2 + exit 1 +fi + FLOW_LOCAL="${HOME}/flow-local" HMAC_KEY=$(echo -n "key-${DATA_PLANE}" | base64 | tr -d '\n') +SIDECAR_PORT=$((BROKER_PORT + 60)) + +mise run local:runtime-sidecar "${DATA_PLANE}" "${BROKER_PORT}" # Configure all reactor instances for i in $(seq 0 $((COUNT - 1))); do @@ -29,7 +37,7 @@ for i in $(seq 0 $((COUNT - 1))); do # Note the local cert is signed with a wildcard of *.flow.localhost, # and CONSUMER_HOST must match for the connector networking feature to function. -# SSL_CERT_FILE is required for calls to the control agent API. +# SSL_CERT_FILE is required for calls to the runtime sidecar and agent API. BROKER_ADDRESS=https://broker-${DATA_PLANE}.flow.localhost:${BROKER_PORT} BROKER_AUTH_KEYS=${HMAC_KEY} @@ -44,6 +52,7 @@ CONSUMER_PEER_CA_FILE=${FLOW_LOCAL}/ca.crt CONSUMER_PORT=${REACTOR_PORT} CONSUMER_SERVER_CERT_FILE=${FLOW_LOCAL}/server.crt CONSUMER_SERVER_CERT_KEY_FILE=${FLOW_LOCAL}/server.key +CONSUMER_ZONE=local ETCD_ADDRESS=http://etcd.flow.localhost:2379 ETCD_PREFIX=/flow/${DATA_PLANE} FLOW_ALLOW_LOCAL=true @@ -52,8 +61,9 @@ FLOW_CONTROL_API=http://agent.flow.localhost:8675 FLOW_DASHBOARD=http://localhost:3000 FLOW_DATA_PLANE_FQDN=${DATA_PLANE}.dp.estuary-data.com FLOW_NETWORK=supabase_network_flow +FLOW_SIDECAR_PORT=${SIDECAR_PORT} LOG_LEVEL=info -SSL_CERT_FILE=${FLOW_LOCAL}/ca.crt # Required for agent API calls. +SSL_CERT_FILE=${FLOW_LOCAL}/ca.crt # Test (non-production) key for local development. # This matches KMS_KEY in the control-plane development task. @@ -67,6 +77,7 @@ EOF [Unit] PartOf=flow-plane@${DATA_PLANE}.target After=flow-plane@${DATA_PLANE}.target +After=flow-runtime-sidecar@${DATA_PLANE}.service EOF DROPIN_DIR="${HOME}/.config/systemd/user/flow-plane@${DATA_PLANE}.target.d" diff --git a/mise/tasks/local/runtime-sidecar b/mise/tasks/local/runtime-sidecar new file mode 100755 index 00000000000..c511ba32640 --- /dev/null +++ b/mise/tasks/local/runtime-sidecar @@ -0,0 +1,63 @@ +#!/usr/bin/env bash +set -euo pipefail + +#MISE description="Start runtime-v2 sidecar for a local data plane via systemd" +#MISE depends=["local:common", "local:tls-cert"] +#USAGE arg "" help="Name of the data plane" +#USAGE arg "" help="Base port block for the data plane" + +DATA_PLANE="${usage_data_plane}" +BASE_PORT="${usage_base_port}" + +FLOW_LOCAL="${HOME}/flow-local" +HMAC_KEY=$(echo -n "key-${DATA_PLANE}" | base64 | tr -d '\n') +SIDECAR_PORT=$((BASE_PORT + 60)) + +if [ $((BASE_PORT % 100)) -ne 0 ]; then + echo "Error: Port block ${BASE_PORT} must be divisible by 100 (e.g., 8100, 9600)" >&2 + exit 1 +fi + +cat > "${FLOW_LOCAL}/env/runtime-sidecar-${DATA_PLANE}.env" < "${DROPIN_DIR}/plane.conf" < "${DROPIN_DIR}/wants-runtime-sidecar.conf" </`. Note connectors on the +local stack run on the `supabase_network_flow` Docker network — see +`local/README.md` for the endpoint-address implications. + ## One-time setup Done once per workstation. Skip the steps you've already completed. From 33f031aeb0fdbd5850ab2f3178500f3c450f16b8 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger <209664+jgraettinger@users.noreply.github.com> Date: Fri, 15 May 2026 22:15:47 +0000 Subject: [PATCH 20/22] service-kit: observability crate for handler registry, admin surface, and metrics MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add `crates/service-kit/`, a service-agnostic leaf crate that provides the observability foundation for the runtime-v2 sidecar: - `Registry` / `HandlerGuard`: a coarse lifecycle view of in-flight units of work (label / phase / fields), each running inside its own `tracing` handler span. - `admin`: a loopback-only `axum` surface — an auto-refreshing HTML dashboard, `/debug/handlers.json`, a per-handler drill-down page, and a `POST /debug/handlers/{id}/level/{level}` runtime trace-level control. - `trace`: a `tracing_subscriber` layer-filter, composed with the base `EnvFilter`, that admits events at or above an enclosing handler span's override level. - `event!`: a structured-event macro with lazy field capture, feeding both `tracing` and per-handler breadcrumb rings shown on the drill-down page. - `metrics`: a Prometheus registry and `/metrics` route. The crate is added inert; the following commit wires it into the sidecar's Shuffle and Leader services. --- Cargo.lock | 147 ++++- Cargo.toml | 7 +- crates/service-kit/Cargo.toml | 28 + crates/service-kit/README.md | 70 +++ crates/service-kit/src/admin.rs | 569 +++++++++++++++++ crates/service-kit/src/event.rs | 952 +++++++++++++++++++++++++++++ crates/service-kit/src/handlers.rs | 522 ++++++++++++++++ crates/service-kit/src/lib.rs | 27 + crates/service-kit/src/metrics.rs | 120 ++++ crates/service-kit/src/trace.rs | 201 ++++++ plans/runtime-v2/plan.md | 18 + 11 files changed, 2637 insertions(+), 24 deletions(-) create mode 100644 crates/service-kit/Cargo.toml create mode 100644 crates/service-kit/README.md create mode 100644 crates/service-kit/src/admin.rs create mode 100644 crates/service-kit/src/event.rs create mode 100644 crates/service-kit/src/handlers.rs create mode 100644 crates/service-kit/src/lib.rs create mode 100644 crates/service-kit/src/metrics.rs create mode 100644 crates/service-kit/src/trace.rs diff --git a/Cargo.lock b/Cargo.lock index f32be10df48..087d98aa4e4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1030,7 +1030,7 @@ dependencies = [ "http 1.3.1", "http-body 0.4.6", "hyper 0.14.32", - "hyper 1.7.0", + "hyper 1.9.0", "hyper-rustls 0.24.2", "hyper-rustls 0.27.7", "hyper-util", @@ -1178,7 +1178,7 @@ dependencies = [ "http 1.3.1", "http-body 1.0.1", "http-body-util", - "hyper 1.7.0", + "hyper 1.9.0", "hyper-util", "itoa", "matchit", @@ -1258,16 +1258,16 @@ dependencies = [ [[package]] name = "axum-server" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "495c05f60d6df0093e8fb6e74aa5846a0ad06abaf96d76166283720bf740f8ab" +checksum = "c1ab4a3ec9ea8a657c72d99a03a824af695bd0fb5ec639ccbd9cd3543b41a5f9" dependencies = [ "arc-swap", "bytes", "fs-err", "http 1.3.1", "http-body 1.0.1", - "hyper 1.7.0", + "hyper 1.9.0", "hyper-util", "pin-project-lite", "rustls 0.23.32", @@ -3137,6 +3137,17 @@ dependencies = [ "pin-project-lite", ] +[[package]] +name = "evmap" +version = "11.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b8874945f036109c72242964c1174cf99434e30cfa45bf45fedc983f50046f8" +dependencies = [ + "hashbag", + "left-right", + "smallvec", +] + [[package]] name = "extractors" version = "0.0.0" @@ -3640,6 +3651,21 @@ dependencies = [ "uuid", ] +[[package]] +name = "generator" +version = "0.8.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52f04ae4152da20c76fe800fa48659201d5cf627c5149ca0b707b69d7eef6cf9" +dependencies = [ + "cc", + "cfg-if", + "libc", + "log", + "rustversion", + "windows-link 0.2.0", + "windows-result 0.4.0", +] + [[package]] name = "generic-array" version = "0.14.7" @@ -3860,6 +3886,12 @@ dependencies = [ "thiserror 2.0.17", ] +[[package]] +name = "hashbag" +version = "0.1.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7040a10f52cba493ddb09926e15d10a9d8a28043708a405931fe4c6f19fac064" + [[package]] name = "hashbrown" version = "0.12.3" @@ -4139,9 +4171,9 @@ dependencies = [ [[package]] name = "hyper" -version = "1.7.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb3aa54a13a0dfe7fbe3a59e0c76093041720fdc77b110cc0fc260fafb4dc51e" +checksum = "6299f016b246a94207e63da54dbe807655bf9e00044f73ded42c3ac5305fbcca" dependencies = [ "atomic-waker", "bytes", @@ -4154,7 +4186,6 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "pin-utils", "smallvec", "tokio", "want", @@ -4183,7 +4214,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e3c93eb611681b207e1fe55d5a71ecf91572ec8a6705cdb6857f7d8d5242cf58" dependencies = [ "http 1.3.1", - "hyper 1.7.0", + "hyper 1.9.0", "hyper-util", "rustls 0.23.32", "rustls-native-certs 0.8.1", @@ -4200,7 +4231,7 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2b90d566bffbce6a75bd8b09a05aa8c2cb1fabb6cb348f8840c9e4c90a0d83b0" dependencies = [ - "hyper 1.7.0", + "hyper 1.9.0", "hyper-util", "pin-project-lite", "tokio", @@ -4228,7 +4259,7 @@ checksum = "70206fc6890eaca9fde8a0bf71caa2ddfc9fe045ac9e5c70df101a7dbde866e0" dependencies = [ "bytes", "http-body-util", - "hyper 1.7.0", + "hyper 1.9.0", "hyper-util", "native-tls", "tokio", @@ -4249,7 +4280,7 @@ dependencies = [ "futures-util", "http 1.3.1", "http-body 1.0.1", - "hyper 1.7.0", + "hyper 1.9.0", "ipnet", "libc", "percent-encoding", @@ -4747,6 +4778,17 @@ dependencies = [ "spin", ] +[[package]] +name = "left-right" +version = "0.11.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f0c21e4c8ff95f487fb34e6f9182875f42c84cef966d29216bf115d9bba835a" +dependencies = [ + "crossbeam-utils", + "loom", + "slab", +] + [[package]] name = "lexical-core" version = "1.0.6" @@ -4972,6 +5014,19 @@ version = "0.4.28" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "34080505efa8e45a4b816c349525ebe327ceaa8559756f0356cba97ef3bf7432" +[[package]] +name = "loom" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "419e0dc8046cb947daa77eb95ae174acfbddb7673b4151f56d1eed8e93fbfaca" +dependencies = [ + "cfg-if", + "generator", + "scoped-tls", + "tracing", + "tracing-subscriber", +] + [[package]] name = "lru" version = "0.12.5" @@ -5065,23 +5120,24 @@ checksum = "f52b00d39961fc5b2736ea853c9cc86238e165017a493d1d5c8eac6bdc4cc273" [[package]] name = "metrics" -version = "0.24.2" +version = "0.24.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25dea7ac8057892855ec285c440160265225438c3c45072613c25a4b26e98ef5" +checksum = "89550ee9f79e88fef3119de263694973a8adb26c21d75322164fb8c493039fe2" dependencies = [ - "ahash", "portable-atomic", + "rapidhash", ] [[package]] name = "metrics-exporter-prometheus" -version = "0.17.2" +version = "0.18.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b166dea96003ee2531cf14833efedced545751d800f03535801d833313f8c15" +checksum = "1db0d8f1fc9e62caebd0319e11eaec5822b0186c171568f0480b46a0137f9108" dependencies = [ "base64 0.22.1", + "evmap", "http-body-util", - "hyper 1.7.0", + "hyper 1.9.0", "hyper-rustls 0.27.7", "hyper-util", "indexmap 2.11.4", @@ -5089,6 +5145,7 @@ dependencies = [ "metrics", "metrics-util", "quanta", + "rustls 0.23.32", "thiserror 2.0.17", "tokio", "tracing", @@ -5100,11 +5157,15 @@ version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fe8db7a05415d0f919ffb905afa37784f71901c9a773188876984b4f769ab986" dependencies = [ + "aho-corasick", "crossbeam-epoch", "crossbeam-utils", "hashbrown 0.15.5", + "indexmap 2.11.4", "metrics", + "ordered-float 4.6.0", "quanta", + "radix_trie", "rand 0.9.2", "rand_xoshiro", "sketches-ddsketch", @@ -5618,6 +5679,15 @@ dependencies = [ "num-traits", ] +[[package]] +name = "ordered-float" +version = "4.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7bb71e1b3fa6ca1c61f383464aaf2bb0e2f8e772a1f01d486832464de363b951" +dependencies = [ + "num-traits", +] + [[package]] name = "outref" version = "0.5.2" @@ -6682,6 +6752,15 @@ dependencies = [ "rand_core 0.9.3", ] +[[package]] +name = "rapidhash" +version = "4.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5e48930979c155e2f33aa36ab3119b5ee81332beb6482199a8ecd6029b80b59" +dependencies = [ + "rustversion", +] + [[package]] name = "raw-cpuid" version = "11.6.0" @@ -6899,7 +6978,7 @@ dependencies = [ "http 1.3.1", "http-body 1.0.1", "http-body-util", - "hyper 1.7.0", + "hyper 1.9.0", "hyper-rustls 0.27.7", "hyper-tls 0.6.0", "hyper-util", @@ -7451,6 +7530,12 @@ dependencies = [ "syn 2.0.106", ] +[[package]] +name = "scoped-tls" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1cf6437eb19a8f4a6cc0f7dca544973b0b78843adbfeb3683d1a94a0024a294" + [[package]] name = "scopeguard" version = "1.2.0" @@ -7710,6 +7795,26 @@ dependencies = [ "unsafe-libyaml", ] +[[package]] +name = "service-kit" +version = "0.0.0" +dependencies = [ + "anyhow", + "axum", + "chrono", + "metrics", + "metrics-exporter-prometheus", + "metrics-util", + "proto-gazette", + "serde", + "serde_json", + "tokio", + "tower", + "tower-http", + "tracing", + "tracing-subscriber", +] + [[package]] name = "sha1" version = "0.10.6" @@ -8521,7 +8626,7 @@ checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" dependencies = [ "byteorder", "integer-encoding", - "ordered-float", + "ordered-float 2.10.1", ] [[package]] @@ -8801,7 +8906,7 @@ dependencies = [ "http 1.3.1", "http-body 1.0.1", "http-body-util", - "hyper 1.7.0", + "hyper 1.9.0", "hyper-timeout", "hyper-util", "percent-encoding", diff --git a/Cargo.toml b/Cargo.toml index 7fcfc282ac2..395880fc5d3 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -102,8 +102,9 @@ lz4 = "1" lz4_flex = "0" mime = "0" memchr = "2" -metrics = "0" -metrics-exporter-prometheus = "0" +metrics = "0.24" +metrics-exporter-prometheus = "0.18" +metrics-util = "0.20" prometheus = "0" md5 = "0" num-bigint = "0" @@ -234,7 +235,7 @@ aws-sdk-sts = "1" # Used for the agent http server axum = { version = "0", features = ["macros"] } -axum-server = { version = "0", features = ["tls-rustls"] } +axum-server = { version = "0.7.3", features = ["tls-rustls"] } axum-extra = { version = "0", features = ["typed-header", "query"] } async-graphql = { version = "7", features = [ "chrono", diff --git a/crates/service-kit/Cargo.toml b/crates/service-kit/Cargo.toml new file mode 100644 index 00000000000..7753a52642d --- /dev/null +++ b/crates/service-kit/Cargo.toml @@ -0,0 +1,28 @@ +[package] +name = "service-kit" +version.workspace = true +rust-version.workspace = true +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true + +[dependencies] +proto-gazette = { path = "../proto-gazette" } + +anyhow = { workspace = true } +axum = { workspace = true } +chrono = { workspace = true } +metrics-exporter-prometheus = { workspace = true } +metrics-util = { workspace = true } +serde = { workspace = true } +serde_json = { workspace = true } +tokio = { workspace = true } +tower-http = { workspace = true } +tracing = { workspace = true } +tracing-subscriber = { workspace = true } + +[dev-dependencies] +metrics = { workspace = true } +tower = { workspace = true } diff --git a/crates/service-kit/README.md b/crates/service-kit/README.md new file mode 100644 index 00000000000..dedc28b84ec --- /dev/null +++ b/crates/service-kit/README.md @@ -0,0 +1,70 @@ +# service-kit + +Building blocks for the operational surface of a long-running async service: +a loopback HTTP port exposing what the process is doing right now, with +controls to debug it without restarting. + +Service-agnostic. Used by Estuary reactors and runtime-next; nothing here +knows about Flow. + +## Surface + +A service constructs a [`Registry`], wires its tracing/event/metrics layers +in, and serves the admin router on a loopback port. Every spawned unit of +work registers a [`HandlerGuard`]; its lifecycle, identity, and recent events +become visible on the dashboard. + +| Module | Role | +| ------------------ | --------------------------------------------------------------------------------------- | +| [`handlers`] | `Registry` / `HandlerGuard` — in-flight handler inventory plus a recently-finished ring | +| [`admin`] | HTML dashboard, JSON views, per-handler drill-down, trace-override `POST` endpoint | +| [`trace`] | Per-handler `tracing` verbosity override — additive filter composed with the base | +| [`event`] | Opt-in per-handler event tracks (named ring buffers) + `event!` macro; lazy capture | +| [`metrics`] | Prometheus `/metrics` exporter folded into the admin router; histogram upkeep tick | + +## Entry points + +- `Registry::new()` / `Registry::register(kind)` — register a handler; run + its body inside `guard.span()` so tracing/event layers can find it. +- `admin::serve(name, registry, addr, shutdown)` — bind the loopback admin + port and serve until shutdown. +- `trace::layer_filter(base, registry)` — wrap a `tracing_subscriber` + `EnvFilter` so operator overrides bypass it. +- `event::layer(registry)` — install alongside `fmt` so `event!` calls + capture into per-handler tracks. +- `metrics::install_recorder()` — idempotently install the global + `metrics` recorder; called transitively by `admin::build_router`. + +## Non-obvious details + +- **No auth.** Bind admin on loopback only. +- **Trace-override is additive** — it raises verbosity for one handler but + never suppresses what the base filter would keep. Cost when no override + is set is one extra `enabled()` check per disabled callsite (atomic load, + short scope walk only inside a handler span). +- **Handler spans must always be created.** `OverrideFilter` short-circuits + to `true` for the `service_kit::handler` target — that's where override + state is hung. Don't filter the target out at the base. +- **`event!` capture is lazy.** A literal message stays `&'static str`; an + interpolated message stores `(template, captured args)` and is rendered + only when read. Values must be cheap-to-capture types (numbers, bools, + `&'static str`, `String`, `Arc`, `Clock`) or a `lazy`/`json`/`debug` + thunk — borrowed `&str` or `?x`/`%x` formatters won't compile; the call + site converts. See [`event`] for the full contract. +- **`event!` is a no-op outside a handler span** *and* under any subscriber + that isn't a `tracing_subscriber::registry::Registry`. The accompanying + `tracing` event still fires. +- **Prometheus histogram upkeep** isn't driven by scrapes; `install_recorder` + spawns its own tick because we own the HTTP surface (the upstream + `PrometheusBuilder::install` convenience constructor isn't used). Idle- + metric pruning *is* scrape-driven, so it only fires when a scraper polls. +- **`metrics` recorder install is process-global and panics on conflict.** + Service-kit owns that slot. + +[`Registry`]: src/handlers.rs +[`HandlerGuard`]: src/handlers.rs +[`handlers`]: src/handlers.rs +[`admin`]: src/admin.rs +[`trace`]: src/trace.rs +[`event`]: src/event.rs +[`metrics`]: src/metrics.rs diff --git a/crates/service-kit/src/admin.rs b/crates/service-kit/src/admin.rs new file mode 100644 index 00000000000..65e1e953712 --- /dev/null +++ b/crates/service-kit/src/admin.rs @@ -0,0 +1,569 @@ +//! The admin surface: a loopback HTTP endpoint presenting a service's +//! [`Registry`] as an auto-refreshing HTML dashboard, a per-handler drill-down +//! page (identity, phase, and recent [`crate::event`] tracks), plus JSON views. +//! The Prometheus scrape endpoint from [`crate::metrics`] is folded in too, so +//! `/metrics` is reachable on the same loopback port. + +use crate::{HandlerDetail, Registry, Snapshot}; +use std::fmt::Write; +use std::net::SocketAddr; + +#[derive(Clone)] +struct AdminState { + registry: Registry, + // Service name shown in the page title; the dashboard serves one service. + title: std::sync::Arc, +} + +/// Build the admin router for `service_name`. +/// Must be called from within a Tokio runtime. +pub fn build_router(service_name: impl Into, registry: Registry) -> axum::Router<()> { + use axum::routing::{get, post}; + + let state = AdminState { + registry, + title: service_name.into().into(), + }; + axum::Router::new() + .route("/", get(index)) + .route("/debug/handlers.json", get(handlers_json)) + .route("/debug/handlers/{id}", get(handler)) + .route("/debug/handlers/{id}/detail.json", get(handler_detail_json)) + // POST (not GET): this mutates process state, so it shouldn't be + // reachable by a link prefetch or an open-all-tabs. + .route("/debug/handlers/{id}/level/{level}", post(set_level)) + .with_state(state) + // `merge` (not `nest`): keep `/metrics` at the root of the admin port, + // matching what Prometheus scrapers expect by default. + .merge(crate::metrics::install_recorder()) + .layer(tower_http::trace::TraceLayer::new_for_http()) +} + +/// Bind `addr` and serve the admin surface until `shutdown` resolves. `addr` +/// should be loopback-only — there is no authentication on this surface. +pub async fn serve( + service_name: impl Into, + registry: Registry, + addr: SocketAddr, + shutdown: impl std::future::Future + Send + 'static, +) -> anyhow::Result<()> { + let service_name = service_name.into(); + let listener = tokio::net::TcpListener::bind(addr) + .await + .map_err(|err| anyhow::anyhow!("binding {service_name} admin surface on {addr}: {err}"))?; + tracing::info!(%addr, service = %service_name, "service-kit admin surface listening"); + + axum::serve(listener, build_router(service_name.clone(), registry)) + .with_graceful_shutdown(shutdown) + .await + .map_err(|err| anyhow::anyhow!("serving {service_name} admin surface: {err}")) +} + +async fn handlers_json( + axum::extract::State(state): axum::extract::State, +) -> axum::Json { + axum::Json(state.registry.snapshot()) +} + +async fn handler( + axum::extract::State(state): axum::extract::State, + axum::extract::Path(id): axum::extract::Path, + axum::extract::Query(params): axum::extract::Query>, +) -> axum::response::Result> { + let detail = state + .registry + .handler_detail(id) + .ok_or((axum::http::StatusCode::NOT_FOUND, "no such handler"))?; + // Drill-down time display: `?t=zulu` to flip from the default relative ages + // to absolute RFC-3339 timestamps; preserved across the page's 2s refresh. + let zulu = params.get("t").is_some_and(|v| v == "zulu"); + Ok(axum::response::Html(render_handler( + &state.title, + &detail, + zulu, + ))) +} + +async fn handler_detail_json( + axum::extract::State(state): axum::extract::State, + axum::extract::Path(id): axum::extract::Path, +) -> axum::response::Result> { + let detail = state + .registry + .handler_detail(id) + .ok_or((axum::http::StatusCode::NOT_FOUND, "no such handler"))?; + Ok(axum::Json(detail)) +} + +/// `POST` target of the dashboard's trace-level buttons; replies with a +/// see-other redirect back to the index (post/redirect/get). +async fn set_level( + axum::extract::State(state): axum::extract::State, + axum::extract::Path((id, level)): axum::extract::Path<(u64, String)>, +) -> axum::response::Result { + let level = + parse_level(&level).ok_or((axum::http::StatusCode::BAD_REQUEST, "unknown trace level"))?; + // A miss (handler already finished) is unremarkable — fall through to the + // refreshed index, which will no longer list it. + let _ = state.registry.set_trace_override(id, level); + Ok(axum::response::Redirect::to("/")) +} + +/// Parse a path segment into a trace-override level; `Some(None)` clears it. +fn parse_level(s: &str) -> Option> { + Some(match s { + "off" | "clear" | "none" => None, + "error" => Some(tracing::Level::ERROR), + "warn" => Some(tracing::Level::WARN), + "info" => Some(tracing::Level::INFO), + "debug" => Some(tracing::Level::DEBUG), + "trace" => Some(tracing::Level::TRACE), + _ => return None, + }) +} + +async fn index( + axum::extract::State(state): axum::extract::State, +) -> axum::response::Html { + axum::response::Html(render_index(&state.title, &state.registry.snapshot())) +} + +/// Inline page styling shared by [`render_index`] and [`render_handler`]. +const PAGE_STYLE: &str = "body{font:13px/1.4 ui-monospace,Menlo,Consolas,monospace;margin:1.5rem;color:#222}\ +h1,h2,h3{font-size:1rem;margin:1.2rem 0 .4rem}\ +table{border-collapse:collapse;width:100%;margin-bottom:.8rem}\ +th,td{text-align:left;padding:.25rem .6rem;border-bottom:1px solid #ddd;vertical-align:top}\ +th{background:#f3f3f3}\ +a{color:#06c;text-decoration:none}a:hover{text-decoration:underline}\ +form{display:inline}\ +button{font:inherit;color:#06c;background:none;border:0;padding:0;cursor:pointer}button:hover{text-decoration:underline}\ +.kind{color:#06c}.phase{font-weight:bold}.muted{color:#888}.on{font-weight:bold;color:#c30}"; + +/// Open an auto-refreshing HTML page with the shared style; `head_title` is the +/// already-escaped `` text. +fn page_open(out: &mut String, head_title: &str) { + let _ = write!( + out, + "<!doctype html><html><head><meta charset=\"utf-8\">\ + <meta http-equiv=\"refresh\" content=\"2\">\ + <title>{head_title}", + ); +} + +/// A link to a handler's drill-down page, wrapping the given (pre-escaped) text. +fn handler_link(id: u64, text: &str) -> String { + format!("{text}") +} + +fn render_index(title: &str, snapshot: &Snapshot) -> String { + let title = esc(title); + let mut out = String::new(); + page_open(&mut out, &format!("{title} handlers")); + + let _ = write!( + out, + "

{title} handlers

{} live, {} recently finished — auto-refreshes every 2s · json

", + snapshot.live.len(), + snapshot.recent.len(), + ); + + out.push_str("

Live

"); + if snapshot.live.is_empty() { + out.push_str("

(none)

"); + } else { + out.push_str( + "", + ); + for h in &snapshot.live { + let _ = write!( + out, + "", + handler_link(h.id, &h.id.to_string()), + handler_link(h.id, &esc(h.kind)), + esc(&h.label), + fmt_age(h.age_seconds), + esc(&h.phase), + fmt_age(h.phase_age_seconds), + fmt_trace_controls(h.id, h.trace_override), + fmt_fields(&h.fields), + ); + } + out.push_str("
idkindlabelagephasetracefields
{}{}{}{}{} ({}){}{}
"); + } + + out.push_str("

Recently finished

"); + if snapshot.recent.is_empty() { + out.push_str("

(none)

"); + } else { + out.push_str( + "", + ); + // Newest first. + for h in snapshot.recent.iter().rev() { + let _ = write!( + out, + "", + handler_link(h.id, &h.id.to_string()), + handler_link(h.id, &esc(h.kind)), + esc(&h.label), + fmt_age(h.age_seconds), + esc(&h.final_phase), + ); + } + out.push_str("
idkindlabelran forfinal phase
{}{}{}{}{}
"); + } + + out.push_str(""); + out +} + +fn render_handler(title: &str, h: &HandlerDetail, zulu: bool) -> String { + let title = esc(title); + let mut out = String::new(); + page_open(&mut out, &format!("{title} handler #{}", h.id)); + + let status = if h.finished { "finished" } else { "live" }; + // Toggle: the currently-active mode is bold, the alternative is a link + // back to the same handler with the opposite `t=` query. + let toggle = if zulu { + format!("age · zulu", h.id) + } else { + format!( + "age · zulu", + h.id, + ) + }; + let _ = write!( + out, + "

← {title} handlers · json · times: {toggle}

\ +

handler #{} ({status})

", + h.id, h.id, + ); + + out.push_str(""); + let _ = write!( + out, + "", + esc(h.kind) + ); + let _ = write!(out, "", esc(&h.label)); + + // Phase parenthetical: phase age (default) or absolute phase-change time + // (zulu). `phase_since_rfc3339` and `phase_age_seconds` are both `None` + // on a finished handler, in which case there is no parenthetical. + let phase_paren = match (zulu, &h.phase_since_rfc3339, h.phase_age_seconds) { + (true, Some(at), _) => { + format!(" (since {})", esc(at)) + } + (false, _, Some(s)) => format!(" ({})", fmt_age(s)), + _ => String::new(), + }; + let _ = write!( + out, + "", + esc(&h.phase), + ); + + // Age row. For a finished handler, `age_seconds` is the total runtime — + // a duration with no zulu equivalent, so it renders the same in both + // modes. For a live handler in zulu mode, show the absolute start time. + let (age_label, age_value) = if h.finished { + ("ran for", fmt_age(h.age_seconds)) + } else if zulu { + match &h.started_at_rfc3339 { + Some(at) => ("started", esc(at)), + None => ("age", fmt_age(h.age_seconds)), + } + } else { + ("age", fmt_age(h.age_seconds)) + }; + let _ = write!(out, ""); + + if !h.finished { + let _ = write!( + out, + "", + fmt_trace_controls(h.id, h.trace_override), + ); + } + if !h.fields.is_empty() { + let _ = write!( + out, + "", + fmt_fields(&h.fields) + ); + } + out.push_str("
kind{}
label{}
phase{}{phase_paren}
{age_label}{age_value}
trace{}
fields{}
"); + + out.push_str("

Tracks

"); + if h.tracks.is_empty() { + out.push_str("

(no events captured)

"); + } else { + let time_col = if zulu { "at" } else { "age" }; + for (name, events) in &h.tracks { + let _ = write!(out, "

{}

", esc(name)); + let _ = write!( + out, + "", + ); + // Oldest first: the operator reads top-to-bottom following the + // sequence of events that led to the handler's current state. + for e in events { + let when = if zulu { + esc(&e.at_rfc3339) + } else { + fmt_age(e.age_seconds) + }; + let _ = write!( + out, + "", + e.level, + esc(&e.message), + fmt_fields(&e.fields), + ); + } + out.push_str("
{time_col}levelmessagefields
{when}{}{}{}
"); + } + } + + out.push_str(""); + out +} + +/// The current override (highlighted if set) plus the buttons to change it — +/// tiny `POST` forms rather than links, since the target mutates process state. +fn fmt_trace_controls(id: u64, current: Option<&'static str>) -> String { + let mut out = String::new(); + match current { + Some(level) => { + let _ = write!(out, "{level} "); + } + None => out.push_str("base "), + } + for label in ["trace", "debug", "off"] { + let _ = write!( + out, + "
", + ); + } + out +} + +fn fmt_fields(fields: &[(&'static str, String)]) -> String { + fields + .iter() + .map(|(k, v)| format!("{}={}", esc(k), esc(v))) + .collect::>() + .join(" ") +} + +fn fmt_age(secs: u64) -> String { + if secs < 60 { + format!("{secs}s") + } else if secs < 3600 { + format!("{}m{}s", secs / 60, secs % 60) + } else { + format!("{}h{}m", secs / 3600, (secs % 3600) / 60) + } +} + +/// Minimal HTML-escaping for text interpolated into the dashboard. +fn esc(s: &str) -> String { + let mut out = String::with_capacity(s.len()); + for c in s.chars() { + match c { + '&' => out.push_str("&"), + '<' => out.push_str("<"), + '>' => out.push_str(">"), + '"' => out.push_str("""), + '\'' => out.push_str("'"), + _ => out.push(c), + } + } + out +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::event::EventView; + use crate::{FinishedView, HandlerDetail, HandlerView}; + + #[test] + fn render_index_includes_handlers_and_escapes() { + let snapshot = Snapshot { + live: vec![HandlerView { + id: 7, + kind: "leader.materialize", + label: "acmeCo/".to_string(), + phase: "running".to_string(), + age_seconds: 65, + phase_age_seconds: 3, + fields: vec![("shards", "2".to_string())], + trace_override: Some("TRACE"), + }], + recent: vec![FinishedView { + id: 6, + kind: "shuffle.log", + label: "dir/0".to_string(), + final_phase: "done".to_string(), + age_seconds: 4000, + tracks: Default::default(), + }], + }; + + let html = render_index("my-service", &snapshot); + assert!(html.contains("my-service handlers")); + assert!(html.contains("leader.materialize")); + assert!(html.contains("acmeCo/<svc>")); + assert!(!html.contains("acmeCo/")); + assert!(html.contains("shards=2")); + assert!(html.contains("1m5s")); + assert!(html.contains("1h6m")); + assert!(html.contains("shuffle.log")); + assert!(html.contains("7")); + assert!(html.contains("leader.materialize")); + assert!(html.contains("6")); + assert!(html.contains("shuffle.log")); + assert!(html.contains("running (3s)")); + // Trace controls for the live handler — `POST` forms, not links. + assert!(html.contains(">TRACE")); + assert!( + html.contains( + "