From daa62cc8323034b955f430008049cc188d02977c Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Wed, 12 Feb 2025 21:58:30 -0600 Subject: [PATCH] protocols: bundle a number of protocol extensions These have variously been discussed over the last couple weeks: * Add resource_path_pointers to materialization Response.Spec, to bring materializations toward parity with capture path pointers. * Add state_json to materialization Request.Apply, and state to Response.Applied. This allows a materialization connector to complete a post-commit apply under the last MaterializationSpec before applying schema changes which could break those staged files. * Add inactive_transforms to CollectionSpec.Derivation, and inactive_bindings to CaptureSpec and MaterializationSpec. We intend to retain previously-active Binding states in these fields, to allow connectors to ascertain whether a resource is brand new vs active in the past; to better initialize resources depending on historical activation context, and perhaps to clean up de-activated resources. * Improve various comments. None of these new fields are utilized yet, this is just protocol change groundwork. --- crates/agent/src/connector_tags.rs | 9 +- crates/dekaf/src/connector.rs | 1 + crates/flowctl/src/raw/materialize_fixture.rs | 1 + crates/proto-flow/src/capture.rs | 31 +- crates/proto-flow/src/flow.rs | 27 +- crates/proto-flow/src/flow.serde.rs | 54 ++ crates/proto-flow/src/materialize.rs | 41 +- crates/proto-flow/src/materialize.serde.rs | 53 ++ crates/proto-flow/tests/regression.rs | 6 + .../regression__materialize_request_json.snap | 3 +- ...regression__materialize_request_proto.snap | 165 ++--- ...regression__materialize_response_json.snap | 12 +- ...egression__materialize_response_proto.snap | 47 +- crates/runtime/src/capture/task.rs | 1 + crates/runtime/src/derive/task.rs | 1 + crates/runtime/src/harness/materialize.rs | 6 +- crates/runtime/src/materialize/protocol.rs | 4 + crates/runtime/src/materialize/task.rs | 1 + crates/validation/src/capture.rs | 1 + crates/validation/src/derivation.rs | 1 + crates/validation/src/materialization.rs | 1 + crates/validation/tests/common.rs | 3 + ...n_is_skipped_when_shards_are_disabled.snap | 2 + ..._tests__disabled_bindings_are_ignored.snap | 6 + .../scenario_tests__golden_all_visits.snap | 7 + ...zation_constraints_on_excluded_fields.snap | 1 + .../transition_tests__deletions.snap | 3 + .../snapshots/transition_tests__inserts.snap | 3 + ..._update_collection_becomes_derivation.snap | 2 + .../snapshots/transition_tests__updates.snap | 6 + go/protocols/capture/capture.pb.go | 31 +- go/protocols/capture/capture.proto | 32 +- go/protocols/derive/derive.pb.go | 164 ++--- go/protocols/derive/derive.proto | 3 + go/protocols/flow/flow.pb.go | 681 +++++++++++------- go/protocols/flow/flow.proto | 24 +- go/protocols/materialize/materialize.pb.go | 419 +++++++---- go/protocols/materialize/materialize.proto | 39 +- 38 files changed, 1237 insertions(+), 655 deletions(-) diff --git a/crates/agent/src/connector_tags.rs b/crates/agent/src/connector_tags.rs index 0e29d213f7..d5a72e205f 100644 --- a/crates/agent/src/connector_tags.rs +++ b/crates/agent/src/connector_tags.rs @@ -281,9 +281,10 @@ async fn spec_materialization( resource_config_schema_json, documentation_url, oauth2, + resource_path_pointers, } = spec; - let oauth = if let Some(oa) = oauth2 { + let oauth2 = if let Some(oa) = oauth2 { Some(serde_json::value::to_raw_value(&oa).expect("serializing oauth2 config")) } else { None @@ -294,10 +295,8 @@ async fn spec_materialization( .context("parsing endpoint config schema")?, resource_config_schema: RawValue::from_string(resource_config_schema_json) .context("parsing resource config schema")?, - - // materialization connectors don't currently specify resrouce_path_pointers, though perhaps they should - resource_path_pointers: Vec::new(), - oauth2: oauth, + resource_path_pointers, + oauth2, }) } diff --git a/crates/dekaf/src/connector.rs b/crates/dekaf/src/connector.rs index b6ce4d3b42..f3d6cd833a 100644 --- a/crates/dekaf/src/connector.rs +++ b/crates/dekaf/src/connector.rs @@ -88,6 +88,7 @@ pub async fn unary_materialize( "https://docs.estuary.dev/guides/dekaf_reading_collections_from_kafka" .to_string(), oauth2: None, + resource_path_pointers: vec!["/topic_name".to_string()], }), ..Default::default() }); diff --git a/crates/flowctl/src/raw/materialize_fixture.rs b/crates/flowctl/src/raw/materialize_fixture.rs index 16adfc54d3..8b28643638 100644 --- a/crates/flowctl/src/raw/materialize_fixture.rs +++ b/crates/flowctl/src/raw/materialize_fixture.rs @@ -57,6 +57,7 @@ pub async fn do_materialize_fixture( version: "test".to_string(), last_materialization: None, last_version: String::new(), + state_json: String::new(), }), ..Default::default() }); diff --git a/crates/proto-flow/src/capture.rs b/crates/proto-flow/src/capture.rs index 8a7246c363..d2bdb2fc77 100644 --- a/crates/proto-flow/src/capture.rs +++ b/crates/proto-flow/src/capture.rs @@ -102,11 +102,14 @@ pub mod request { pub backfill: u32, } } - /// Apply a capture configuration and bindings to its endpoint. - /// Apply is run out-of-band with ongoing connector invocations, - /// and may be run many times for a single capture name, - /// where each invocation has varying bindings, or even no bindings. - /// The connector performs any required setup or cleanup. + /// Apply an updated capture specification to its endpoint, + /// in preparation for an Open of a capture session. + /// Apply is run by the leader shard of a capture task + /// (having key_begin: 0) while the capture is quiescent. + /// Apply may be called multiple times for a given `version` and + /// `last_version`, even if a prior call succeeded from the connector's + /// perspective, so implementations must be idempotent. However, the next + /// session will not Open until it's preceding Apply has durably completed. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Apply { @@ -192,23 +195,21 @@ pub mod response { /// JSON schema of the connector's configuration. #[prost(string, tag = "2")] pub config_schema_json: ::prost::alloc::string::String, - /// JSON schema of the connecor's resource configuration. + /// JSON schema of the connector's resource configuration. #[prost(string, tag = "3")] pub resource_config_schema_json: ::prost::alloc::string::String, - /// URL for connector's documention. + /// URL for connector's documentation. #[prost(string, tag = "4")] pub documentation_url: ::prost::alloc::string::String, /// Optional OAuth2 configuration. #[prost(message, optional, tag = "5")] pub oauth2: ::core::option::Option, - /// One or more JSON pointers, which are used to extract the `resource_path` - /// from a given `resource` of this connector. For example, a database - /// capture connector might have a `resource` that's represented like: - /// `{"schema": "foo", "table": "bar", "otherConfig": true}`. In that case - /// it could use `resource_path_pointers: \["/schema", "/table"\]`, which - /// would result in a `resource_path` of `\["foo", "bar"\]`. This allows - /// `otherConfig` to be changed by the user without impacting the identity of - /// the resource. + /// One or more JSON pointers, which are used to extract resource paths + /// from resource configurations of this connector. For example, + /// a database connector might have a resource config like: + /// {"schema": "foo", "table": "bar", "other": "config", "answer": 42} + /// The connector would specify `resource_path_pointers: \["/schema", "/table"\]`, + /// which would result in a `resource_path` of `\["foo", "bar"\]`. #[prost(string, repeated, tag = "6")] pub resource_path_pointers: ::prost::alloc::vec::Vec< ::prost::alloc::string::String, diff --git a/crates/proto-flow/src/flow.rs b/crates/proto-flow/src/flow.rs index a124d7b24c..bd366410d7 100644 --- a/crates/proto-flow/src/flow.rs +++ b/crates/proto-flow/src/flow.rs @@ -300,6 +300,11 @@ pub mod collection_spec { /// Network ports of this derivation. #[prost(message, repeated, tag = "7")] pub network_ports: ::prost::alloc::vec::Vec, + /// Transforms which were previously active for this task, but are no longer. + /// Inactive transforms are unique by their transform name, and have no + /// overlap with active transforms. + #[prost(message, repeated, tag = "8")] + pub inactive_transforms: ::prost::alloc::vec::Vec, } /// Nested message and enum types in `Derivation`. pub mod derivation { @@ -309,7 +314,7 @@ pub mod collection_spec { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Transform { - /// Stable name of this transform. + /// Stable and unique name of this transform. #[prost(string, tag = "1")] pub name: ::prost::alloc::string::String, /// Source collection which is read by this transform. @@ -498,12 +503,16 @@ pub struct CaptureSpec { /// Network ports of this capture. #[prost(message, repeated, tag = "8")] pub network_ports: ::prost::alloc::vec::Vec, + /// Bindings which were previously active for this task, but are no longer. + /// Inactive bindings are unique by their resource path, and have no + /// overlap with active bindings. + #[prost(message, repeated, tag = "9")] + pub inactive_bindings: ::prost::alloc::vec::Vec, } /// Nested message and enum types in `CaptureSpec`. pub mod capture_spec { - /// Bindings of endpoint resources and collections into which they're captured. - /// Bindings are ordered and unique on the bound collection name, - /// and are also unique on the resource path. + /// Bindings between endpoint resources, uniquely identified by their + /// resource path, and the collections into which they're captured. /// /// Next tag: 6. #[allow(clippy::derive_partial_eq_without_eq)] @@ -594,12 +603,16 @@ pub struct MaterializationSpec { /// Network ports of this materialization. #[prost(message, repeated, tag = "7")] pub network_ports: ::prost::alloc::vec::Vec, + /// Bindings which were previously active for this task, but are no longer. + /// Inactive bindings are unique by their resource path, and have no + /// overlap with active bindings. + #[prost(message, repeated, tag = "9")] + pub inactive_bindings: ::prost::alloc::vec::Vec, } /// Nested message and enum types in `MaterializationSpec`. pub mod materialization_spec { - /// Bindings of endpoint resources and collections from which they're - /// materialized. Bindings are ordered and unique on the bound collection name, - /// and are also unique on the resource path. + /// Bindings between endpoint resources, uniquely identified by their + /// resource path, and the collections from which they're materialized. /// /// Next tag: 14. #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/crates/proto-flow/src/flow.serde.rs b/crates/proto-flow/src/flow.serde.rs index fb81db1aaf..eb7747e5c1 100644 --- a/crates/proto-flow/src/flow.serde.rs +++ b/crates/proto-flow/src/flow.serde.rs @@ -451,6 +451,9 @@ impl serde::Serialize for CaptureSpec { if !self.network_ports.is_empty() { len += 1; } + if !self.inactive_bindings.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("flow.CaptureSpec", len)?; if !self.name.is_empty() { struct_ser.serialize_field("name", &self.name)?; @@ -478,6 +481,9 @@ impl serde::Serialize for CaptureSpec { if !self.network_ports.is_empty() { struct_ser.serialize_field("networkPorts", &self.network_ports)?; } + if !self.inactive_bindings.is_empty() { + struct_ser.serialize_field("inactiveBindings", &self.inactive_bindings)?; + } struct_ser.end() } } @@ -502,6 +508,8 @@ impl<'de> serde::Deserialize<'de> for CaptureSpec { "recoveryLogTemplate", "network_ports", "networkPorts", + "inactive_bindings", + "inactiveBindings", ]; #[allow(clippy::enum_variant_names)] @@ -514,6 +522,7 @@ impl<'de> serde::Deserialize<'de> for CaptureSpec { ShardTemplate, RecoveryLogTemplate, NetworkPorts, + InactiveBindings, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -543,6 +552,7 @@ impl<'de> serde::Deserialize<'de> for CaptureSpec { "shardTemplate" | "shard_template" => Ok(GeneratedField::ShardTemplate), "recoveryLogTemplate" | "recovery_log_template" => Ok(GeneratedField::RecoveryLogTemplate), "networkPorts" | "network_ports" => Ok(GeneratedField::NetworkPorts), + "inactiveBindings" | "inactive_bindings" => Ok(GeneratedField::InactiveBindings), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -570,6 +580,7 @@ impl<'de> serde::Deserialize<'de> for CaptureSpec { let mut shard_template__ = None; let mut recovery_log_template__ = None; let mut network_ports__ = None; + let mut inactive_bindings__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { @@ -622,6 +633,12 @@ impl<'de> serde::Deserialize<'de> for CaptureSpec { } network_ports__ = Some(map_.next_value()?); } + GeneratedField::InactiveBindings => { + if inactive_bindings__.is_some() { + return Err(serde::de::Error::duplicate_field("inactiveBindings")); + } + inactive_bindings__ = Some(map_.next_value()?); + } } } Ok(CaptureSpec { @@ -633,6 +650,7 @@ impl<'de> serde::Deserialize<'de> for CaptureSpec { shard_template: shard_template__, recovery_log_template: recovery_log_template__, network_ports: network_ports__.unwrap_or_default(), + inactive_bindings: inactive_bindings__.unwrap_or_default(), }) } } @@ -1156,6 +1174,9 @@ impl serde::Serialize for collection_spec::Derivation { if !self.network_ports.is_empty() { len += 1; } + if !self.inactive_transforms.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("flow.CollectionSpec.Derivation", len)?; if self.connector_type != 0 { let v = collection_spec::derivation::ConnectorType::try_from(self.connector_type) @@ -1184,6 +1205,9 @@ impl serde::Serialize for collection_spec::Derivation { if !self.network_ports.is_empty() { struct_ser.serialize_field("networkPorts", &self.network_ports)?; } + if !self.inactive_transforms.is_empty() { + struct_ser.serialize_field("inactiveTransforms", &self.inactive_transforms)?; + } struct_ser.end() } } @@ -1207,6 +1231,8 @@ impl<'de> serde::Deserialize<'de> for collection_spec::Derivation { "recoveryLogTemplate", "network_ports", "networkPorts", + "inactive_transforms", + "inactiveTransforms", ]; #[allow(clippy::enum_variant_names)] @@ -1218,6 +1244,7 @@ impl<'de> serde::Deserialize<'de> for collection_spec::Derivation { ShardTemplate, RecoveryLogTemplate, NetworkPorts, + InactiveTransforms, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -1246,6 +1273,7 @@ impl<'de> serde::Deserialize<'de> for collection_spec::Derivation { "shardTemplate" | "shard_template" => Ok(GeneratedField::ShardTemplate), "recoveryLogTemplate" | "recovery_log_template" => Ok(GeneratedField::RecoveryLogTemplate), "networkPorts" | "network_ports" => Ok(GeneratedField::NetworkPorts), + "inactiveTransforms" | "inactive_transforms" => Ok(GeneratedField::InactiveTransforms), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -1272,6 +1300,7 @@ impl<'de> serde::Deserialize<'de> for collection_spec::Derivation { let mut shard_template__ = None; let mut recovery_log_template__ = None; let mut network_ports__ = None; + let mut inactive_transforms__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::ConnectorType => { @@ -1316,6 +1345,12 @@ impl<'de> serde::Deserialize<'de> for collection_spec::Derivation { } network_ports__ = Some(map_.next_value()?); } + GeneratedField::InactiveTransforms => { + if inactive_transforms__.is_some() { + return Err(serde::de::Error::duplicate_field("inactiveTransforms")); + } + inactive_transforms__ = Some(map_.next_value()?); + } } } Ok(collection_spec::Derivation { @@ -1326,6 +1361,7 @@ impl<'de> serde::Deserialize<'de> for collection_spec::Derivation { shard_template: shard_template__, recovery_log_template: recovery_log_template__, network_ports: network_ports__.unwrap_or_default(), + inactive_transforms: inactive_transforms__.unwrap_or_default(), }) } } @@ -3567,6 +3603,9 @@ impl serde::Serialize for MaterializationSpec { if !self.network_ports.is_empty() { len += 1; } + if !self.inactive_bindings.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("flow.MaterializationSpec", len)?; if !self.name.is_empty() { struct_ser.serialize_field("name", &self.name)?; @@ -3591,6 +3630,9 @@ impl serde::Serialize for MaterializationSpec { if !self.network_ports.is_empty() { struct_ser.serialize_field("networkPorts", &self.network_ports)?; } + if !self.inactive_bindings.is_empty() { + struct_ser.serialize_field("inactiveBindings", &self.inactive_bindings)?; + } struct_ser.end() } } @@ -3613,6 +3655,8 @@ impl<'de> serde::Deserialize<'de> for MaterializationSpec { "recoveryLogTemplate", "network_ports", "networkPorts", + "inactive_bindings", + "inactiveBindings", ]; #[allow(clippy::enum_variant_names)] @@ -3624,6 +3668,7 @@ impl<'de> serde::Deserialize<'de> for MaterializationSpec { ShardTemplate, RecoveryLogTemplate, NetworkPorts, + InactiveBindings, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -3652,6 +3697,7 @@ impl<'de> serde::Deserialize<'de> for MaterializationSpec { "shardTemplate" | "shard_template" => Ok(GeneratedField::ShardTemplate), "recoveryLogTemplate" | "recovery_log_template" => Ok(GeneratedField::RecoveryLogTemplate), "networkPorts" | "network_ports" => Ok(GeneratedField::NetworkPorts), + "inactiveBindings" | "inactive_bindings" => Ok(GeneratedField::InactiveBindings), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -3678,6 +3724,7 @@ impl<'de> serde::Deserialize<'de> for MaterializationSpec { let mut shard_template__ = None; let mut recovery_log_template__ = None; let mut network_ports__ = None; + let mut inactive_bindings__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { @@ -3722,6 +3769,12 @@ impl<'de> serde::Deserialize<'de> for MaterializationSpec { } network_ports__ = Some(map_.next_value()?); } + GeneratedField::InactiveBindings => { + if inactive_bindings__.is_some() { + return Err(serde::de::Error::duplicate_field("inactiveBindings")); + } + inactive_bindings__ = Some(map_.next_value()?); + } } } Ok(MaterializationSpec { @@ -3732,6 +3785,7 @@ impl<'de> serde::Deserialize<'de> for MaterializationSpec { shard_template: shard_template__, recovery_log_template: recovery_log_template__, network_ports: network_ports__.unwrap_or_default(), + inactive_bindings: inactive_bindings__.unwrap_or_default(), }) } } diff --git a/crates/proto-flow/src/materialize.rs b/crates/proto-flow/src/materialize.rs index 709479dc5b..bf793c70a8 100644 --- a/crates/proto-flow/src/materialize.rs +++ b/crates/proto-flow/src/materialize.rs @@ -103,11 +103,14 @@ pub mod request { pub backfill: u32, } } - /// Apply a materialization configuration and bindings to its endpoint. - /// Apply is run out-of-band with ongoing connector invocations, - /// and may be run many times for a single materialization name, - /// where each invocation has varying bindings, or even no bindings. - /// The connector performs any required setup or cleanup. + /// Apply an updated materialization specification to its endpoint, + /// in preparation for an Open of a materialization session. + /// Apply is run by the leader shard of a materialization task + /// (having key_begin: 0) while the materialization is quiescent. + /// Apply may be called multiple times for a given `version` and + /// `last_version`, even if a prior call succeeded from the connector's + /// perspective, so implementations must be idempotent. However, the next + /// session will not Open until it's preceding Apply has durably completed. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Apply { @@ -127,6 +130,11 @@ pub mod request { /// Version of the last applied MaterializationSpec. #[prost(string, tag = "5")] pub last_version: ::prost::alloc::string::String, + /// Last-persisted connector checkpoint state from a previous session. + /// The Apply RPC may use this state to perform a post-commit apply + /// of files staged under the `last_materialization` specification. + #[prost(string, tag = "6")] + pub state_json: ::prost::alloc::string::String, } /// Open a materialization stream. /// @@ -158,10 +166,10 @@ pub mod request { /// due to expected propagation races in Flow's distributed runtime. #[prost(string, tag = "2")] pub version: ::prost::alloc::string::String, - /// Range of documents to be processed by this invocation. + /// Range of documents to be processed by this session. #[prost(message, optional, tag = "3")] pub range: ::core::option::Option, - /// Last-persisted connector checkpoint state from a previous invocation. + /// Last-persisted connector checkpoint state from a previous session. #[prost(string, tag = "4")] pub state_json: ::prost::alloc::string::String, } @@ -270,15 +278,25 @@ pub mod response { /// JSON schema of the connector's configuration. #[prost(string, tag = "2")] pub config_schema_json: ::prost::alloc::string::String, - /// JSON schema of the connecor's resource configuration. + /// JSON schema of the connector's resource configuration. #[prost(string, tag = "3")] pub resource_config_schema_json: ::prost::alloc::string::String, - /// URL for connector's documention. + /// URL for connector's documentation. #[prost(string, tag = "4")] pub documentation_url: ::prost::alloc::string::String, /// Optional OAuth2 configuration. #[prost(message, optional, tag = "5")] pub oauth2: ::core::option::Option, + /// One or more JSON pointers, which are used to extract resource paths + /// from resource configurations of this connector. For example, + /// a database connector might have a resource config like: + /// {"schema": "foo", "table": "bar", "other": "config", "answer": 42} + /// The connector would specify `resource_path_pointers: \["/schema", "/table"\]`, + /// which would result in a `resource_path` of `\["foo", "bar"\]`. + #[prost(string, repeated, tag = "6")] + pub resource_path_pointers: ::prost::alloc::vec::Vec< + ::prost::alloc::string::String, + >, } /// Validated responds to Request.Validate. #[allow(clippy::derive_partial_eq_without_eq)] @@ -410,6 +428,11 @@ pub mod response { /// If empty, this Apply is to be considered a "no-op". #[prost(string, tag = "1")] pub action_description: ::prost::alloc::string::String, + /// Optional *transactional* update to ConnectorState. + /// This update commits atomically with the Flow recovery log checkpoint + /// which marks the current specification as having been applied. + #[prost(message, optional, tag = "2")] + pub state: ::core::option::Option, } /// Opened responds to Request.Open. /// After Opened, the connector sends only Loaded, Flushed, diff --git a/crates/proto-flow/src/materialize.serde.rs b/crates/proto-flow/src/materialize.serde.rs index 70c03bc801..1b78395114 100644 --- a/crates/proto-flow/src/materialize.serde.rs +++ b/crates/proto-flow/src/materialize.serde.rs @@ -630,6 +630,9 @@ impl serde::Serialize for request::Apply { if !self.last_version.is_empty() { len += 1; } + if !self.state_json.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("materialize.Request.Apply", len)?; if let Some(v) = self.materialization.as_ref() { struct_ser.serialize_field("materialization", v)?; @@ -643,6 +646,9 @@ impl serde::Serialize for request::Apply { if !self.last_version.is_empty() { struct_ser.serialize_field("lastVersion", &self.last_version)?; } + if !self.state_json.is_empty() { + struct_ser.serialize_field("state", crate::as_raw_json(&self.state_json)?)?; + } struct_ser.end() } } @@ -659,6 +665,8 @@ impl<'de> serde::Deserialize<'de> for request::Apply { "lastMaterialization", "last_version", "lastVersion", + "state_json", + "state", ]; #[allow(clippy::enum_variant_names)] @@ -667,6 +675,7 @@ impl<'de> serde::Deserialize<'de> for request::Apply { Version, LastMaterialization, LastVersion, + StateJson, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -692,6 +701,7 @@ impl<'de> serde::Deserialize<'de> for request::Apply { "version" => Ok(GeneratedField::Version), "lastMaterialization" | "last_materialization" => Ok(GeneratedField::LastMaterialization), "lastVersion" | "last_version" => Ok(GeneratedField::LastVersion), + "state" | "state_json" => Ok(GeneratedField::StateJson), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -715,6 +725,7 @@ impl<'de> serde::Deserialize<'de> for request::Apply { let mut version__ = None; let mut last_materialization__ = None; let mut last_version__ = None; + let mut state_json__ : Option> = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Materialization => { @@ -741,6 +752,12 @@ impl<'de> serde::Deserialize<'de> for request::Apply { } last_version__ = Some(map_.next_value()?); } + GeneratedField::StateJson => { + if state_json__.is_some() { + return Err(serde::de::Error::duplicate_field("state")); + } + state_json__ = Some(map_.next_value()?); + } } } Ok(request::Apply { @@ -748,6 +765,7 @@ impl<'de> serde::Deserialize<'de> for request::Apply { version: version__.unwrap_or_default(), last_materialization: last_materialization__, last_version: last_version__.unwrap_or_default(), + state_json: state_json__.map(|r| Box::::from(r).into()).unwrap_or_default(), }) } } @@ -2195,10 +2213,16 @@ impl serde::Serialize for response::Applied { if !self.action_description.is_empty() { len += 1; } + if self.state.is_some() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("materialize.Response.Applied", len)?; if !self.action_description.is_empty() { struct_ser.serialize_field("actionDescription", &self.action_description)?; } + if let Some(v) = self.state.as_ref() { + struct_ser.serialize_field("state", v)?; + } struct_ser.end() } } @@ -2211,11 +2235,13 @@ impl<'de> serde::Deserialize<'de> for response::Applied { const FIELDS: &[&str] = &[ "action_description", "actionDescription", + "state", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { ActionDescription, + State, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -2238,6 +2264,7 @@ impl<'de> serde::Deserialize<'de> for response::Applied { { match value { "actionDescription" | "action_description" => Ok(GeneratedField::ActionDescription), + "state" => Ok(GeneratedField::State), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -2258,6 +2285,7 @@ impl<'de> serde::Deserialize<'de> for response::Applied { V: serde::de::MapAccess<'de>, { let mut action_description__ = None; + let mut state__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::ActionDescription => { @@ -2266,10 +2294,17 @@ impl<'de> serde::Deserialize<'de> for response::Applied { } action_description__ = Some(map_.next_value()?); } + GeneratedField::State => { + if state__.is_some() { + return Err(serde::de::Error::duplicate_field("state")); + } + state__ = map_.next_value()?; + } } } Ok(response::Applied { action_description: action_description__.unwrap_or_default(), + state: state__, }) } } @@ -2593,6 +2628,9 @@ impl serde::Serialize for response::Spec { if self.oauth2.is_some() { len += 1; } + if !self.resource_path_pointers.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("materialize.Response.Spec", len)?; if self.protocol != 0 { struct_ser.serialize_field("protocol", &self.protocol)?; @@ -2609,6 +2647,9 @@ impl serde::Serialize for response::Spec { if let Some(v) = self.oauth2.as_ref() { struct_ser.serialize_field("oauth2", v)?; } + if !self.resource_path_pointers.is_empty() { + struct_ser.serialize_field("resourcePathPointers", &self.resource_path_pointers)?; + } struct_ser.end() } } @@ -2627,6 +2668,8 @@ impl<'de> serde::Deserialize<'de> for response::Spec { "documentation_url", "documentationUrl", "oauth2", + "resource_path_pointers", + "resourcePathPointers", ]; #[allow(clippy::enum_variant_names)] @@ -2636,6 +2679,7 @@ impl<'de> serde::Deserialize<'de> for response::Spec { ResourceConfigSchemaJson, DocumentationUrl, Oauth2, + ResourcePathPointers, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -2662,6 +2706,7 @@ impl<'de> serde::Deserialize<'de> for response::Spec { "resourceConfigSchema" | "resource_config_schema_json" => Ok(GeneratedField::ResourceConfigSchemaJson), "documentationUrl" | "documentation_url" => Ok(GeneratedField::DocumentationUrl), "oauth2" => Ok(GeneratedField::Oauth2), + "resourcePathPointers" | "resource_path_pointers" => Ok(GeneratedField::ResourcePathPointers), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -2686,6 +2731,7 @@ impl<'de> serde::Deserialize<'de> for response::Spec { let mut resource_config_schema_json__ : Option> = None; let mut documentation_url__ = None; let mut oauth2__ = None; + let mut resource_path_pointers__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Protocol => { @@ -2720,6 +2766,12 @@ impl<'de> serde::Deserialize<'de> for response::Spec { } oauth2__ = map_.next_value()?; } + GeneratedField::ResourcePathPointers => { + if resource_path_pointers__.is_some() { + return Err(serde::de::Error::duplicate_field("resourcePathPointers")); + } + resource_path_pointers__ = Some(map_.next_value()?); + } } } Ok(response::Spec { @@ -2728,6 +2780,7 @@ impl<'de> serde::Deserialize<'de> for response::Spec { resource_config_schema_json: resource_config_schema_json__.map(|r| Box::::from(r).into()).unwrap_or_default(), documentation_url: documentation_url__.unwrap_or_default(), oauth2: oauth2__, + resource_path_pointers: resource_path_pointers__.unwrap_or_default(), }) } } diff --git a/crates/proto-flow/tests/regression.rs b/crates/proto-flow/tests/regression.rs index 0028d010bb..ad4e356e17 100644 --- a/crates/proto-flow/tests/regression.rs +++ b/crates/proto-flow/tests/regression.rs @@ -203,6 +203,7 @@ fn ex_capture_spec() -> flow::CaptureSpec { state_key: "a%2Fcdc%2Ftable+baz.v3".to_string(), }], network_ports: ex_network_ports(), + inactive_bindings: Vec::new(), } } @@ -240,6 +241,7 @@ fn ex_derivation_spec() -> flow::CollectionSpec { flow::collection_spec::derivation::ShuffleType::Integer as i32, ], network_ports: ex_network_ports(), + inactive_transforms: Vec::new(), }); spec @@ -290,6 +292,7 @@ fn ex_materialization_spec() -> flow::MaterializationSpec { state_key: "some%20path.v1".to_string(), }], network_ports: ex_network_ports(), + inactive_bindings: Vec::new(), } } @@ -575,6 +578,7 @@ fn ex_materialize_request() -> materialize::Request { version: "11:22:33:44".to_string(), last_materialization: None, last_version: "00:11:22:33".to_string(), + state_json: json!({"connector":"state"}).to_string(), }), open: Some(materialize::request::Open { materialization: Some(ex_materialization_spec()), @@ -614,6 +618,7 @@ fn ex_materialize_response() -> materialize::Response { resource_config_schema_json: json!({"resource": "schema"}).to_string(), documentation_url: "https://example/docs".to_string(), oauth2: Some(ex_oauth2()), + resource_path_pointers: vec!["/schema".to_string(), "/table".to_string()], }), validated: Some(materialize::response::Validated { bindings: vec![materialize::response::validated::Binding { @@ -639,6 +644,7 @@ fn ex_materialize_response() -> materialize::Response { }), applied: Some(materialize::response::Applied { action_description: "I did some stuff".to_string(), + state: Some(ex_connector_state()), }), opened: Some(materialize::response::Opened { runtime_checkpoint: Some(ex_consumer_checkpoint()), diff --git a/crates/proto-flow/tests/snapshots/regression__materialize_request_json.snap b/crates/proto-flow/tests/snapshots/regression__materialize_request_json.snap index f8ea42b3da..9883d9d332 100644 --- a/crates/proto-flow/tests/snapshots/regression__materialize_request_json.snap +++ b/crates/proto-flow/tests/snapshots/regression__materialize_request_json.snap @@ -283,7 +283,8 @@ expression: json_test(msg) ] }, "version": "11:22:33:44", - "lastVersion": "00:11:22:33" + "lastVersion": "00:11:22:33", + "state": {"connector":"state"} }, "open": { "materialization": { diff --git a/crates/proto-flow/tests/snapshots/regression__materialize_request_proto.snap b/crates/proto-flow/tests/snapshots/regression__materialize_request_proto.snap index 9b7d539f2a..0bf451789f 100644 --- a/crates/proto-flow/tests/snapshots/regression__materialize_request_proto.snap +++ b/crates/proto-flow/tests/snapshots/regression__materialize_request_proto.snap @@ -39,7 +39,7 @@ expression: proto_test(msg) |64223a22 636f6e66 6967227d 1a130a0b| d":"config"}.... 00000220 |6f746865 722f6669 656c6412 0434322e| other/field..42. 00000230 |35200332 0b30303a 31313a32 323a3333| 5 .2.00:11:22:33 00000240 -|1afd070a e0070a16 61636d65 436f2f6d| ........acmeCo/m 00000250 +|1a94080a e0070a16 61636d65 436f2f6d| ........acmeCo/m 00000250 |61746572 69616c69 7a617469 6f6e1008| aterialization.. 00000260 |1a1d7b22 6d617465 7269616c 697a6522| ..{"materialize" 00000270 |3a7b2263 6f6e6669 67223a34 327d7d22| :{"config":42}}" 00000280 @@ -103,84 +103,85 @@ expression: proto_test(msg) |08ac023a 0c08903f 12056874 74707318| ...:...?..https. 00000620 |013a0308 a846120b 31313a32 323a3333| .:...F..11:22:33 00000630 |3a34342a 0b30303a 31313a32 323a3333| :44*.00:11:22:33 00000640 -|22a2080a e0070a16 61636d65 436f2f6d| ".......acmeCo/m 00000650 -|61746572 69616c69 7a617469 6f6e1008| aterialization.. 00000660 -|1a1d7b22 6d617465 7269616c 697a6522| ..{"materialize" 00000670 -|3a7b2263 6f6e6669 67223a34 327d7d22| :{"config":42}}" 00000680 -|d4050a15 7b227265 736f7572 6365223a| ....{"resource": 00000690 -|22636f6e 66696722 7d120473 6f6d6512| "config"}..some. 000006a0 -|04706174 681aa303 0a116163 6d65436f| .path.....acmeCo 000006b0 -|2f636f6c 6c656374 696f6e1a 082f6b65| /collection../ke 000006c0 -|792f6f6e 651a082f 6b65792f 74776f22| y/one../key/two" 000006d0 -|0b2f5f6d 6574612f 75756964 2a047479| ./_meta/uuid*.ty 000006e0 -|70652a06 72656769 6f6e3281 010a092f| pe*.region2..../ 000006f0 -|6a736f6e 2f707472 1207612d 6669656c| json/ptr..a-fiel 00000700 -|64200132 690a0769 6e746567 65720a06| d .2i..integer.. 00000710 -|73747269 6e671a13 1a037479 70220464| string....typ".d 00000720 -|61746530 b9603a03 656e6322 05746974| ate0.`:.enc".tit 00000730 -|6c652a04 64657363 320e7b22 64656622| le*.desc2.{"def" 00000740 -|3a226175 6c74227d 40014a0b 08011100| :"ault"}@.J..... 00000750 -|00000000 408fc052 15080a10 01181422| ....@..R......." 00000760 -|046e756c 6c220769 6e746567 65723a0e| .null".integer:. 00000770 -|7b226163 6b223a22 74727565 227d4212| {"ack":"true"}B. 00000780 -|7b227772 69746522 3a227363 68656d61| {"write":"schema 00000790 -|227d4aa3 010a1270 61727469 74696f6e| "}J....partition 000007a0 -|2f74656d 706c6174 6510031a 400a1e0a| /template...@... 000007b0 -|0f657374 75617279 2e646576 2f666f6f| .estuary.dev/foo 000007c0 -|120b6c61 62656c2d 76616c75 650a1e0a| ..label-value... 000007d0 -|0f657374 75617279 2e646576 2f626172| .estuary.dev/bar 000007e0 -|120b6f74 6865722d 76616c75 65224208| ..other-value"B. 000007f0 -|e9ec0610 031a1273 333a2f2f 6275636b| .......s3://buck 00000800 -|65742f70 72656669 78220308 ac023207| et/prefix"....2. 00000810 -|083e1080 e59a773a 18506174 687b7b50| .>....w:.Path{{P 00000820 -|6f737466 69782e54 656d706c 6174657d| ostfix.Template} 00000830 -|7d300438 cfb0f501 5a117b22 72656164| }0.8....Z.{"read 00000840 -|223a2273 6368656d 61227d22 550a076b| ":"schema"}"U..k 00000850 -|65792f6f 6e651207 76616c2f 74776f1a| ey/one..val/two. 00000860 -|0d666c6f 775f646f 63756d65 6e74221d| .flow_document". 00000870 -|0a07615f 6669656c 6412127b 22666965| ..a_field..{"fie 00000880 -|6c64223a 22636f6e 66696722 7d22130a| ld":"config"}".. 00000890 -|0b6f7468 65722f66 69656c64 12043432| .other/field..42 000008a0 -|2e353a5b 0a400a1e 0a0f6573 74756172| .5:[.@....estuar 000008b0 -|792e6465 762f666f 6f120b6c 6162656c| y.dev/foo..label 000008c0 -|2d76616c 75650a1e 0a0f6573 74756172| -value....estuar 000008d0 -|792e6465 762f6261 72120b6f 74686572| y.dev/bar..other 000008e0 -|2d76616c 75651217 0a150a08 6d792d6c| -value......my-l 000008f0 -|6162656c 12077072 65666978 2f180142| abel..prefix/..B 00000900 -|316d6174 65726961 6c697a65 2f61636d| 1materialize/acm 00000910 -|65436f2f 6d617465 7269616c 697a6174| eCo/materializat 00000920 -|696f6e2f 736f6d65 25323070 6174682e| ion/some%20path. 00000930 -|76314803 520608cb c8d6a606 5a060880| v1H.R.......Z... 00000940 -|888ba106 60016a0e 736f6d65 25323070| ....`.j.some%20p 00000950 -|6174682e 76312a80 010a0e73 68617264| ath.v1*....shard 00000960 -|2f74656d 706c6174 651a0f72 65636f76| /template..recov 00000970 -|6572792f 70726566 6978220b 68696e74| ery/prefix".hint 00000980 -|2f707265 66697828 03320208 3c480152| /prefix(.2......w:. 00000830 +|50617468 7b7b506f 73746669 782e5465| Path{{Postfix.Te 00000840 +|6d706c61 74657d7d 300438cf b0f5015a| mplate}}0.8....Z 00000850 +|117b2272 65616422 3a227363 68656d61| .{"read":"schema 00000860 +|227d2255 0a076b65 792f6f6e 65120776| "}"U..key/one..v 00000870 +|616c2f74 776f1a0d 666c6f77 5f646f63| al/two..flow_doc 00000880 +|756d656e 74221d0a 07615f66 69656c64| ument"...a_field 00000890 +|12127b22 6669656c 64223a22 636f6e66| ..{"field":"conf 000008a0 +|6967227d 22130a0b 6f746865 722f6669| ig"}"...other/fi 000008b0 +|656c6412 0434322e 353a5b0a 400a1e0a| eld..42.5:[.@... 000008c0 +|0f657374 75617279 2e646576 2f666f6f| .estuary.dev/foo 000008d0 +|120b6c61 62656c2d 76616c75 650a1e0a| ..label-value... 000008e0 +|0f657374 75617279 2e646576 2f626172| .estuary.dev/bar 000008f0 +|120b6f74 6865722d 76616c75 6512170a| ..other-value... 00000900 +|150a086d 792d6c61 62656c12 07707265| ...my-label..pre 00000910 +|6669782f 18014231 6d617465 7269616c| fix/..B1material 00000920 +|697a652f 61636d65 436f2f6d 61746572| ize/acmeCo/mater 00000930 +|69616c69 7a617469 6f6e2f73 6f6d6525| ialization/some% 00000940 +|32307061 74682e76 31480352 0608cbc8| 20path.v1H.R.... 00000950 +|d6a6065a 06088088 8ba10660 016a0e73| ...Z.......`.j.s 00000960 +|6f6d6525 32307061 74682e76 312a8001| ome%20path.v1*.. 00000970 +|0a0e7368 6172642f 74656d70 6c617465| ..shard/template 00000980 +|1a0f7265 636f7665 72792f70 72656669| ..recovery/prefi 00000990 +|78220b68 696e742f 70726566 69782803| x".hint/prefix(. 000009a0 +|3202083c 48015240 0a1e0a0f 65737475| 2.. { if output_apply { - print!("[\"applied.actionDescription\", {:?}]\n", applied.applied.as_ref().unwrap().action_description); + print!( + "[\"applied.actionDescription\", {:?}]\n", + applied.applied.as_ref().unwrap().action_description + ); } () = co.yield_(applied).await; } diff --git a/crates/runtime/src/materialize/protocol.rs b/crates/runtime/src/materialize/protocol.rs index 358b418402..3c8c3c5daa 100644 --- a/crates/runtime/src/materialize/protocol.rs +++ b/crates/runtime/src/materialize/protocol.rs @@ -47,6 +47,8 @@ pub async fn recv_client_unary( ); } + // TODO(johnny): load and attach Request.Apply.state_json. + apply.last_materialization = last_spec; } @@ -80,6 +82,8 @@ pub fn recv_connector_unary(request: Request, response: Response) -> anyhow::Res } else { verify("client", "unary request").fail(request) } + + // TODO(johnny): extract and apply Response.Apply.state to WriteBatch. } pub async fn recv_client_open(open: &mut Request, db: &RocksDB) -> anyhow::Result<()> { diff --git a/crates/runtime/src/materialize/task.rs b/crates/runtime/src/materialize/task.rs index a10e43021e..308262ae56 100644 --- a/crates/runtime/src/materialize/task.rs +++ b/crates/runtime/src/materialize/task.rs @@ -20,6 +20,7 @@ impl Task { network_ports: _, recovery_log_template: _, shard_template: _, + inactive_bindings: _, } = spec.as_ref().context("missing materialization")?; let range = range.context("missing range")?; diff --git a/crates/validation/src/capture.rs b/crates/validation/src/capture.rs index cb2a10be5b..b4f16d29a1 100644 --- a/crates/validation/src/capture.rs +++ b/crates/validation/src/capture.rs @@ -290,6 +290,7 @@ async fn walk_capture( recovery_log_template: Some(recovery_log_template), shard_template: Some(shard_template), network_ports, + inactive_bindings: Vec::new(), // TODO(johnny) }; let dependency_hash = dependencies.compute_hash(model); diff --git a/crates/validation/src/derivation.rs b/crates/validation/src/derivation.rs index 4942938dea..c72b9d2720 100644 --- a/crates/validation/src/derivation.rs +++ b/crates/validation/src/derivation.rs @@ -516,6 +516,7 @@ async fn walk_derivation( recovery_log_template: Some(recovery_log_template), shard_template: Some(shard_template), network_ports, + inactive_transforms: Vec::new(), }; Some((built_index, validated_response, built_spec, dependency_hash)) diff --git a/crates/validation/src/materialization.rs b/crates/validation/src/materialization.rs index d1ba0754a1..eb9f146d6e 100644 --- a/crates/validation/src/materialization.rs +++ b/crates/validation/src/materialization.rs @@ -369,6 +369,7 @@ async fn walk_materialization( recovery_log_template: Some(recovery_log_template), shard_template: Some(shard_template), network_ports, + inactive_bindings: Vec::new(), }; let dependency_hash = dependencies.compute_hash(model); diff --git a/crates/validation/tests/common.rs b/crates/validation/tests/common.rs index 391442e48e..fca787073f 100644 --- a/crates/validation/tests/common.rs +++ b/crates/validation/tests/common.rs @@ -162,6 +162,7 @@ pub fn run(fixture_yaml: &str, patch_yaml: &str) -> Outcome { bindings: Vec::new(), shard_template: Some(shard_template), config_json: String::new(), + inactive_bindings: Vec::new(), }; live.captures.insert_row( capture, @@ -217,6 +218,7 @@ pub fn run(fixture_yaml: &str, patch_yaml: &str) -> Outcome { shard_template: Some(shard_template), shuffle_key_types: Vec::new(), transforms: Vec::new(), + inactive_transforms: Vec::new(), }) } else { None @@ -272,6 +274,7 @@ pub fn run(fixture_yaml: &str, patch_yaml: &str) -> Outcome { bindings: Vec::new(), shard_template: Some(shard_template), config_json: String::new(), + inactive_bindings: Vec::new(), }; live.materializations.insert_row( materialization, diff --git a/crates/validation/tests/snapshots/scenario_tests__connector_validation_is_skipped_when_shards_are_disabled.snap b/crates/validation/tests/snapshots/scenario_tests__connector_validation_is_skipped_when_shards_are_disabled.snap index cb3febd153..ee23c139fd 100644 --- a/crates/validation/tests/snapshots/scenario_tests__connector_validation_is_skipped_when_shards_are_disabled.snap +++ b/crates/validation/tests/snapshots/scenario_tests__connector_validation_is_skipped_when_shards_are_disabled.snap @@ -378,6 +378,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, @@ -1053,6 +1054,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, diff --git a/crates/validation/tests/snapshots/scenario_tests__disabled_bindings_are_ignored.snap b/crates/validation/tests/snapshots/scenario_tests__disabled_bindings_are_ignored.snap index d0ae7831d2..1030085e64 100644 --- a/crates/validation/tests/snapshots/scenario_tests__disabled_bindings_are_ignored.snap +++ b/crates/validation/tests/snapshots/scenario_tests__disabled_bindings_are_ignored.snap @@ -156,6 +156,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, @@ -507,6 +508,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, @@ -1050,6 +1052,7 @@ Outcome { }, ), network_ports: [], + inactive_transforms: [], }, ), }, @@ -1614,6 +1617,7 @@ Outcome { }, ), network_ports: [], + inactive_transforms: [], }, ), }, @@ -1772,6 +1776,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, @@ -2169,6 +2174,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, diff --git a/crates/validation/tests/snapshots/scenario_tests__golden_all_visits.snap b/crates/validation/tests/snapshots/scenario_tests__golden_all_visits.snap index 7f63218be6..92bdb10852 100644 --- a/crates/validation/tests/snapshots/scenario_tests__golden_all_visits.snap +++ b/crates/validation/tests/snapshots/scenario_tests__golden_all_visits.snap @@ -473,6 +473,7 @@ Outcome { public: false, }, ], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, @@ -1193,6 +1194,7 @@ Outcome { public: false, }, ], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, @@ -2378,6 +2380,7 @@ Outcome { }, ), network_ports: [], + inactive_transforms: [], }, ), }, @@ -3923,6 +3926,7 @@ Outcome { }, ), network_ports: [], + inactive_transforms: [], }, ), }, @@ -4620,6 +4624,7 @@ Outcome { }, ), network_ports: [], + inactive_transforms: [], }, ), }, @@ -6367,6 +6372,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, @@ -7426,6 +7432,7 @@ Outcome { public: false, }, ], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, diff --git a/crates/validation/tests/snapshots/scenario_tests__materialization_constraints_on_excluded_fields.snap b/crates/validation/tests/snapshots/scenario_tests__materialization_constraints_on_excluded_fields.snap index 2237fe007b..d69d666548 100644 --- a/crates/validation/tests/snapshots/scenario_tests__materialization_constraints_on_excluded_fields.snap +++ b/crates/validation/tests/snapshots/scenario_tests__materialization_constraints_on_excluded_fields.snap @@ -725,6 +725,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, diff --git a/crates/validation/tests/snapshots/transition_tests__deletions.snap b/crates/validation/tests/snapshots/transition_tests__deletions.snap index 77fd8eb4ca..72882629c3 100644 --- a/crates/validation/tests/snapshots/transition_tests__deletions.snap +++ b/crates/validation/tests/snapshots/transition_tests__deletions.snap @@ -49,6 +49,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, is_touch: 0, dependency_hash: NULL, @@ -159,6 +160,7 @@ Outcome { }, ), network_ports: [], + inactive_transforms: [], }, ), }, @@ -211,6 +213,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, is_touch: 0, dependency_hash: NULL, diff --git a/crates/validation/tests/snapshots/transition_tests__inserts.snap b/crates/validation/tests/snapshots/transition_tests__inserts.snap index b9d1860dfb..38276f06d7 100644 --- a/crates/validation/tests/snapshots/transition_tests__inserts.snap +++ b/crates/validation/tests/snapshots/transition_tests__inserts.snap @@ -398,6 +398,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, @@ -1289,6 +1290,7 @@ Outcome { }, ), network_ports: [], + inactive_transforms: [], }, ), }, @@ -1747,6 +1749,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: NULL, is_touch: 0, diff --git a/crates/validation/tests/snapshots/transition_tests__update_collection_becomes_derivation.snap b/crates/validation/tests/snapshots/transition_tests__update_collection_becomes_derivation.snap index 3c2e5c25e5..850b3aaa8f 100644 --- a/crates/validation/tests/snapshots/transition_tests__update_collection_becomes_derivation.snap +++ b/crates/validation/tests/snapshots/transition_tests__update_collection_becomes_derivation.snap @@ -81,6 +81,7 @@ Outcome { }, ), network_ports: [], + inactive_transforms: [], }, ), }, @@ -520,6 +521,7 @@ Outcome { }, ), network_ports: [], + inactive_transforms: [], }, ), }, diff --git a/crates/validation/tests/snapshots/transition_tests__updates.snap b/crates/validation/tests/snapshots/transition_tests__updates.snap index 0809f66772..65e3831805 100644 --- a/crates/validation/tests/snapshots/transition_tests__updates.snap +++ b/crates/validation/tests/snapshots/transition_tests__updates.snap @@ -479,6 +479,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: CaptureSpec { name: "the/capture", @@ -515,6 +516,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, is_touch: 0, dependency_hash: 5b38dc32c776e2b2, @@ -1595,6 +1597,7 @@ Outcome { }, ), network_ports: [], + inactive_transforms: [], }, ), }, @@ -1655,6 +1658,7 @@ Outcome { }, ), network_ports: [], + inactive_transforms: [], }, ), }, @@ -2193,6 +2197,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, previous_spec: MaterializationSpec { name: "the/materialization", @@ -2228,6 +2233,7 @@ Outcome { }, ), network_ports: [], + inactive_bindings: [], }, is_touch: 0, dependency_hash: 5b38dc32c776e2b2, diff --git a/go/protocols/capture/capture.pb.go b/go/protocols/capture/capture.pb.go index 5b16fc0d78..3d982b9a27 100644 --- a/go/protocols/capture/capture.pb.go +++ b/go/protocols/capture/capture.pb.go @@ -272,11 +272,14 @@ func (m *Request_Validate_Binding) XXX_DiscardUnknown() { var xxx_messageInfo_Request_Validate_Binding proto.InternalMessageInfo -// Apply a capture configuration and bindings to its endpoint. -// Apply is run out-of-band with ongoing connector invocations, -// and may be run many times for a single capture name, -// where each invocation has varying bindings, or even no bindings. -// The connector performs any required setup or cleanup. +// Apply an updated capture specification to its endpoint, +// in preparation for an Open of a capture session. +// Apply is run by the leader shard of a capture task +// (having key_begin: 0) while the capture is quiescent. +// Apply may be called multiple times for a given `version` and +// `last_version`, even if a prior call succeeded from the connector's +// perspective, so implementations must be idempotent. However, the next +// session will not Open until it's preceding Apply has durably completed. type Request_Apply struct { // Capture to be applied. Capture *flow.CaptureSpec `protobuf:"bytes,1,opt,name=capture,proto3" json:"capture,omitempty"` @@ -478,20 +481,18 @@ type Response_Spec struct { Protocol uint32 `protobuf:"varint,1,opt,name=protocol,proto3" json:"protocol,omitempty"` // JSON schema of the connector's configuration. ConfigSchemaJson encoding_json.RawMessage `protobuf:"bytes,2,opt,name=config_schema_json,json=configSchema,proto3,casttype=encoding/json.RawMessage" json:"config_schema_json,omitempty"` - // JSON schema of the connecor's resource configuration. + // JSON schema of the connector's resource configuration. ResourceConfigSchemaJson encoding_json.RawMessage `protobuf:"bytes,3,opt,name=resource_config_schema_json,json=resourceConfigSchema,proto3,casttype=encoding/json.RawMessage" json:"resource_config_schema_json,omitempty"` - // URL for connector's documention. + // URL for connector's documentation. DocumentationUrl string `protobuf:"bytes,4,opt,name=documentation_url,json=documentationUrl,proto3" json:"documentation_url,omitempty"` // Optional OAuth2 configuration. Oauth2 *flow.OAuth2 `protobuf:"bytes,5,opt,name=oauth2,proto3" json:"oauth2,omitempty"` - // One or more JSON pointers, which are used to extract the `resource_path` - // from a given `resource` of this connector. For example, a database - // capture connector might have a `resource` that's represented like: - // `{"schema": "foo", "table": "bar", "otherConfig": true}`. In that case - // it could use `resource_path_pointers: ["/schema", "/table"]`, which - // would result in a `resource_path` of `["foo", "bar"]`. This allows - // `otherConfig` to be changed by the user without impacting the identity of - // the resource. + // One or more JSON pointers, which are used to extract resource paths + // from resource configurations of this connector. For example, + // a database connector might have a resource config like: + // {"schema": "foo", "table": "bar", "other": "config", "answer": 42} + // The connector would specify `resource_path_pointers: ["/schema", "/table"]`, + // which would result in a `resource_path` of `["foo", "bar"]`. ResourcePathPointers []string `protobuf:"bytes,6,rep,name=resource_path_pointers,json=resourcePathPointers,proto3" json:"resource_path_pointers,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` diff --git a/go/protocols/capture/capture.proto b/go/protocols/capture/capture.proto index 81109e28ed..e207a1cf53 100644 --- a/go/protocols/capture/capture.proto +++ b/go/protocols/capture/capture.proto @@ -127,11 +127,14 @@ message Request { } Validate validate = 3; - // Apply a capture configuration and bindings to its endpoint. - // Apply is run out-of-band with ongoing connector invocations, - // and may be run many times for a single capture name, - // where each invocation has varying bindings, or even no bindings. - // The connector performs any required setup or cleanup. + // Apply an updated capture specification to its endpoint, + // in preparation for an Open of a capture session. + // Apply is run by the leader shard of a capture task + // (having key_begin: 0) while the capture is quiescent. + // Apply may be called multiple times for a given `version` and + // `last_version`, even if a prior call succeeded from the connector's + // perspective, so implementations must be idempotent. However, the next + // session will not Open until it's preceding Apply has durably completed. message Apply { // Capture to be applied. flow.CaptureSpec capture = 1; @@ -191,24 +194,21 @@ message Response { (gogoproto.casttype) = "encoding/json.RawMessage", json_name = "configSchema" ]; - // JSON schema of the connecor's resource configuration. + // JSON schema of the connector's resource configuration. string resource_config_schema_json = 3 [ (gogoproto.casttype) = "encoding/json.RawMessage", json_name = "resourceConfigSchema" ]; - // URL for connector's documention. + // URL for connector's documentation. string documentation_url = 4; // Optional OAuth2 configuration. flow.OAuth2 oauth2 = 5; - - // One or more JSON pointers, which are used to extract the `resource_path` - // from a given `resource` of this connector. For example, a database - // capture connector might have a `resource` that's represented like: - // `{"schema": "foo", "table": "bar", "otherConfig": true}`. In that case - // it could use `resource_path_pointers: ["/schema", "/table"]`, which - // would result in a `resource_path` of `["foo", "bar"]`. This allows - // `otherConfig` to be changed by the user without impacting the identity of - // the resource. + // One or more JSON pointers, which are used to extract resource paths + // from resource configurations of this connector. For example, + // a database connector might have a resource config like: + // {"schema": "foo", "table": "bar", "other": "config", "answer": 42} + // The connector would specify `resource_path_pointers: ["/schema", "/table"]`, + // which would result in a `resource_path` of `["foo", "bar"]`. repeated string resource_path_pointers = 6; } Spec spec = 1; diff --git a/go/protocols/derive/derive.pb.go b/go/protocols/derive/derive.pb.go index b94a11fec9..40de6de883 100644 --- a/go/protocols/derive/derive.pb.go +++ b/go/protocols/derive/derive.pb.go @@ -906,88 +906,88 @@ func init() { func init() { proto.RegisterFile("go/protocols/derive/derive.proto", fileDescriptor_4410d076c75e1e4f) } var fileDescriptor_4410d076c75e1e4f = []byte{ - // 1282 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x57, 0xdd, 0x6e, 0x1b, 0xc5, - 0x17, 0xef, 0x3a, 0xfe, 0xda, 0x13, 0xc7, 0x69, 0xa7, 0xf9, 0xff, 0x59, 0x6d, 0xa2, 0x34, 0x94, - 0x22, 0x0c, 0x11, 0x0e, 0x72, 0x2b, 0x28, 0xa5, 0x45, 0x34, 0x6e, 0x83, 0x2a, 0xa8, 0x52, 0x4d, - 0x9a, 0x56, 0xe2, 0xc6, 0x9a, 0xec, 0x8e, 0xed, 0x6d, 0xd6, 0x3b, 0xcb, 0xcc, 0x6c, 0x2a, 0xbf, - 0x48, 0xcb, 0x25, 0xe2, 0x21, 0x90, 0x78, 0x83, 0x5e, 0xf2, 0x04, 0x95, 0x28, 0xd7, 0xbc, 0x00, - 0x57, 0x68, 0x3e, 0x76, 0x6d, 0x27, 0x4e, 0x71, 0xef, 0xb8, 0x49, 0xe6, 0x9c, 0xf3, 0x3b, 0x67, - 0x66, 0xcf, 0x39, 0x73, 0x7e, 0x63, 0xd8, 0x1a, 0xb0, 0x9d, 0x94, 0x33, 0xc9, 0x02, 0x16, 0x8b, - 0x9d, 0x90, 0xf2, 0xe8, 0x84, 0xda, 0x7f, 0x6d, 0xad, 0x47, 0x55, 0x23, 0xf9, 0x5b, 0x01, 0x4b, - 0x44, 0x36, 0xa2, 0xbc, 0xc0, 0x17, 0x0b, 0x83, 0xf4, 0x37, 0x66, 0x62, 0xf5, 0x63, 0xf6, 0x5c, - 0xff, 0xb1, 0xd6, 0xb5, 0x01, 0x1b, 0x30, 0xbd, 0xdc, 0x51, 0x2b, 0xa3, 0xbd, 0xfa, 0xd7, 0x2a, - 0xd4, 0x30, 0xfd, 0x31, 0xa3, 0x42, 0xa2, 0x16, 0x94, 0x45, 0x4a, 0x03, 0xcf, 0xd9, 0x72, 0x5a, - 0xcb, 0x9d, 0xb5, 0xb6, 0x3d, 0x86, 0x35, 0xb7, 0x0f, 0x52, 0x1a, 0x60, 0x8d, 0x40, 0x37, 0xa0, - 0x7e, 0x42, 0xe2, 0x28, 0x24, 0x92, 0x7a, 0x25, 0x8d, 0xf6, 0x4e, 0xa3, 0x9f, 0x58, 0x3b, 0x2e, - 0x90, 0x2a, 0x3e, 0x4b, 0x69, 0xe2, 0x2d, 0xcd, 0x8f, 0xbf, 0x9f, 0xd2, 0x04, 0x6b, 0x84, 0x42, - 0x72, 0x4a, 0x42, 0xaf, 0x3c, 0x1f, 0x89, 0x29, 0x09, 0xb1, 0x46, 0xa0, 0x6d, 0xa8, 0xf4, 0xe3, - 0x4c, 0x0c, 0xbd, 0x8a, 0x86, 0xfe, 0xef, 0x34, 0x74, 0x4f, 0x19, 0xb1, 0xc1, 0xa0, 0xaf, 0xa1, - 0x21, 0x24, 0xe1, 0xb2, 0x17, 0xb0, 0xd1, 0x28, 0x92, 0x5e, 0x55, 0xfb, 0xac, 0x9f, 0xf9, 0x50, - 0x85, 0xe9, 0x6a, 0x08, 0x5e, 0x16, 0x13, 0x41, 0x6d, 0xc6, 0xa9, 0xa0, 0xd2, 0xab, 0xcd, 0xdf, - 0x0c, 0x2b, 0x23, 0x36, 0x18, 0xb4, 0x0e, 0xf5, 0x28, 0x91, 0x94, 0x27, 0x24, 0xf6, 0xc2, 0x2d, - 0xa7, 0xd5, 0xc0, 0xee, 0xb5, 0x5c, 0xe1, 0xbf, 0x74, 0xa0, 0xac, 0xf2, 0x89, 0x1e, 0x43, 0x33, - 0x60, 0x49, 0x42, 0x03, 0xc9, 0x78, 0x4f, 0x8e, 0x53, 0xaa, 0xb3, 0xdf, 0xec, 0x7c, 0xda, 0xd6, - 0xa5, 0xeb, 0xb2, 0x38, 0xa6, 0x81, 0x8c, 0x58, 0xa2, 0xd0, 0xed, 0x7b, 0x6a, 0x3f, 0xa2, 0xc4, - 0x76, 0x37, 0xf7, 0x7a, 0x3c, 0x4e, 0x29, 0x5e, 0x09, 0xa6, 0x45, 0xf4, 0x25, 0x2c, 0x07, 0x2c, - 0xe9, 0x47, 0x83, 0xde, 0x33, 0xc1, 0x12, 0x5d, 0x22, 0x77, 0x77, 0xe3, 0xef, 0xd7, 0x57, 0x3c, - 0x9a, 0x04, 0x2c, 0x8c, 0x92, 0xc1, 0x8e, 0x32, 0xb4, 0x31, 0x79, 0xfe, 0x90, 0x0a, 0x41, 0x06, - 0x14, 0x57, 0x8d, 0x83, 0xff, 0x53, 0x0d, 0xea, 0x79, 0xed, 0xfe, 0x73, 0xa7, 0x43, 0xb7, 0x00, - 0x82, 0x62, 0xd3, 0xa2, 0x91, 0xe6, 0x1c, 0x66, 0xb7, 0xfc, 0xea, 0xf5, 0x95, 0x0b, 0x78, 0x0a, - 0x8d, 0x76, 0x01, 0x24, 0x27, 0x89, 0xe8, 0x33, 0x3e, 0x12, 0x5e, 0x79, 0x6b, 0xa9, 0xb5, 0xdc, - 0xb9, 0x7a, 0x5e, 0xdb, 0xb6, 0x1f, 0xe7, 0x50, 0x3c, 0xe5, 0x85, 0x9e, 0xc2, 0x25, 0x31, 0xcc, - 0xfa, 0xfd, 0x98, 0xf6, 0x8e, 0xe9, 0x58, 0xa7, 0x44, 0x78, 0x95, 0xad, 0xa5, 0x56, 0xb3, 0xb3, - 0xfd, 0x6f, 0x39, 0x39, 0x30, 0x8e, 0x3a, 0x23, 0xab, 0x36, 0xca, 0x77, 0x74, 0xac, 0x64, 0x81, - 0xde, 0x87, 0x46, 0xca, 0xd9, 0x33, 0x1a, 0xc8, 0x1e, 0x67, 0xcc, 0xb4, 0xa6, 0x8b, 0x97, 0xad, - 0x0e, 0x33, 0x26, 0xd1, 0x1e, 0x40, 0x34, 0x4a, 0x19, 0x97, 0xbd, 0x11, 0x49, 0xbd, 0x9a, 0x3e, - 0xff, 0x47, 0xe7, 0x9e, 0xff, 0x81, 0x86, 0x3e, 0x24, 0xe9, 0xfd, 0x44, 0xf2, 0x31, 0x76, 0xa3, - 0x5c, 0x46, 0x77, 0x60, 0x35, 0x26, 0x42, 0x5d, 0x82, 0x22, 0x91, 0xf5, 0xf3, 0x13, 0x89, 0x9b, - 0x0a, 0x3c, 0xd1, 0xa9, 0x93, 0x6a, 0xf7, 0x13, 0xca, 0x85, 0xf2, 0x75, 0xcd, 0x49, 0x95, 0xee, - 0x89, 0x51, 0xf9, 0x2f, 0x4a, 0xe0, 0x16, 0xf9, 0x43, 0x08, 0xca, 0x09, 0x19, 0x99, 0xd6, 0x71, - 0xb1, 0x5e, 0x9f, 0xaa, 0x63, 0xe9, 0x9d, 0xea, 0x78, 0x08, 0x7e, 0x5e, 0x83, 0x98, 0x8c, 0x8e, - 0x42, 0xd2, 0x9b, 0xee, 0xa6, 0xa5, 0x05, 0xba, 0xe9, 0xb2, 0xf5, 0xff, 0x5e, 0xbb, 0x77, 0x4d, - 0x6b, 0xed, 0x01, 0x9a, 0x13, 0xae, 0xbc, 0x40, 0xb8, 0x46, 0x3c, 0x1d, 0xc7, 0x87, 0xfa, 0x11, - 0x09, 0x8e, 0xfb, 0x51, 0x1c, 0xeb, 0xa1, 0xb4, 0x82, 0x0b, 0xd9, 0xbf, 0x0d, 0xcd, 0xd9, 0xba, - 0xa0, 0x8b, 0xb0, 0x74, 0x4c, 0xc7, 0x36, 0x37, 0x6a, 0x89, 0xd6, 0xa0, 0x72, 0x42, 0xe2, 0xcc, - 0x0c, 0x56, 0x17, 0x1b, 0xe1, 0x56, 0xe9, 0xa6, 0xe3, 0xff, 0xea, 0x40, 0x59, 0x0d, 0x49, 0x74, - 0x63, 0x26, 0x7b, 0xce, 0x5b, 0x8a, 0x37, 0x9d, 0x37, 0x0f, 0x6a, 0x79, 0xcd, 0x4c, 0xe8, 0x5c, - 0x44, 0x1f, 0x42, 0x85, 0x93, 0x64, 0x40, 0xed, 0x85, 0x5a, 0x35, 0xa1, 0xb0, 0x52, 0xe9, 0x28, - 0xc6, 0x8a, 0xbe, 0x00, 0x10, 0x92, 0x48, 0x6a, 0x32, 0x53, 0x59, 0x20, 0x33, 0x15, 0x8d, 0xf7, - 0x7f, 0x29, 0x41, 0x59, 0xcd, 0x6c, 0xb4, 0x01, 0x6e, 0x71, 0x99, 0xf4, 0xb9, 0x57, 0xf0, 0x44, - 0x81, 0x3e, 0x80, 0x72, 0x96, 0x45, 0xa1, 0x6d, 0x07, 0x7b, 0x8a, 0xc3, 0xc3, 0x07, 0xf7, 0x1e, - 0x11, 0x2e, 0x05, 0xd6, 0x46, 0xf4, 0x39, 0xd4, 0x6c, 0xf5, 0xec, 0x69, 0x37, 0xe6, 0xb1, 0x43, - 0x7e, 0xd9, 0x70, 0x0e, 0x46, 0xd7, 0xa1, 0x1e, 0xb2, 0x60, 0xf1, 0xa2, 0x2e, 0x85, 0x2c, 0xf0, - 0x9f, 0x41, 0xed, 0x60, 0xe2, 0xaf, 0x6e, 0xbc, 0xf6, 0x77, 0x16, 0xf1, 0x57, 0xb5, 0xfc, 0x3f, - 0x54, 0x53, 0x12, 0x1c, 0x53, 0xf3, 0x4d, 0x0d, 0x6c, 0x25, 0x75, 0x25, 0x86, 0x44, 0x0c, 0xf5, - 0x17, 0xac, 0x60, 0xbd, 0xf6, 0x6b, 0x50, 0xd1, 0x64, 0xe5, 0x63, 0x58, 0x9e, 0x62, 0x20, 0xd4, - 0x05, 0xc4, 0xb3, 0x44, 0x46, 0x23, 0xda, 0x0b, 0x86, 0x34, 0x38, 0x4e, 0x59, 0x94, 0xc8, 0xa2, - 0xe8, 0xf9, 0xa3, 0xa0, 0xdd, 0x2d, 0x6c, 0xf8, 0x92, 0xc5, 0x4f, 0x54, 0x2a, 0xb8, 0x26, 0xa7, - 0xab, 0xbf, 0xd5, 0xa1, 0x8e, 0xa9, 0x48, 0x59, 0x22, 0x28, 0xfa, 0x78, 0x86, 0xf0, 0xa7, 0xe8, - 0xcc, 0xd8, 0xa7, 0x19, 0xff, 0x26, 0xb8, 0x39, 0x8f, 0xe7, 0x05, 0xf2, 0xcf, 0xe0, 0xf3, 0xe1, - 0x13, 0xe2, 0x09, 0x18, 0xed, 0x40, 0x55, 0x71, 0x3a, 0x0d, 0x6d, 0xbd, 0xde, 0x3b, 0xe3, 0xb6, - 0xaf, 0xcd, 0xd8, 0xc2, 0xd4, 0x56, 0x69, 0x76, 0x14, 0x47, 0x62, 0x48, 0xf3, 0x17, 0xc0, 0xd9, - 0xad, 0x1e, 0xe5, 0x08, 0x3c, 0x01, 0xa3, 0x0e, 0xd4, 0x34, 0xd1, 0xd3, 0xd0, 0x3e, 0x07, 0xbc, - 0x33, 0x7e, 0x7b, 0xc6, 0x8e, 0x73, 0x20, 0xba, 0x0f, 0x4d, 0x4d, 0xf1, 0x34, 0x9c, 0x7d, 0x15, - 0x6c, 0x9e, 0xcd, 0x86, 0x81, 0xd9, 0x87, 0xc1, 0x8a, 0x98, 0x16, 0xdf, 0xce, 0xf6, 0x2f, 0x4a, - 0x96, 0xed, 0x7d, 0xa8, 0xe7, 0x0f, 0x34, 0xdb, 0xfe, 0x85, 0xac, 0xe6, 0x8f, 0x1d, 0x3c, 0x22, - 0x18, 0xd2, 0x11, 0x59, 0x9c, 0x1c, 0x1b, 0xc6, 0xef, 0x40, 0xbb, 0xa1, 0xa7, 0xb0, 0xce, 0xa9, - 0x60, 0x19, 0x0f, 0x68, 0x6f, 0x4e, 0xc0, 0x45, 0xe6, 0xe3, 0x5a, 0x1e, 0xa0, 0x3b, 0x1d, 0x78, - 0x1b, 0x2e, 0x85, 0x2c, 0xc8, 0x46, 0x34, 0x91, 0x9a, 0xd0, 0x7a, 0x19, 0x8f, 0xcd, 0x55, 0xc2, - 0x17, 0x67, 0x0c, 0x87, 0x3c, 0x46, 0xd7, 0xa0, 0xca, 0x48, 0x26, 0x87, 0x1d, 0x5b, 0x89, 0x86, - 0xb9, 0xcd, 0xfb, 0x77, 0x95, 0x0e, 0x5b, 0x9b, 0xff, 0xb2, 0x04, 0x6e, 0xd1, 0x34, 0xe8, 0xde, - 0x0c, 0x41, 0x3b, 0x9a, 0xe0, 0xae, 0x9d, 0xdf, 0x64, 0xe7, 0x52, 0xf4, 0xea, 0x80, 0x26, 0x94, - 0x2b, 0x48, 0xaf, 0x1f, 0xc5, 0x54, 0x78, 0x25, 0x1d, 0xaa, 0xfd, 0x96, 0x50, 0xdf, 0xe6, 0x1e, - 0x7b, 0xca, 0xc1, 0x50, 0x66, 0x73, 0x30, 0xa3, 0xf4, 0x5b, 0xd3, 0xa4, 0xb6, 0x0e, 0xae, 0x7a, - 0x7f, 0xf6, 0x58, 0x12, 0x9b, 0xe9, 0x5d, 0xc7, 0x75, 0xa5, 0xd8, 0x4f, 0xe2, 0xb1, 0x7f, 0x17, - 0x2e, 0xcf, 0x09, 0xf8, 0x4e, 0xb3, 0xbe, 0x0e, 0x55, 0x73, 0x2d, 0xfc, 0x6f, 0xc0, 0x2d, 0x9a, - 0x7d, 0x66, 0x8a, 0x39, 0x8b, 0x4e, 0x31, 0x17, 0x6a, 0xb6, 0xed, 0xfd, 0xaf, 0x60, 0x65, 0xa6, - 0x8d, 0xd1, 0x27, 0x60, 0x66, 0xf4, 0x69, 0x16, 0xb1, 0xef, 0xb5, 0x03, 0x65, 0xb3, 0x63, 0xbc, - 0x73, 0x1b, 0xdc, 0xc2, 0xa0, 0xae, 0xb5, 0x7e, 0xdb, 0x50, 0xb4, 0x7a, 0x6a, 0x00, 0xfb, 0x17, - 0x4f, 0x27, 0xba, 0xe5, 0x7c, 0xe6, 0xec, 0xde, 0x79, 0xf5, 0xc7, 0xe6, 0x85, 0x57, 0x6f, 0x36, - 0x9d, 0xdf, 0xdf, 0x6c, 0x3a, 0x3f, 0xff, 0xb9, 0xe9, 0xfc, 0xb0, 0x3d, 0x88, 0xe4, 0x30, 0x3b, - 0x6a, 0x07, 0x6c, 0xb4, 0x43, 0x85, 0xcc, 0x08, 0x1f, 0x9b, 0x5f, 0x2d, 0x73, 0x7e, 0x13, 0x1d, - 0x55, 0xb5, 0xe6, 0xfa, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x72, 0xa9, 0x92, 0x85, 0x31, 0x0d, - 0x00, 0x00, + // 1287 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x57, 0x4f, 0x6f, 0x1b, 0x45, + 0x14, 0xef, 0x3a, 0xfe, 0xb3, 0xfb, 0xe2, 0x38, 0xe9, 0x34, 0xc0, 0x6a, 0x13, 0xa5, 0xa1, 0x14, + 0x61, 0x88, 0x70, 0x90, 0x5b, 0x41, 0x29, 0x2d, 0xa2, 0x71, 0x1b, 0x54, 0xa0, 0x4a, 0x35, 0x69, + 0x5a, 0x89, 0x8b, 0x35, 0xd9, 0x1d, 0xdb, 0xdb, 0xac, 0x77, 0x96, 0x9d, 0xd9, 0x54, 0xfe, 0x22, + 0x94, 0x03, 0x07, 0xc4, 0x87, 0xe0, 0xca, 0xb5, 0x47, 0x3e, 0x41, 0x25, 0xca, 0x99, 0x2f, 0xc0, + 0x09, 0xcd, 0x9f, 0x5d, 0xdb, 0xb1, 0x53, 0xdc, 0x1b, 0x97, 0x64, 0xde, 0x7b, 0xbf, 0xf7, 0x66, + 0xf6, 0xbd, 0x37, 0xef, 0x37, 0x86, 0xed, 0x3e, 0xdb, 0x4d, 0x52, 0x26, 0x98, 0xcf, 0x22, 0xbe, + 0x1b, 0xd0, 0x34, 0x3c, 0xa5, 0xe6, 0x5f, 0x4b, 0xe9, 0x51, 0x55, 0x4b, 0xde, 0xb6, 0xcf, 0x62, + 0x9e, 0x0d, 0x69, 0x5a, 0xe0, 0x8b, 0x85, 0x46, 0x7a, 0x9b, 0x53, 0xb1, 0x7a, 0x11, 0x7b, 0xa6, + 0xfe, 0x18, 0xeb, 0x7a, 0x9f, 0xf5, 0x99, 0x5a, 0xee, 0xca, 0x95, 0xd6, 0x5e, 0xf9, 0x7b, 0x15, + 0x6a, 0x98, 0xfe, 0x90, 0x51, 0x2e, 0x50, 0x13, 0xca, 0x3c, 0xa1, 0xbe, 0x6b, 0x6d, 0x5b, 0xcd, + 0xe5, 0xf6, 0x7a, 0xcb, 0x1c, 0xc3, 0x98, 0x5b, 0x87, 0x09, 0xf5, 0xb1, 0x42, 0xa0, 0xeb, 0x60, + 0x9f, 0x92, 0x28, 0x0c, 0x88, 0xa0, 0x6e, 0x49, 0xa1, 0xdd, 0xb3, 0xe8, 0xc7, 0xc6, 0x8e, 0x0b, + 0xa4, 0x8c, 0xcf, 0x12, 0x1a, 0xbb, 0x4b, 0xf3, 0xe3, 0x1f, 0x24, 0x34, 0xc6, 0x0a, 0x21, 0x91, + 0x29, 0x25, 0x81, 0x5b, 0x9e, 0x8f, 0xc4, 0x94, 0x04, 0x58, 0x21, 0xd0, 0x0e, 0x54, 0x7a, 0x51, + 0xc6, 0x07, 0x6e, 0x45, 0x41, 0xdf, 0x3a, 0x0b, 0xdd, 0x97, 0x46, 0xac, 0x31, 0xe8, 0x4b, 0xa8, + 0x73, 0x41, 0x52, 0xd1, 0xf5, 0xd9, 0x70, 0x18, 0x0a, 0xb7, 0xaa, 0x7c, 0x36, 0x66, 0x3e, 0x54, + 0x62, 0x3a, 0x0a, 0x82, 0x97, 0xf9, 0x58, 0x90, 0x9b, 0xa5, 0x94, 0x53, 0xe1, 0xd6, 0xe6, 0x6f, + 0x86, 0xa5, 0x11, 0x6b, 0x0c, 0xda, 0x00, 0x3b, 0x8c, 0x05, 0x4d, 0x63, 0x12, 0xb9, 0xc1, 0xb6, + 0xd5, 0xac, 0x63, 0xe7, 0x6a, 0xae, 0xf0, 0x9e, 0x5b, 0x50, 0x96, 0xf9, 0x44, 0x8f, 0xa0, 0xe1, + 0xb3, 0x38, 0xa6, 0xbe, 0x60, 0x69, 0x57, 0x8c, 0x12, 0xaa, 0xb2, 0xdf, 0x68, 0x7f, 0xdc, 0x52, + 0xa5, 0xeb, 0xb0, 0x28, 0xa2, 0xbe, 0x08, 0x59, 0x2c, 0xd1, 0xad, 0xbb, 0x72, 0x3f, 0x22, 0xc5, + 0x56, 0x27, 0xf7, 0x7a, 0x34, 0x4a, 0x28, 0x5e, 0xf1, 0x27, 0x45, 0xf4, 0x39, 0x2c, 0xfb, 0x2c, + 0xee, 0x85, 0xfd, 0xee, 0x53, 0xce, 0x62, 0x55, 0x22, 0x67, 0x6f, 0xf3, 0x9f, 0x97, 0x97, 0x5d, + 0x1a, 0xfb, 0x2c, 0x08, 0xe3, 0xfe, 0xae, 0x34, 0xb4, 0x30, 0x79, 0xf6, 0x80, 0x72, 0x4e, 0xfa, + 0x14, 0x57, 0xb5, 0x83, 0xf7, 0x53, 0x0d, 0xec, 0xbc, 0x76, 0xff, 0xbb, 0xd3, 0xa1, 0x9b, 0x00, + 0x7e, 0xb1, 0x69, 0xd1, 0x48, 0x73, 0x0e, 0xb3, 0x57, 0x7e, 0xf1, 0xf2, 0xf2, 0x05, 0x3c, 0x81, + 0x46, 0x7b, 0x00, 0x22, 0x25, 0x31, 0xef, 0xb1, 0x74, 0xc8, 0xdd, 0xf2, 0xf6, 0x52, 0x73, 0xb9, + 0x7d, 0xe5, 0xbc, 0xb6, 0x6d, 0x3d, 0xca, 0xa1, 0x78, 0xc2, 0x0b, 0x3d, 0x81, 0x8b, 0x7c, 0x90, + 0xf5, 0x7a, 0x11, 0xed, 0x9e, 0xd0, 0x91, 0x4a, 0x09, 0x77, 0x2b, 0xdb, 0x4b, 0xcd, 0x46, 0x7b, + 0xe7, 0xbf, 0x72, 0x72, 0xa8, 0x1d, 0x55, 0x46, 0x56, 0x4d, 0x94, 0x6f, 0xe9, 0x48, 0xca, 0x1c, + 0xbd, 0x0b, 0xf5, 0x24, 0x65, 0x4f, 0xa9, 0x2f, 0xba, 0x29, 0x63, 0xba, 0x35, 0x1d, 0xbc, 0x6c, + 0x74, 0x98, 0x31, 0x81, 0xf6, 0x01, 0xc2, 0x61, 0xc2, 0x52, 0xd1, 0x1d, 0x92, 0xc4, 0xad, 0xa9, + 0xf3, 0x7f, 0x70, 0xee, 0xf9, 0xef, 0x2b, 0xe8, 0x03, 0x92, 0xdc, 0x8b, 0x45, 0x3a, 0xc2, 0x4e, + 0x98, 0xcb, 0xe8, 0x36, 0xac, 0x46, 0x84, 0xcb, 0x4b, 0x50, 0x24, 0xd2, 0x3e, 0x3f, 0x91, 0xb8, + 0x21, 0xc1, 0x63, 0x9d, 0x3c, 0xa9, 0x72, 0x3f, 0xa5, 0x29, 0x97, 0xbe, 0x8e, 0x3e, 0xa9, 0xd4, + 0x3d, 0xd6, 0x2a, 0xef, 0xc7, 0x12, 0x38, 0x45, 0xfe, 0x10, 0x82, 0x72, 0x4c, 0x86, 0xba, 0x75, + 0x1c, 0xac, 0xd6, 0x67, 0xea, 0x58, 0x7a, 0xa3, 0x3a, 0x1e, 0x81, 0x97, 0xd7, 0x20, 0x22, 0xc3, + 0xe3, 0x80, 0x74, 0x27, 0xbb, 0x69, 0x69, 0x81, 0x6e, 0xba, 0x64, 0xfc, 0xbf, 0x53, 0xee, 0x1d, + 0xdd, 0x5a, 0xfb, 0x80, 0xe6, 0x84, 0x2b, 0x2f, 0x10, 0xae, 0x1e, 0x4d, 0xc6, 0xf1, 0xc0, 0x3e, + 0x26, 0xfe, 0x49, 0x2f, 0x8c, 0x22, 0x35, 0x94, 0x56, 0x70, 0x21, 0x7b, 0xb7, 0xa0, 0x31, 0x5d, + 0x17, 0xb4, 0x06, 0x4b, 0x27, 0x74, 0x64, 0x72, 0x23, 0x97, 0x68, 0x1d, 0x2a, 0xa7, 0x24, 0xca, + 0xf4, 0x60, 0x75, 0xb0, 0x16, 0x6e, 0x96, 0x6e, 0x58, 0xde, 0x6f, 0x16, 0x94, 0xe5, 0x90, 0x44, + 0xd7, 0xa7, 0xb2, 0x67, 0xbd, 0xa6, 0x78, 0x93, 0x79, 0x73, 0xa1, 0x96, 0xd7, 0x4c, 0x87, 0xce, + 0x45, 0xf4, 0x3e, 0x54, 0x52, 0x12, 0xf7, 0xa9, 0xb9, 0x50, 0xab, 0x3a, 0x14, 0x96, 0x2a, 0x15, + 0x45, 0x5b, 0xd1, 0x67, 0x00, 0x5c, 0x10, 0x41, 0x75, 0x66, 0x2a, 0x0b, 0x64, 0xa6, 0xa2, 0xf0, + 0xde, 0xaf, 0x25, 0x28, 0xcb, 0x99, 0x8d, 0x36, 0xc1, 0x29, 0x2e, 0x93, 0x3a, 0xf7, 0x0a, 0x1e, + 0x2b, 0xd0, 0x7b, 0x50, 0xce, 0xb2, 0x30, 0x30, 0xed, 0x60, 0x4e, 0x71, 0x74, 0x74, 0xff, 0xee, + 0x43, 0x92, 0x0a, 0x8e, 0x95, 0x11, 0x7d, 0x0a, 0x35, 0x53, 0x3d, 0x73, 0xda, 0xcd, 0x79, 0xec, + 0x90, 0x5f, 0x36, 0x9c, 0x83, 0xd1, 0x35, 0xb0, 0x03, 0xe6, 0x2f, 0x5e, 0xd4, 0xa5, 0x80, 0xf9, + 0xde, 0x53, 0xa8, 0x1d, 0x8e, 0xfd, 0xe5, 0x8d, 0x57, 0xfe, 0xd6, 0x22, 0xfe, 0xb2, 0x96, 0x6f, + 0x43, 0x35, 0x21, 0xfe, 0x09, 0xd5, 0xdf, 0x54, 0xc7, 0x46, 0x92, 0x57, 0x62, 0x40, 0xf8, 0x40, + 0x7d, 0xc1, 0x0a, 0x56, 0x6b, 0xaf, 0x06, 0x15, 0x45, 0x56, 0x1e, 0x86, 0xe5, 0x09, 0x06, 0x42, + 0x1d, 0x40, 0x69, 0x16, 0x8b, 0x70, 0x48, 0xbb, 0xfe, 0x80, 0xfa, 0x27, 0x09, 0x0b, 0x63, 0x51, + 0x14, 0x3d, 0x7f, 0x14, 0xb4, 0x3a, 0x85, 0x0d, 0x5f, 0x34, 0xf8, 0xb1, 0x4a, 0x06, 0x57, 0xe4, + 0x74, 0xe5, 0x77, 0x1b, 0x6c, 0x4c, 0x79, 0xc2, 0x62, 0x4e, 0xd1, 0x87, 0x53, 0x84, 0x3f, 0x41, + 0x67, 0xda, 0x3e, 0xc9, 0xf8, 0x37, 0xc0, 0xc9, 0x79, 0x3c, 0x2f, 0x90, 0x37, 0x83, 0xcf, 0x87, + 0x4f, 0x80, 0xc7, 0x60, 0xb4, 0x0b, 0x55, 0xc9, 0xe9, 0x34, 0x30, 0xf5, 0x7a, 0x67, 0xc6, 0xed, + 0x40, 0x99, 0xb1, 0x81, 0xc9, 0xad, 0x92, 0xec, 0x38, 0x0a, 0xf9, 0x80, 0xe6, 0x2f, 0x80, 0xd9, + 0xad, 0x1e, 0xe6, 0x08, 0x3c, 0x06, 0xa3, 0x36, 0xd4, 0x14, 0xd1, 0xd3, 0xc0, 0x3c, 0x07, 0xdc, + 0x19, 0xbf, 0x7d, 0x6d, 0xc7, 0x39, 0x10, 0xdd, 0x83, 0x86, 0xa2, 0x78, 0x1a, 0x4c, 0xbf, 0x0a, + 0xb6, 0x66, 0xb3, 0xa1, 0x61, 0xe6, 0x61, 0xb0, 0xc2, 0x27, 0xc5, 0xd7, 0xb3, 0xfd, 0xcf, 0x25, + 0xc3, 0xf6, 0x1e, 0xd8, 0xf9, 0x03, 0xcd, 0xb4, 0x7f, 0x21, 0xcb, 0xf9, 0x63, 0x06, 0x0f, 0xf7, + 0x07, 0x74, 0x48, 0x16, 0x27, 0xc7, 0xba, 0xf6, 0x3b, 0x54, 0x6e, 0xe8, 0x09, 0x6c, 0xa4, 0x94, + 0xb3, 0x2c, 0xf5, 0x69, 0x77, 0x4e, 0xc0, 0x45, 0xe6, 0xe3, 0x7a, 0x1e, 0xa0, 0x33, 0x19, 0x78, + 0x07, 0x2e, 0x06, 0xcc, 0xcf, 0x86, 0x34, 0x16, 0x8a, 0xd0, 0xba, 0x59, 0x1a, 0xe9, 0xab, 0x84, + 0xd7, 0xa6, 0x0c, 0x47, 0x69, 0x84, 0xae, 0x42, 0x95, 0x91, 0x4c, 0x0c, 0xda, 0xa6, 0x12, 0x75, + 0x7d, 0x9b, 0x0f, 0xee, 0x48, 0x1d, 0x36, 0xb6, 0x6f, 0xca, 0x76, 0x75, 0xad, 0xe6, 0x3d, 0x2f, + 0x81, 0x53, 0xb4, 0x0e, 0xba, 0x3b, 0x45, 0xd3, 0x96, 0xa2, 0xb9, 0xab, 0xe7, 0xb7, 0xda, 0xb9, + 0x44, 0xbd, 0xda, 0xa7, 0x31, 0x4d, 0x25, 0xa4, 0xdb, 0x0b, 0x23, 0xca, 0xdd, 0x92, 0x0a, 0xd5, + 0x7a, 0x4d, 0xa8, 0xaf, 0x73, 0x8f, 0x7d, 0xe9, 0xa0, 0x89, 0xb3, 0xd1, 0x9f, 0x52, 0x7a, 0xcd, + 0x49, 0x6a, 0xdb, 0x00, 0x47, 0xbe, 0x42, 0xbb, 0x2c, 0x8e, 0xf4, 0x0c, 0xb7, 0xb1, 0x2d, 0x15, + 0x07, 0x71, 0x34, 0xf2, 0xee, 0xc0, 0xa5, 0x39, 0x01, 0xdf, 0x68, 0xe2, 0xdb, 0x50, 0xd5, 0x97, + 0xc3, 0xfb, 0x0a, 0x9c, 0xa2, 0xe5, 0xa7, 0x66, 0x99, 0xb5, 0xe8, 0x2c, 0x73, 0xa0, 0x66, 0x9a, + 0xdf, 0xfb, 0x02, 0x56, 0xa6, 0x9a, 0x19, 0x7d, 0x04, 0x7a, 0x52, 0x9f, 0xe5, 0x12, 0xf3, 0x6a, + 0x3b, 0x94, 0x36, 0x33, 0xcc, 0xdb, 0xb7, 0xc0, 0x29, 0x0c, 0xf2, 0x72, 0xab, 0x17, 0x0e, 0x45, + 0xab, 0x67, 0xc6, 0xb0, 0xb7, 0x76, 0x36, 0xd1, 0x4d, 0xeb, 0x13, 0x6b, 0xef, 0xf6, 0x8b, 0x3f, + 0xb7, 0x2e, 0xbc, 0x78, 0xb5, 0x65, 0xfd, 0xf1, 0x6a, 0xcb, 0xfa, 0xe5, 0xaf, 0x2d, 0xeb, 0xfb, + 0x9d, 0x7e, 0x28, 0x06, 0xd9, 0x71, 0xcb, 0x67, 0xc3, 0x5d, 0xca, 0x45, 0x46, 0xd2, 0x91, 0xfe, + 0xed, 0x32, 0xe7, 0x97, 0xd1, 0x71, 0x55, 0x69, 0xae, 0xfd, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x72, + 0x38, 0xa4, 0x72, 0x37, 0x0d, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/go/protocols/derive/derive.proto b/go/protocols/derive/derive.proto index d110fa2453..4f133c8ea4 100644 --- a/go/protocols/derive/derive.proto +++ b/go/protocols/derive/derive.proto @@ -186,6 +186,9 @@ message Response { string documentation_url = 4; // Optional OAuth2 configuration. flow.OAuth2 oauth2 = 5; + // Derivations don't use resource_path_pointers, + // as transforms are uniquely identified by their name. + reserved 6; } Spec spec = 1; diff --git a/go/protocols/flow/flow.pb.go b/go/protocols/flow/flow.pb.go index 8fbf9ba031..ee3e26dbdb 100644 --- a/go/protocols/flow/flow.pb.go +++ b/go/protocols/flow/flow.pb.go @@ -853,10 +853,14 @@ type CollectionSpec_Derivation struct { // Template for recovery logs of shards of this derivation. RecoveryLogTemplate *protocol.JournalSpec `protobuf:"bytes,6,opt,name=recovery_log_template,json=recoveryLogTemplate,proto3" json:"recovery_log_template,omitempty"` // Network ports of this derivation. - NetworkPorts []*NetworkPort `protobuf:"bytes,7,rep,name=network_ports,json=networkPorts,proto3" json:"network_ports,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + NetworkPorts []*NetworkPort `protobuf:"bytes,7,rep,name=network_ports,json=networkPorts,proto3" json:"network_ports,omitempty"` + // Transforms which were previously active for this task, but are no longer. + // Inactive transforms are unique by their transform name, and have no + // overlap with active transforms. + InactiveTransforms []*CollectionSpec_Derivation_Transform `protobuf:"bytes,8,rep,name=inactive_transforms,json=inactiveTransforms,proto3" json:"inactive_transforms,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *CollectionSpec_Derivation) Reset() { *m = CollectionSpec_Derivation{} } @@ -894,7 +898,7 @@ var xxx_messageInfo_CollectionSpec_Derivation proto.InternalMessageInfo // Transforms of the derivation. type CollectionSpec_Derivation_Transform struct { - // Stable name of this transform. + // Stable and unique name of this transform. Name Transform `protobuf:"bytes,1,opt,name=name,proto3,casttype=Transform" json:"name,omitempty"` // Source collection which is read by this transform. Collection CollectionSpec `protobuf:"bytes,2,opt,name=collection,proto3" json:"collection"` @@ -1030,10 +1034,14 @@ type CaptureSpec struct { // Template for recovery logs of shards of this capture. RecoveryLogTemplate *protocol.JournalSpec `protobuf:"bytes,7,opt,name=recovery_log_template,json=recoveryLogTemplate,proto3" json:"recovery_log_template,omitempty"` // Network ports of this capture. - NetworkPorts []*NetworkPort `protobuf:"bytes,8,rep,name=network_ports,json=networkPorts,proto3" json:"network_ports,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + NetworkPorts []*NetworkPort `protobuf:"bytes,8,rep,name=network_ports,json=networkPorts,proto3" json:"network_ports,omitempty"` + // Bindings which were previously active for this task, but are no longer. + // Inactive bindings are unique by their resource path, and have no + // overlap with active bindings. + InactiveBindings []*CaptureSpec_Binding `protobuf:"bytes,9,rep,name=inactive_bindings,json=inactiveBindings,proto3" json:"inactive_bindings,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *CaptureSpec) Reset() { *m = CaptureSpec{} } @@ -1069,9 +1077,8 @@ func (m *CaptureSpec) XXX_DiscardUnknown() { var xxx_messageInfo_CaptureSpec proto.InternalMessageInfo -// Bindings of endpoint resources and collections into which they're captured. -// Bindings are ordered and unique on the bound collection name, -// and are also unique on the resource path. +// Bindings between endpoint resources, uniquely identified by their +// resource path, and the collections into which they're captured. type CaptureSpec_Binding struct { // JSON-encoded configuration of the bound resource. ResourceConfigJson encoding_json.RawMessage `protobuf:"bytes,1,opt,name=resource_config_json,json=resourceConfig,proto3,casttype=encoding/json.RawMessage" json:"resource_config_json,omitempty"` @@ -1137,10 +1144,14 @@ type MaterializationSpec struct { // Template for recovery logs of shards of this materialization. RecoveryLogTemplate *protocol.JournalSpec `protobuf:"bytes,6,opt,name=recovery_log_template,json=recoveryLogTemplate,proto3" json:"recovery_log_template,omitempty"` // Network ports of this materialization. - NetworkPorts []*NetworkPort `protobuf:"bytes,7,rep,name=network_ports,json=networkPorts,proto3" json:"network_ports,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + NetworkPorts []*NetworkPort `protobuf:"bytes,7,rep,name=network_ports,json=networkPorts,proto3" json:"network_ports,omitempty"` + // Bindings which were previously active for this task, but are no longer. + // Inactive bindings are unique by their resource path, and have no + // overlap with active bindings. + InactiveBindings []*MaterializationSpec_Binding `protobuf:"bytes,9,rep,name=inactive_bindings,json=inactiveBindings,proto3" json:"inactive_bindings,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *MaterializationSpec) Reset() { *m = MaterializationSpec{} } @@ -1176,9 +1187,8 @@ func (m *MaterializationSpec) XXX_DiscardUnknown() { var xxx_messageInfo_MaterializationSpec proto.InternalMessageInfo -// Bindings of endpoint resources and collections from which they're -// materialized. Bindings are ordered and unique on the bound collection name, -// and are also unique on the resource path. +// Bindings between endpoint resources, uniquely identified by their +// resource path, and the collections from which they're materialized. type MaterializationSpec_Binding struct { // JSON-encoded configuration of the bound resource. ResourceConfigJson encoding_json.RawMessage `protobuf:"bytes,1,opt,name=resource_config_json,json=resourceConfig,proto3,casttype=encoding/json.RawMessage" json:"resource_config_json,omitempty"` @@ -2326,240 +2336,243 @@ func init() { func init() { proto.RegisterFile("go/protocols/flow/flow.proto", fileDescriptor_d0677502142fec31) } var fileDescriptor_d0677502142fec31 = []byte{ - // 3717 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x5a, 0xcb, 0x8f, 0x1b, 0xc9, - 0x79, 0x9f, 0xe6, 0x9b, 0x1f, 0x1f, 0xd3, 0x53, 0x33, 0x92, 0x28, 0xee, 0x6a, 0x38, 0xa2, 0x63, - 0xac, 0x76, 0xe5, 0xe5, 0xac, 0x67, 0x61, 0x67, 0x25, 0x41, 0x31, 0xf8, 0x1a, 0x2d, 0x25, 0x0e, - 0xc9, 0x34, 0x39, 0x5e, 0xcb, 0x80, 0xd1, 0x68, 0x76, 0x17, 0x39, 0xad, 0x69, 0x76, 0x33, 0xdd, - 0xc5, 0xd1, 0xd0, 0x97, 0x04, 0x81, 0x81, 0x00, 0x8b, 0x04, 0xf1, 0x25, 0xb0, 0x6f, 0xde, 0x18, - 0x48, 0x90, 0x7f, 0x21, 0xc9, 0x25, 0x40, 0x2e, 0x7b, 0x0a, 0x0c, 0x04, 0x48, 0x80, 0x20, 0xd0, - 0x22, 0xce, 0xc1, 0x3e, 0x25, 0xb7, 0x20, 0xd0, 0x29, 0xa8, 0x47, 0x37, 0x9b, 0xc3, 0xd1, 0x3c, - 0xd6, 0xce, 0x61, 0x2f, 0x44, 0xd7, 0xf7, 0xaa, 0xaa, 0xef, 0xab, 0xfa, 0x7d, 0x5f, 0x55, 0x11, - 0xde, 0x1e, 0x3b, 0xbb, 0x53, 0xd7, 0x21, 0x8e, 0xee, 0x58, 0xde, 0xee, 0xc8, 0x72, 0x5e, 0xb2, - 0x9f, 0x0a, 0xa3, 0xa1, 0x18, 0xfd, 0x2e, 0x6e, 0x0f, 0x5d, 0xe7, 0x18, 0xbb, 0x81, 0x5c, 0xf0, - 0xc1, 0xa5, 0x8a, 0x3b, 0xba, 0x63, 0x7b, 0xb3, 0xc9, 0x05, 0x12, 0x5b, 0x63, 0x67, 0xec, 0xb0, - 0xcf, 0x5d, 0xfa, 0x25, 0xa8, 0xa5, 0xb1, 0xe3, 0x8c, 0x2d, 0xcc, 0x85, 0x87, 0xb3, 0xd1, 0x2e, - 0x31, 0x27, 0xd8, 0x23, 0xda, 0x64, 0xca, 0x05, 0xca, 0x8f, 0x21, 0xde, 0xb7, 0x4c, 0x1d, 0xa3, - 0x2d, 0x88, 0x0f, 0xf1, 0xd8, 0xb4, 0x0b, 0xd2, 0x8e, 0x74, 0x2f, 0xa7, 0xf0, 0x06, 0x92, 0x21, - 0x8a, 0x6d, 0xa3, 0x10, 0x61, 0x34, 0xfa, 0xf9, 0x30, 0xfb, 0x8b, 0xbf, 0x2c, 0xad, 0xfd, 0xf8, - 0xe7, 0xa5, 0xb5, 0x9f, 0xfe, 0xbc, 0xb4, 0x56, 0xb6, 0x21, 0x7d, 0x78, 0xd8, 0x6a, 0xf4, 0x34, - 0x97, 0x78, 0x08, 0x41, 0xcc, 0x76, 0x0c, 0xcc, 0x2c, 0x24, 0x14, 0xf6, 0x8d, 0x1e, 0x41, 0x5c, - 0xb7, 0x1c, 0xfd, 0x98, 0x99, 0x48, 0xd4, 0xbe, 0xfe, 0xfa, 0x55, 0xe9, 0xee, 0xd8, 0xa9, 0x8c, - 0xb5, 0x1f, 0x62, 0x42, 0x70, 0xc5, 0xc0, 0x27, 0xbb, 0xba, 0xe3, 0xe2, 0xdd, 0x09, 0xf6, 0x3c, - 0x6d, 0x8c, 0x2b, 0x75, 0x2a, 0xac, 0x70, 0x9d, 0x87, 0xf2, 0xaf, 0x3f, 0x2b, 0x49, 0x4b, 0xfd, - 0xfd, 0x79, 0x04, 0xa0, 0xe7, 0x3a, 0x2f, 0xb0, 0x4e, 0x4c, 0x87, 0x0d, 0x6f, 0x4a, 0x5c, 0xd6, - 0x61, 0x5a, 0xa1, 0x9f, 0x74, 0x1a, 0x23, 0x13, 0x5b, 0x7c, 0xc8, 0x69, 0x85, 0x37, 0x50, 0x11, - 0x52, 0xf8, 0x74, 0x6a, 0x99, 0xba, 0x49, 0x0a, 0xd1, 0x1d, 0xe9, 0x5e, 0x4a, 0x09, 0xda, 0xe8, - 0x1e, 0xc8, 0xa6, 0xa7, 0x4e, 0x35, 0x97, 0x98, 0xd4, 0xa6, 0x7a, 0x8c, 0xe7, 0x85, 0x18, 0x93, - 0xc9, 0x9b, 0x5e, 0xcf, 0x27, 0x3f, 0xc3, 0x73, 0xf4, 0x3b, 0x90, 0xa7, 0x92, 0xae, 0x39, 0xd1, - 0xdc, 0x39, 0x93, 0x8b, 0x33, 0xb9, 0xac, 0xe9, 0xf5, 0x38, 0x91, 0x4a, 0x7d, 0x08, 0x69, 0xd3, - 0x1e, 0x61, 0x17, 0xdb, 0x3a, 0x2e, 0x24, 0x76, 0xa4, 0x7b, 0x99, 0xbd, 0xf5, 0x0a, 0x0b, 0x78, - 0xcb, 0x27, 0xd7, 0x62, 0x9f, 0xbf, 0x2a, 0xad, 0x29, 0x0b, 0x39, 0xf4, 0x11, 0xac, 0xbf, 0x74, - 0x4d, 0x82, 0xd5, 0x85, 0x6a, 0xf2, 0x5c, 0x55, 0x25, 0xcf, 0xe4, 0x82, 0x76, 0xf9, 0xdf, 0x13, - 0x90, 0x0e, 0x5a, 0x74, 0xfa, 0x64, 0x3e, 0xc5, 0x5e, 0x41, 0xda, 0x89, 0xd2, 0xe9, 0xb3, 0x06, - 0xaa, 0x40, 0xc2, 0x23, 0xae, 0x69, 0x8f, 0xd9, 0xe4, 0x33, 0x7b, 0x37, 0xcf, 0x18, 0xad, 0xf4, - 0x19, 0x57, 0x11, 0x52, 0xcc, 0x8a, 0x49, 0x2c, 0xcc, 0xfc, 0x40, 0xad, 0xd0, 0x06, 0xda, 0x81, - 0x8c, 0x81, 0x3d, 0xdd, 0x35, 0xa7, 0xd4, 0x21, 0x6c, 0xee, 0x69, 0x25, 0x4c, 0x42, 0x8f, 0x20, - 0x6b, 0xe0, 0x91, 0x36, 0xb3, 0x88, 0xfa, 0xc2, 0x73, 0x6c, 0x36, 0xfb, 0x74, 0xed, 0xed, 0xd7, - 0xaf, 0x4a, 0x05, 0x6c, 0xeb, 0x8e, 0x61, 0xda, 0xe3, 0x5d, 0xca, 0xa8, 0x28, 0xda, 0xcb, 0x03, - 0x1e, 0x70, 0x25, 0x29, 0x34, 0xd0, 0x4d, 0x48, 0x78, 0x58, 0x77, 0x31, 0x61, 0x33, 0x4f, 0x29, - 0xa2, 0x45, 0x07, 0x8f, 0x4f, 0x4d, 0x8f, 0x78, 0x85, 0xd4, 0x8e, 0x74, 0x2f, 0xbf, 0x3a, 0xf8, - 0x26, 0xe3, 0x2a, 0x42, 0x0a, 0x7d, 0x13, 0x92, 0x36, 0xdd, 0x29, 0xa6, 0x5e, 0x48, 0xb3, 0xd9, - 0xde, 0x3a, 0xab, 0xd0, 0xe1, 0x6c, 0xc5, 0x97, 0x43, 0xf7, 0x21, 0xae, 0xb9, 0xae, 0x36, 0x2f, - 0x00, 0x53, 0xb8, 0x71, 0x56, 0xa1, 0x4a, 0x99, 0x0a, 0x97, 0x29, 0xfe, 0x4c, 0x82, 0x04, 0xf7, - 0x17, 0xba, 0x0b, 0x59, 0xdd, 0xb1, 0x09, 0xb6, 0x89, 0x4a, 0x1d, 0xcd, 0xbc, 0x9b, 0x56, 0x32, - 0x82, 0x36, 0x98, 0x4f, 0x31, 0x9d, 0xd5, 0xc8, 0x71, 0x27, 0x1a, 0x11, 0xbe, 0x14, 0x2d, 0xf4, - 0x2e, 0xc8, 0xbe, 0xaa, 0xef, 0x1a, 0x36, 0xef, 0xb4, 0xb2, 0x2e, 0xe8, 0x4d, 0x41, 0x46, 0x77, - 0x00, 0x26, 0xda, 0xa9, 0x6a, 0x61, 0x7b, 0x4c, 0x8e, 0x98, 0x4f, 0x73, 0x4a, 0x7a, 0xa2, 0x9d, - 0xb6, 0x19, 0xe1, 0x69, 0x2c, 0x25, 0xc9, 0x91, 0xa7, 0xb1, 0x54, 0x44, 0x8e, 0x3e, 0x8d, 0xa5, - 0xe2, 0x72, 0xa2, 0xf8, 0x87, 0x90, 0x14, 0x53, 0x44, 0x25, 0xc8, 0x1c, 0x69, 0x9e, 0x3a, 0x31, - 0x6d, 0x73, 0x32, 0x9b, 0xb0, 0x8d, 0x92, 0x52, 0xe0, 0x48, 0xf3, 0x0e, 0x38, 0x05, 0x15, 0x20, - 0xe9, 0x33, 0xe9, 0x8e, 0x91, 0x14, 0xbf, 0x19, 0xa8, 0x6a, 0xa7, 0x8c, 0x1b, 0x5d, 0xa8, 0x72, - 0x0a, 0x53, 0x15, 0xcc, 0x98, 0x50, 0xe5, 0xcd, 0xe2, 0x8f, 0x24, 0x88, 0x33, 0x9f, 0xa1, 0xb7, - 0x20, 0x3d, 0x31, 0x6d, 0xd5, 0x24, 0x78, 0xe2, 0x09, 0x64, 0x49, 0x4d, 0x4c, 0xbb, 0x45, 0xdb, - 0xa8, 0x0c, 0x39, 0xd1, 0x83, 0x10, 0x88, 0xb0, 0x3e, 0x32, 0xbc, 0x0f, 0x2e, 0x43, 0x0d, 0x04, - 0xfc, 0xa8, 0x30, 0xe0, 0x33, 0xef, 0x00, 0x50, 0x86, 0xca, 0x97, 0x7c, 0x8c, 0x2d, 0xf9, 0x34, - 0xa5, 0x50, 0xd7, 0x7b, 0xe5, 0x1a, 0x24, 0xf8, 0xda, 0x40, 0x19, 0x48, 0xb6, 0x3a, 0xdf, 0xad, - 0xb6, 0x5b, 0x0d, 0x79, 0x0d, 0xa5, 0x20, 0x76, 0x70, 0xd8, 0x1f, 0xc8, 0x12, 0x4a, 0x42, 0xf4, - 0xa0, 0xfa, 0x5c, 0x8e, 0xa0, 0x2c, 0xa4, 0x5a, 0x07, 0xbd, 0x76, 0xab, 0xde, 0x1a, 0xc8, 0x51, - 0x04, 0x90, 0xa8, 0x57, 0x3b, 0x9d, 0xee, 0x40, 0x8e, 0x95, 0x9f, 0x43, 0xa6, 0x83, 0xc9, 0x4b, - 0xc7, 0x3d, 0xee, 0x39, 0x2e, 0x5b, 0xa4, 0xf6, 0x6c, 0x32, 0xc4, 0xae, 0x98, 0x8c, 0x68, 0x51, - 0x80, 0xf1, 0xf1, 0x58, 0x20, 0x4f, 0xd0, 0xa6, 0x3a, 0xd3, 0xd9, 0xd0, 0x32, 0x75, 0xe1, 0x43, - 0xd1, 0x2a, 0xff, 0x4f, 0x1e, 0xf2, 0x75, 0xc7, 0xb2, 0x38, 0x96, 0xf5, 0xa7, 0x58, 0x47, 0x65, - 0x88, 0xd9, 0xda, 0x84, 0x23, 0x68, 0xba, 0x96, 0x7f, 0xfd, 0xaa, 0x04, 0x0b, 0x09, 0x85, 0xf1, - 0x50, 0x03, 0x36, 0x38, 0x54, 0x78, 0xfa, 0x11, 0x9e, 0x68, 0x7c, 0xa7, 0xa5, 0xae, 0xb0, 0xd3, - 0x32, 0x4c, 0xad, 0xcf, 0xb4, 0x50, 0x0d, 0x64, 0x17, 0x6b, 0xc6, 0x92, 0x91, 0xcc, 0x15, 0x8c, - 0x00, 0xd5, 0x12, 0x36, 0x64, 0x88, 0x52, 0x10, 0x8c, 0x32, 0xbf, 0xd3, 0x4f, 0x74, 0x1b, 0x52, - 0xb3, 0x99, 0x69, 0xa8, 0x14, 0x94, 0xf9, 0x7a, 0x4f, 0xd2, 0x76, 0x8f, 0xb8, 0x74, 0xc1, 0x2f, - 0x30, 0x96, 0xa1, 0xb2, 0x57, 0x88, 0x33, 0xcd, 0xf5, 0x80, 0xbe, 0xcf, 0xc8, 0xe8, 0x23, 0xc8, - 0x4c, 0x03, 0x8c, 0xf7, 0x0a, 0x89, 0x9d, 0xe8, 0xbd, 0xcc, 0x9e, 0xcc, 0x37, 0xe5, 0x02, 0xfc, - 0x05, 0x88, 0x86, 0x45, 0xa9, 0x6f, 0x34, 0xfd, 0x58, 0x25, 0x78, 0x32, 0xb5, 0x34, 0x82, 0xf9, - 0xb4, 0x92, 0x57, 0xf1, 0x8d, 0xa6, 0x1f, 0x0f, 0x84, 0x16, 0x6a, 0x00, 0x5a, 0x0c, 0xd5, 0xb7, - 0x25, 0xc0, 0xe4, 0x46, 0x25, 0xc8, 0xbb, 0x4f, 0x9d, 0x99, 0x6b, 0x6b, 0x16, 0x0d, 0x9c, 0xb2, - 0x11, 0x28, 0x04, 0x56, 0xbe, 0x03, 0x60, 0x60, 0xd7, 0x3c, 0xd1, 0x18, 0x5a, 0x66, 0x99, 0x76, - 0x89, 0x4f, 0x62, 0x39, 0xea, 0x95, 0x46, 0x20, 0xa6, 0x84, 0x54, 0x8a, 0xff, 0x02, 0x00, 0x0b, - 0x16, 0x1a, 0x40, 0x5e, 0x77, 0x6c, 0x1b, 0xeb, 0xc4, 0x71, 0x39, 0xdc, 0x48, 0x0c, 0x0f, 0xdf, - 0xbf, 0xc4, 0x66, 0xa5, 0xee, 0x6b, 0xd1, 0x5d, 0xa1, 0xe4, 0xf4, 0x70, 0x13, 0x3d, 0x00, 0x0a, - 0x57, 0x23, 0x73, 0xcc, 0x7d, 0x15, 0xb9, 0x82, 0xaf, 0x12, 0x5c, 0x01, 0x75, 0x01, 0x88, 0xab, - 0xd9, 0x1e, 0x45, 0x34, 0x8f, 0xad, 0x82, 0xcc, 0xde, 0xbb, 0x97, 0x0d, 0x66, 0xe0, 0x6b, 0x88, - 0xf0, 0x85, 0x4c, 0xa0, 0x4f, 0x60, 0xc3, 0x3b, 0x9a, 0x8d, 0x46, 0x16, 0xa6, 0xc9, 0x35, 0xb4, - 0xab, 0xf3, 0x7b, 0xf7, 0x2f, 0xb3, 0xdb, 0xe7, 0x8a, 0x6c, 0x8a, 0xeb, 0xc2, 0xca, 0x33, 0x3c, - 0x67, 0x40, 0x80, 0x1e, 0x42, 0xde, 0x3b, 0xd2, 0x5c, 0x63, 0x11, 0xcc, 0x38, 0x0b, 0xc7, 0x66, - 0xc5, 0x2f, 0xab, 0x2a, 0x7d, 0xca, 0x67, 0xa1, 0xcc, 0x31, 0xd1, 0x20, 0x8c, 0x2d, 0xb8, 0xe1, - 0x62, 0xdd, 0x39, 0xc1, 0xee, 0x5c, 0xb5, 0x9c, 0xf1, 0xc2, 0x44, 0xe2, 0xa2, 0xf5, 0xb0, 0xe9, - 0xeb, 0xb4, 0x9d, 0x71, 0x60, 0xea, 0xdb, 0x90, 0xb3, 0x39, 0x96, 0xa8, 0x53, 0xc7, 0x25, 0x5e, - 0x21, 0xc9, 0x7c, 0xb6, 0xc1, 0xe7, 0x16, 0x82, 0x19, 0x25, 0x6b, 0x2f, 0x1a, 0x5e, 0xf1, 0x67, - 0x71, 0x48, 0x07, 0x7e, 0x43, 0x77, 0x97, 0x30, 0x22, 0xf7, 0xfa, 0x55, 0x69, 0xc1, 0x14, 0x10, - 0xf1, 0x10, 0x40, 0x0f, 0x3c, 0xc5, 0x62, 0x9a, 0xd9, 0xdb, 0x3a, 0xcf, 0x83, 0x7e, 0x10, 0x16, - 0xd2, 0xa8, 0x1d, 0x5e, 0xfc, 0x1e, 0xb6, 0xd8, 0x52, 0x11, 0x75, 0xc3, 0xad, 0xc5, 0x64, 0xdb, - 0xda, 0x10, 0x5b, 0x7d, 0xc1, 0x16, 0x66, 0x16, 0x9b, 0xc0, 0x67, 0x70, 0x5c, 0x34, 0x1d, 0xd7, - 0x24, 0xbc, 0xa8, 0xca, 0x29, 0x41, 0x1b, 0x7d, 0x03, 0x10, 0x83, 0x20, 0x03, 0x5b, 0xda, 0x5c, - 0xf5, 0xb0, 0xee, 0xd8, 0x0c, 0x13, 0xa8, 0x14, 0x03, 0xa7, 0x06, 0x65, 0xf4, 0x39, 0x9d, 0xa6, - 0xa3, 0xd0, 0xe2, 0x60, 0xa0, 0x90, 0x56, 0x60, 0x11, 0x69, 0x74, 0x08, 0x45, 0x5f, 0xc0, 0xd2, - 0x26, 0x43, 0x43, 0x53, 0xc3, 0x0b, 0xfb, 0x2a, 0x20, 0xb0, 0x29, 0xf4, 0xdb, 0x4c, 0xbd, 0xce, - 0x57, 0xf9, 0x3e, 0xa0, 0x73, 0xcc, 0x5d, 0x05, 0x6f, 0xb3, 0x56, 0xd8, 0xce, 0x5b, 0x90, 0x66, - 0xb3, 0x75, 0x6c, 0x6b, 0xce, 0xb0, 0x24, 0xa5, 0xa4, 0x28, 0xa1, 0x6b, 0x5b, 0x73, 0x54, 0x81, - 0xcd, 0x17, 0x7c, 0xf5, 0xa8, 0x1c, 0x95, 0x67, 0xa3, 0x91, 0x79, 0xca, 0xca, 0x91, 0xb4, 0xb2, - 0x21, 0x58, 0x0a, 0x45, 0x5e, 0xc6, 0x40, 0x0f, 0x00, 0x6c, 0x87, 0xa8, 0x43, 0x3c, 0x72, 0x5c, - 0xcc, 0x70, 0x3b, 0xb3, 0x57, 0xac, 0xf0, 0x5a, 0xbf, 0xe2, 0xd7, 0xfa, 0x95, 0x81, 0x5f, 0xeb, - 0x2b, 0x69, 0xdb, 0x21, 0x35, 0x26, 0x8c, 0x7e, 0x17, 0x68, 0x43, 0xd5, 0x46, 0x04, 0xbb, 0x02, - 0x95, 0x2e, 0xd2, 0x4c, 0xd9, 0x0e, 0xa9, 0x52, 0x59, 0x1a, 0xca, 0xa1, 0xa6, 0x1f, 0x8f, 0x4c, - 0xcb, 0x2a, 0xe4, 0x78, 0x28, 0xfd, 0x76, 0xf9, 0x07, 0x90, 0x5b, 0x42, 0x19, 0x54, 0x84, 0x9b, - 0x22, 0xe1, 0xaa, 0xf5, 0x6e, 0xa7, 0xd3, 0xac, 0x0f, 0xba, 0x8a, 0x3a, 0x78, 0xde, 0x6b, 0xca, - 0x6b, 0x34, 0xbd, 0xf6, 0x7f, 0xbf, 0xdd, 0x1a, 0x34, 0x65, 0x09, 0xe5, 0x01, 0x28, 0xb5, 0x5f, - 0x57, 0x5a, 0xbd, 0x81, 0x1c, 0x41, 0x69, 0x88, 0xb7, 0x0e, 0xaa, 0x4f, 0x9a, 0x72, 0x94, 0x7e, - 0xb6, 0xbb, 0xf5, 0x6a, 0x5b, 0x8e, 0x95, 0x0f, 0x20, 0x13, 0xda, 0xdf, 0xa8, 0x00, 0x5b, 0xbe, - 0xf1, 0xfe, 0xc7, 0x87, 0xfb, 0xfb, 0xed, 0xa6, 0x6f, 0x3a, 0x03, 0xc9, 0x5a, 0xb7, 0xdb, 0x6e, - 0x56, 0x3b, 0xb2, 0xc4, 0x93, 0xfe, 0xa0, 0xf9, 0xa4, 0xa9, 0xc8, 0x11, 0xd6, 0xe9, 0x40, 0x69, - 0x75, 0x9e, 0xc8, 0xd1, 0xf2, 0x4f, 0x22, 0x90, 0x67, 0xa9, 0x86, 0x2f, 0x53, 0xba, 0xea, 0x11, - 0xc4, 0x8e, 0xf1, 0xdc, 0x2f, 0x9b, 0xd9, 0x37, 0xcd, 0xdb, 0x27, 0x9a, 0x35, 0xc3, 0xb4, 0x2e, - 0xa1, 0x54, 0xd1, 0xa2, 0x8e, 0x30, 0x1c, 0x7d, 0x36, 0xc1, 0x36, 0x11, 0x15, 0x5f, 0xd0, 0x46, - 0x27, 0x70, 0x83, 0xe5, 0xb6, 0xf0, 0x62, 0x51, 0x27, 0xda, 0x94, 0xc1, 0x58, 0x66, 0xef, 0x1b, - 0x7c, 0x13, 0x2e, 0x77, 0xce, 0x9b, 0x7c, 0x99, 0x3c, 0xf5, 0x1c, 0xfb, 0x40, 0x9b, 0x36, 0x6d, - 0xe2, 0xce, 0x6b, 0x6f, 0x7f, 0xfa, 0xc5, 0x45, 0x29, 0x6b, 0xb4, 0x50, 0x2b, 0x36, 0xe1, 0xd6, - 0x1b, 0xac, 0xf8, 0x59, 0x5a, 0x9c, 0x91, 0x68, 0x96, 0xde, 0x82, 0x38, 0x9b, 0x8a, 0x7f, 0x46, - 0x62, 0x8d, 0x87, 0x91, 0x8f, 0xa4, 0xf2, 0xff, 0xc6, 0x21, 0x53, 0xd7, 0xa6, 0x64, 0xe6, 0x62, - 0x56, 0x8f, 0x94, 0x96, 0xb0, 0x26, 0xf3, 0xfa, 0x55, 0x29, 0x29, 0xd8, 0x02, 0x69, 0xf6, 0x57, - 0x92, 0x52, 0x84, 0x25, 0x25, 0x3f, 0xd1, 0x2d, 0x6c, 0x5d, 0x2b, 0x0d, 0x45, 0xaf, 0x91, 0x86, - 0xbe, 0x05, 0xa9, 0xa1, 0x69, 0x53, 0x11, 0x4f, 0x78, 0xf9, 0xf6, 0x6a, 0xe7, 0x35, 0x2e, 0xa1, - 0x04, 0xa2, 0xb4, 0x1e, 0x31, 0x6d, 0x82, 0xdd, 0x13, 0xcd, 0x3a, 0x83, 0x3d, 0xeb, 0x3e, 0xdd, - 0x87, 0x9e, 0xd5, 0xf4, 0x91, 0xf8, 0xcd, 0xd3, 0x47, 0xf2, 0x37, 0x4f, 0x1f, 0xa9, 0xab, 0xa5, - 0x8f, 0xff, 0x96, 0x20, 0x29, 0xe6, 0x8f, 0x9e, 0xc2, 0x96, 0x8b, 0x3d, 0x67, 0xe6, 0xea, 0x78, - 0x09, 0xcf, 0xa4, 0x2b, 0x38, 0x3c, 0xef, 0x6b, 0x0a, 0x44, 0xfb, 0x1a, 0xe4, 0x02, 0x5b, 0x53, - 0x8d, 0x1c, 0x89, 0x6d, 0x92, 0xf5, 0x89, 0x3d, 0x8d, 0x1c, 0x9d, 0x49, 0x45, 0xd1, 0x6b, 0xa5, - 0xa2, 0x30, 0xe2, 0xc4, 0x96, 0x11, 0x87, 0xc2, 0xa9, 0x47, 0x68, 0x89, 0xe7, 0x1f, 0xc3, 0xd3, - 0x4a, 0x8a, 0x11, 0x9e, 0xe1, 0x79, 0x79, 0xef, 0x2c, 0x1c, 0x2d, 0xd5, 0xff, 0x01, 0xc6, 0x24, - 0x17, 0x18, 0x93, 0x2a, 0xff, 0x2b, 0xc0, 0xe6, 0x81, 0x46, 0xb0, 0x6b, 0x6a, 0x96, 0xf9, 0x43, - 0x2d, 0x28, 0xc9, 0xdf, 0x59, 0xda, 0x02, 0x9b, 0xaf, 0x5f, 0x95, 0xd6, 0xcf, 0x88, 0x89, 0xad, - 0xd0, 0x79, 0xc3, 0x56, 0x78, 0x87, 0xcf, 0xf6, 0x1c, 0xdb, 0xff, 0x6f, 0x5b, 0xe2, 0xf1, 0xca, - 0x96, 0xb8, 0xfb, 0xe6, 0x41, 0xac, 0x6e, 0x8d, 0xaf, 0x78, 0xb9, 0xf4, 0x4f, 0x89, 0xaf, 0xe8, - 0x7a, 0x3f, 0xbf, 0xf4, 0x4a, 0xfe, 0x16, 0x4a, 0xaf, 0xf4, 0x99, 0xd2, 0xab, 0x0e, 0xeb, 0x3c, - 0x4d, 0x79, 0x7e, 0x12, 0x62, 0x1b, 0x2c, 0x18, 0xea, 0x72, 0x82, 0x12, 0x7d, 0xe4, 0x47, 0xcb, - 0x39, 0xf3, 0x6b, 0x90, 0x33, 0xb0, 0x45, 0x34, 0x75, 0x36, 0x35, 0x34, 0x82, 0x3d, 0xff, 0x36, - 0x8c, 0x11, 0x0f, 0x39, 0x0d, 0x0d, 0x01, 0x19, 0x78, 0xea, 0x62, 0x5d, 0x23, 0xd8, 0x50, 0x45, - 0x81, 0x25, 0x16, 0xc3, 0x87, 0x97, 0x2e, 0xca, 0x4a, 0x23, 0xd0, 0x15, 0x05, 0x80, 0xb2, 0x61, - 0x9c, 0x25, 0xbd, 0xa9, 0x7a, 0x4a, 0x5d, 0xad, 0x7a, 0x82, 0x2f, 0x5d, 0x3d, 0x65, 0xbe, 0x64, - 0xf5, 0x94, 0xbd, 0x08, 0xcb, 0x72, 0xcb, 0x58, 0x56, 0xfc, 0x23, 0x09, 0x36, 0x56, 0xbc, 0x80, - 0xee, 0x00, 0x8c, 0x5d, 0x67, 0x36, 0x55, 0x17, 0xd8, 0xa4, 0xa4, 0x19, 0xa5, 0x43, 0xb1, 0xe8, - 0xb7, 0x5a, 0xc4, 0x97, 0x1f, 0x5f, 0x0d, 0x4e, 0x53, 0x0b, 0x38, 0x4d, 0xd3, 0xcf, 0x46, 0xf3, - 0x59, 0x75, 0x5f, 0x86, 0xf2, 0x7f, 0xa5, 0x21, 0xd1, 0xad, 0xce, 0xc8, 0xd1, 0x9e, 0xb8, 0x26, - 0x39, 0x31, 0x0d, 0xec, 0x5f, 0xda, 0x06, 0x6d, 0xf4, 0x1e, 0x6c, 0x68, 0x33, 0x72, 0xa4, 0xce, - 0x5c, 0x6b, 0x81, 0x1a, 0xbc, 0x42, 0x59, 0xa7, 0x8c, 0x43, 0xd7, 0x0a, 0xa0, 0xe1, 0x01, 0xdc, - 0xd6, 0x74, 0x1d, 0x7b, 0x9e, 0x4a, 0x9c, 0x63, 0x6c, 0x2f, 0xeb, 0xf0, 0x9a, 0xec, 0x26, 0x17, - 0x18, 0x50, 0x7e, 0x58, 0xb5, 0x02, 0x9b, 0x4b, 0xaa, 0x13, 0x4c, 0x8e, 0x1c, 0x83, 0xdf, 0x7d, - 0x28, 0x1b, 0x21, 0xa5, 0x03, 0xc6, 0x60, 0xc3, 0x0a, 0xcb, 0x0f, 0x1d, 0x63, 0x2e, 0xee, 0x36, - 0xd6, 0x43, 0xd2, 0x35, 0xc7, 0x98, 0xa3, 0x1f, 0x49, 0x70, 0x67, 0x49, 0xf8, 0x08, 0x6b, 0x06, - 0x76, 0xbd, 0x45, 0x19, 0x18, 0x67, 0x10, 0x26, 0x4e, 0xb3, 0xdc, 0x29, 0x95, 0xea, 0xc2, 0xca, - 0xc7, 0x5c, 0xfe, 0x1a, 0x55, 0x20, 0xd2, 0x56, 0xb4, 0xd1, 0xa7, 0x12, 0x6c, 0x2f, 0x0d, 0xc3, - 0xc5, 0xde, 0xd4, 0xb1, 0x3d, 0xbc, 0x18, 0x47, 0x22, 0x5c, 0x8e, 0xae, 0x8e, 0x43, 0x11, 0x0a, - 0xd7, 0x18, 0x48, 0xd8, 0xdf, 0xbe, 0xfa, 0x81, 0x36, 0x45, 0x8f, 0xa0, 0xe8, 0xe2, 0x91, 0x8b, - 0xbd, 0xa3, 0xf3, 0x62, 0xc5, 0xaf, 0x3c, 0x6f, 0x09, 0x89, 0x95, 0x60, 0x7d, 0x40, 0xe1, 0x3b, - 0xac, 0x2c, 0xa2, 0x95, 0x65, 0x6a, 0x28, 0xac, 0x26, 0xc2, 0xc5, 0x0e, 0x95, 0x61, 0x0d, 0x16, - 0x2f, 0x0e, 0x05, 0x72, 0x58, 0x9e, 0x05, 0xec, 0x4f, 0x24, 0xd8, 0x5e, 0x16, 0x5f, 0x89, 0x58, - 0xfa, 0x1c, 0x4f, 0x29, 0x21, 0x3b, 0xd7, 0x0f, 0xd9, 0xa6, 0xbb, 0xaa, 0x8e, 0xfe, 0x4c, 0x82, - 0xd2, 0xf2, 0x48, 0x56, 0x83, 0x06, 0x6c, 0x28, 0xef, 0xbf, 0x71, 0x28, 0x5f, 0x22, 0x6a, 0x4b, - 0x9e, 0x0f, 0x85, 0xad, 0xd8, 0x86, 0xed, 0x8b, 0xd7, 0xe5, 0x75, 0xce, 0x15, 0xc5, 0x03, 0x28, - 0x5d, 0xb2, 0xba, 0xae, 0x6b, 0xee, 0x92, 0x10, 0x5c, 0xcb, 0x5c, 0x07, 0x76, 0x2e, 0x73, 0xe3, - 0xb5, 0x4e, 0x51, 0x7f, 0x1b, 0x85, 0xd4, 0x00, 0x7b, 0x84, 0xd5, 0x8f, 0x28, 0x5c, 0x3f, 0x8a, - 0x52, 0x71, 0x17, 0xe2, 0x1e, 0xc1, 0x53, 0x7e, 0xb0, 0xa4, 0x75, 0x15, 0x8b, 0xa8, 0xaf, 0x52, - 0xe9, 0x13, 0x3c, 0x15, 0x68, 0xcc, 0xe5, 0x8a, 0xbf, 0x8a, 0x40, 0x8c, 0x52, 0xd1, 0xb7, 0x68, - 0xaa, 0xc0, 0xd3, 0xf0, 0xfd, 0x5f, 0xe1, 0x1c, 0xed, 0x0a, 0x2b, 0x28, 0x53, 0x54, 0x94, 0x01, - 0xf6, 0x1d, 0x00, 0xa6, 0x66, 0xda, 0x06, 0x3e, 0x15, 0xaf, 0x79, 0xcc, 0x50, 0x8b, 0x12, 0xce, - 0xbe, 0xec, 0x44, 0x57, 0x5f, 0x76, 0x7c, 0x03, 0x9e, 0xee, 0x4c, 0xfd, 0x67, 0x21, 0x66, 0xa0, - 0x4f, 0x09, 0xa8, 0xb2, 0x54, 0xf5, 0xc4, 0xcf, 0xbd, 0xbd, 0x0e, 0x57, 0x3a, 0x0f, 0x20, 0x67, - 0x38, 0xba, 0xd8, 0x65, 0x27, 0x58, 0xe7, 0xd7, 0x39, 0x97, 0xd4, 0x63, 0x31, 0xaa, 0x82, 0x1e, - 0x03, 0x04, 0x19, 0xca, 0xbb, 0x5a, 0x71, 0x14, 0x52, 0x28, 0x6f, 0x43, 0x8c, 0x79, 0x04, 0x20, - 0xd1, 0xea, 0x3c, 0x69, 0xf6, 0x07, 0xfc, 0x42, 0xe2, 0xbb, 0x4d, 0xa5, 0xb5, 0xff, 0x5c, 0x96, - 0xca, 0x3f, 0x91, 0x20, 0xad, 0x68, 0xf6, 0x98, 0x9f, 0x7f, 0xdf, 0x82, 0xf4, 0x31, 0x9e, 0xab, - 0xfc, 0x61, 0x94, 0xba, 0x2d, 0xa9, 0xa4, 0x8e, 0xf1, 0xbc, 0xc6, 0xde, 0x46, 0x6f, 0x41, 0x92, - 0x32, 0xb1, 0x6d, 0x30, 0x8f, 0x25, 0x95, 0xc4, 0x31, 0x9e, 0x37, 0x6d, 0x03, 0x95, 0x21, 0xe7, - 0xaa, 0xec, 0x05, 0x53, 0x68, 0xc6, 0x18, 0x3b, 0xe3, 0xb2, 0xc7, 0x4d, 0xae, 0xbc, 0x0d, 0x19, - 0x5f, 0x86, 0x1a, 0x88, 0x33, 0x89, 0x34, 0x97, 0x68, 0xda, 0xc6, 0x43, 0xf9, 0xa7, 0x9f, 0x95, - 0xd6, 0xce, 0x3c, 0xb5, 0xe6, 0x83, 0x2c, 0xdc, 0xa7, 0xd5, 0x01, 0x7a, 0x04, 0x59, 0x5e, 0x7a, - 0x19, 0x57, 0x2f, 0x6a, 0x93, 0x42, 0x03, 0x95, 0x20, 0x33, 0xc1, 0xee, 0x98, 0x95, 0xb2, 0xfa, - 0x91, 0x78, 0x7a, 0x01, 0x46, 0xea, 0x51, 0x4a, 0xf9, 0xaf, 0x25, 0x48, 0xf7, 0xb1, 0xdb, 0x73, - 0x2c, 0x53, 0x67, 0x97, 0x75, 0x1e, 0x71, 0x55, 0xe2, 0xce, 0x6c, 0x5a, 0x88, 0x88, 0x0a, 0x88, - 0x3f, 0x82, 0xc8, 0x1e, 0x71, 0x07, 0x82, 0xc1, 0xab, 0x9d, 0x07, 0x70, 0xdb, 0xc6, 0x1e, 0x1d, - 0x98, 0x33, 0x7c, 0x71, 0x56, 0x89, 0x2f, 0xbf, 0x9b, 0x5c, 0xa0, 0x3b, 0x7c, 0xb1, 0xac, 0xfa, - 0x01, 0x6c, 0xb1, 0x77, 0xb6, 0xb3, 0x5a, 0xfc, 0xed, 0x07, 0x31, 0xde, 0x92, 0x46, 0xf9, 0xaf, - 0x22, 0x00, 0xcd, 0x53, 0xe2, 0x6a, 0x3a, 0xa9, 0xf6, 0x5a, 0xc5, 0xbf, 0x97, 0x20, 0x21, 0x2a, - 0xf6, 0xf0, 0x73, 0x84, 0xb4, 0xfc, 0x1c, 0xf1, 0x18, 0x32, 0xe1, 0xa7, 0x8f, 0xab, 0xdc, 0x7b, - 0x03, 0x57, 0xa0, 0xb0, 0x40, 0xf7, 0x03, 0x2f, 0xa0, 0xa7, 0xc4, 0xf5, 0xc4, 0x0b, 0x48, 0x9a, - 0x51, 0x7a, 0xc4, 0x5d, 0x79, 0xc1, 0x88, 0x5d, 0xf9, 0x05, 0xa3, 0xfc, 0x1c, 0x62, 0x75, 0xc7, - 0x38, 0x53, 0x62, 0xe5, 0x20, 0x5d, 0xef, 0x76, 0xf6, 0x5b, 0x4f, 0x0e, 0x95, 0x26, 0xbf, 0xd8, - 0x6a, 0x7e, 0x6f, 0xa0, 0x54, 0xeb, 0x03, 0x39, 0x82, 0x10, 0xe4, 0x45, 0xa3, 0xd9, 0x50, 0x0f, - 0x0f, 0x5b, 0x0d, 0x39, 0x8a, 0xb6, 0x40, 0x5e, 0xd0, 0xf6, 0x5b, 0xcd, 0x76, 0xa3, 0x2f, 0xc7, - 0xca, 0xbf, 0x92, 0x20, 0x55, 0x9b, 0x99, 0x96, 0x41, 0xbd, 0xf4, 0x6f, 0x4b, 0x5e, 0x1a, 0x52, - 0xb2, 0x6a, 0x1a, 0xbe, 0x97, 0x58, 0xbb, 0x65, 0x2c, 0x58, 0xc6, 0x50, 0xc0, 0x1c, 0x67, 0x35, - 0x86, 0xec, 0xb9, 0x96, 0x1d, 0x7b, 0x04, 0x5c, 0x88, 0x16, 0xda, 0x83, 0x8c, 0x38, 0x23, 0x31, - 0x8c, 0x8a, 0x31, 0x8c, 0xda, 0xf0, 0x4f, 0x40, 0xc1, 0xc3, 0xa8, 0x02, 0x5c, 0x8a, 0x6d, 0xc6, - 0xfb, 0xb0, 0xb1, 0x38, 0x3a, 0x8b, 0xb3, 0x9c, 0x38, 0xd4, 0xcb, 0x01, 0x43, 0x1c, 0xf8, 0xd0, - 0x5d, 0xc8, 0x0a, 0x87, 0xa9, 0xae, 0xe3, 0x10, 0xfe, 0xc8, 0x1c, 0x38, 0x51, 0x71, 0x1c, 0x52, - 0xde, 0x84, 0x0d, 0x05, 0x7b, 0x98, 0xb0, 0x6d, 0xa2, 0xe0, 0x3f, 0x98, 0x61, 0x8f, 0x94, 0xb7, - 0x00, 0x85, 0x89, 0x1c, 0xe3, 0xcb, 0x8f, 0x01, 0x55, 0x8d, 0x13, 0xcd, 0xd6, 0x31, 0xad, 0xda, - 0x85, 0x2c, 0x7a, 0x07, 0xd6, 0x35, 0x4e, 0x0d, 0xee, 0x86, 0xa8, 0x67, 0x62, 0x4a, 0x5e, 0x90, - 0xc5, 0xd5, 0x50, 0xf9, 0x06, 0x6c, 0x2e, 0xa9, 0x0b, 0xab, 0x1f, 0x41, 0xb6, 0xe1, 0xe8, 0x5e, - 0xd5, 0x36, 0x6a, 0x73, 0x7a, 0x0a, 0x42, 0xc0, 0xc0, 0x4b, 0xec, 0x17, 0x0e, 0x64, 0x5b, 0x10, - 0x1f, 0x52, 0x26, 0x73, 0x6c, 0x4c, 0xe1, 0x8d, 0xf2, 0x5f, 0x48, 0x90, 0x6b, 0xd9, 0x63, 0xec, - 0x11, 0x7f, 0x2c, 0xcb, 0xd8, 0x2a, 0x5d, 0x8a, 0xad, 0xe1, 0x70, 0x46, 0x96, 0xc3, 0xb9, 0x02, - 0xbb, 0xd1, 0xab, 0xc2, 0x6e, 0xf9, 0x9f, 0x23, 0x90, 0xf7, 0xc7, 0xc5, 0x27, 0x89, 0xfe, 0x4e, - 0x5a, 0x9c, 0xbb, 0xf8, 0x8b, 0x24, 0x2d, 0x9e, 0xf8, 0x1d, 0x6a, 0x50, 0xe3, 0x2e, 0xeb, 0xf8, - 0x87, 0xfe, 0x4f, 0xa8, 0x38, 0x4d, 0xdb, 0x1e, 0x2f, 0x52, 0x7e, 0xf0, 0xc7, 0x5f, 0x94, 0xee, - 0x9f, 0xf7, 0xb7, 0x90, 0x33, 0x7f, 0x8b, 0xf1, 0xf5, 0x3f, 0xfd, 0xa2, 0xf4, 0xde, 0x55, 0xc4, - 0xbb, 0xa3, 0x91, 0x87, 0x49, 0x70, 0x06, 0x5c, 0x74, 0x8b, 0x7e, 0x0f, 0xb2, 0xfe, 0xd8, 0x31, - 0xd1, 0x0d, 0xf1, 0x48, 0x12, 0xba, 0x9e, 0xe0, 0x45, 0x45, 0xa5, 0x49, 0x74, 0xc3, 0xdf, 0xa7, - 0x42, 0x81, 0x92, 0x8a, 0x0d, 0xb8, 0x79, 0xfe, 0x5c, 0x2e, 0xab, 0x14, 0xa2, 0xe1, 0x4a, 0xe1, - 0x6f, 0x22, 0x70, 0x6b, 0xa0, 0x79, 0xc7, 0xfe, 0x65, 0x86, 0xeb, 0x9c, 0xce, 0xfd, 0xb8, 0x7f, - 0x1b, 0x62, 0xce, 0x14, 0xf3, 0x88, 0x67, 0xf6, 0xca, 0x22, 0xcb, 0x9f, 0x2f, 0x5c, 0xe9, 0x4e, - 0xb1, 0xad, 0x30, 0x79, 0xb6, 0xd6, 0x34, 0xa2, 0xb1, 0xce, 0xb2, 0x0a, 0xfb, 0x2e, 0xfe, 0x83, - 0x04, 0x31, 0x2a, 0x82, 0xee, 0x41, 0x82, 0x57, 0xb8, 0xc2, 0xac, 0x7c, 0x76, 0xc2, 0x8a, 0xe0, - 0xa3, 0x36, 0xa4, 0xf8, 0x25, 0x90, 0xbf, 0x8c, 0x6a, 0xdf, 0x7c, 0xfd, 0xaa, 0xf4, 0xfe, 0x79, - 0x5e, 0x5f, 0xf9, 0x6f, 0x12, 0xbf, 0x1e, 0x6a, 0x35, 0x94, 0x24, 0x33, 0xd1, 0x62, 0xd9, 0x86, - 0x68, 0xee, 0x18, 0x13, 0x76, 0x95, 0x23, 0xc0, 0x1c, 0x38, 0x89, 0x3d, 0xac, 0x97, 0x20, 0xa3, - 0x5b, 0x26, 0xb6, 0x89, 0xaa, 0x19, 0x86, 0xff, 0x78, 0x0c, 0x9c, 0x54, 0x35, 0x0c, 0xb7, 0xfc, - 0xa7, 0x51, 0x28, 0xac, 0xce, 0x5e, 0x2c, 0xc5, 0x1e, 0xe4, 0xe8, 0xdc, 0x83, 0x9a, 0x59, 0xcc, - 0xee, 0xfe, 0x9b, 0x9c, 0x26, 0x56, 0x23, 0xf3, 0x9a, 0x68, 0x28, 0x59, 0x27, 0xd4, 0x3a, 0xd7, - 0x8b, 0x2f, 0x21, 0x1b, 0xd6, 0x40, 0x8f, 0x21, 0x41, 0x8f, 0xe9, 0x33, 0x4f, 0x54, 0x62, 0x5f, - 0xbf, 0xa4, 0xbb, 0x3e, 0x13, 0x56, 0x84, 0x52, 0x28, 0x16, 0x91, 0x8b, 0x63, 0x51, 0xfe, 0x31, - 0xfb, 0xcb, 0x09, 0x53, 0x4a, 0x40, 0xa4, 0xfb, 0x4c, 0x5e, 0x43, 0x9b, 0xb0, 0xde, 0xff, 0xb8, - 0xaa, 0x34, 0xd4, 0x4e, 0x77, 0xa0, 0xee, 0x77, 0x0f, 0x3b, 0x0d, 0x59, 0xa2, 0xb8, 0xdf, 0xe9, - 0xaa, 0x9c, 0xde, 0x53, 0x5a, 0x07, 0x55, 0xe5, 0xb9, 0x1c, 0x41, 0x37, 0x60, 0x83, 0x0a, 0x2d, - 0x93, 0xa3, 0x34, 0x71, 0xb4, 0x3a, 0x83, 0xa6, 0xd2, 0xa9, 0xb6, 0xd5, 0xa6, 0xa2, 0x74, 0x15, - 0x39, 0x86, 0x36, 0x20, 0xc7, 0xc5, 0xfa, 0x83, 0x6e, 0xaf, 0xd7, 0x6c, 0xc8, 0x71, 0x74, 0x03, - 0xe4, 0x5e, 0x57, 0x19, 0xb0, 0x7e, 0xaa, 0xed, 0x76, 0xf7, 0x93, 0x66, 0x43, 0xfe, 0x75, 0xf2, - 0xbd, 0x0e, 0x64, 0x42, 0x68, 0x4e, 0x53, 0x52, 0xbd, 0x3a, 0xa8, 0xb6, 0xbb, 0x4f, 0xe4, 0x35, - 0xb4, 0x0e, 0x99, 0xa7, 0xfd, 0x6e, 0x47, 0xed, 0xd7, 0x3f, 0x6e, 0x1e, 0x54, 0x65, 0x89, 0xfd, - 0xa1, 0x82, 0xe5, 0x2f, 0x39, 0x46, 0x47, 0xd3, 0xe8, 0xd6, 0x0f, 0x0f, 0x9a, 0x9d, 0x41, 0x5f, - 0xdd, 0x6f, 0x7d, 0x6f, 0x40, 0x73, 0x5a, 0x7c, 0xef, 0x1f, 0x25, 0x48, 0xd2, 0x12, 0xd6, 0xb4, - 0xc7, 0xe8, 0x3b, 0x00, 0x0b, 0xa4, 0x46, 0xe2, 0xef, 0x3b, 0x2b, 0x80, 0x5e, 0x2c, 0xac, 0x32, - 0x44, 0x60, 0x6a, 0x90, 0x09, 0xa1, 0x32, 0x12, 0x82, 0xab, 0x38, 0x5f, 0xbc, 0x7d, 0x0e, 0x47, - 0xd8, 0xf8, 0x10, 0x12, 0x1c, 0xbb, 0xd0, 0xe6, 0x32, 0x92, 0x71, 0xcd, 0xad, 0xf3, 0xe0, 0x6d, - 0xef, 0xfb, 0x90, 0x0d, 0x47, 0x1e, 0x3d, 0x85, 0x38, 0xff, 0xb8, 0x73, 0xe1, 0xf6, 0x2d, 0x6e, - 0x5f, 0xbc, 0x72, 0xee, 0x49, 0x1f, 0x48, 0xb5, 0x47, 0x9f, 0xff, 0xc7, 0xf6, 0xda, 0xe7, 0xbf, - 0xdc, 0x96, 0x7e, 0xf1, 0xcb, 0x6d, 0xe9, 0xb3, 0xff, 0xdc, 0x96, 0xbe, 0xff, 0xee, 0xd8, 0x24, - 0x47, 0xb3, 0x61, 0x45, 0x77, 0x26, 0xbb, 0xd8, 0x23, 0x33, 0xcd, 0x9d, 0xf3, 0xbf, 0x17, 0xae, - 0xfc, 0xe1, 0x70, 0x98, 0x60, 0xed, 0x0f, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x9c, 0x1f, 0x52, - 0xe6, 0x8c, 0x28, 0x00, 0x00, + // 3764 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x5a, 0x4d, 0x8c, 0x1b, 0xc9, + 0x75, 0x9e, 0xe6, 0x3f, 0x1f, 0xc9, 0x99, 0x9e, 0x9a, 0x91, 0x44, 0x71, 0x57, 0xc3, 0x11, 0x1d, + 0x63, 0xb5, 0x2b, 0x2f, 0x67, 0x3d, 0x0b, 0x3b, 0x2b, 0x09, 0x8a, 0xc1, 0x21, 0x39, 0x5a, 0x4a, + 0x1c, 0x92, 0x29, 0x72, 0xbc, 0xd6, 0x02, 0x46, 0xa3, 0xd9, 0x5d, 0xe4, 0xb4, 0x86, 0xec, 0x66, + 0xba, 0x8b, 0xa3, 0xa1, 0x2f, 0x09, 0x02, 0x03, 0x01, 0x16, 0x09, 0xe2, 0x4b, 0x60, 0xdf, 0xbc, + 0x31, 0x90, 0x9f, 0x73, 0x6e, 0x49, 0x2e, 0x06, 0x72, 0xd9, 0x53, 0x60, 0x24, 0xb7, 0x20, 0xd0, + 0x22, 0xce, 0xc1, 0x3e, 0x25, 0x77, 0x9d, 0x82, 0xfa, 0xe9, 0x66, 0x73, 0x38, 0x9a, 0x9f, 0xb5, + 0x73, 0xf0, 0x85, 0xe8, 0x7a, 0x7f, 0x55, 0xf5, 0x5e, 0xd5, 0xf7, 0x5e, 0x55, 0x11, 0xde, 0x1e, + 0x3a, 0x3b, 0x13, 0xd7, 0xa1, 0x8e, 0xe1, 0x8c, 0xbc, 0x9d, 0xc1, 0xc8, 0x79, 0xc9, 0x7f, 0xca, + 0x9c, 0x86, 0x62, 0xec, 0xbb, 0xb0, 0xd5, 0x77, 0x9d, 0x63, 0xe2, 0x06, 0x72, 0xc1, 0x87, 0x90, + 0x2a, 0x6c, 0x1b, 0x8e, 0xed, 0x4d, 0xc7, 0x17, 0x48, 0x6c, 0x0e, 0x9d, 0xa1, 0xc3, 0x3f, 0x77, + 0xd8, 0x97, 0xa4, 0x16, 0x87, 0x8e, 0x33, 0x1c, 0x11, 0x21, 0xdc, 0x9f, 0x0e, 0x76, 0xa8, 0x35, + 0x26, 0x1e, 0xd5, 0xc7, 0x13, 0x21, 0x50, 0x7a, 0x0c, 0xf1, 0xee, 0xc8, 0x32, 0x08, 0xda, 0x84, + 0x78, 0x9f, 0x0c, 0x2d, 0x3b, 0xaf, 0x6c, 0x2b, 0xf7, 0x72, 0x58, 0x34, 0x90, 0x0a, 0x51, 0x62, + 0x9b, 0xf9, 0x08, 0xa7, 0xb1, 0xcf, 0x87, 0xd9, 0x5f, 0xfc, 0x75, 0x71, 0xe5, 0x47, 0x3f, 0x2b, + 0xae, 0xfc, 0xe4, 0x67, 0xc5, 0x95, 0x92, 0x0d, 0xe9, 0xc3, 0xc3, 0x46, 0xad, 0xa3, 0xbb, 0xd4, + 0x43, 0x08, 0x62, 0xb6, 0x63, 0x12, 0x6e, 0x21, 0x81, 0xf9, 0x37, 0x7a, 0x04, 0x71, 0x63, 0xe4, + 0x18, 0xc7, 0xdc, 0x44, 0x62, 0xef, 0xeb, 0xaf, 0x5f, 0x15, 0xef, 0x0e, 0x9d, 0xf2, 0x50, 0xff, + 0x01, 0xa1, 0x94, 0x94, 0x4d, 0x72, 0xb2, 0x63, 0x38, 0x2e, 0xd9, 0x19, 0x13, 0xcf, 0xd3, 0x87, + 0xa4, 0x5c, 0x65, 0xc2, 0x58, 0xe8, 0x3c, 0x54, 0x7f, 0xfd, 0x79, 0x51, 0x59, 0xe8, 0xef, 0x2f, + 0x23, 0x00, 0x1d, 0xd7, 0x79, 0x41, 0x0c, 0x6a, 0x39, 0x7c, 0x78, 0x13, 0xea, 0xf2, 0x0e, 0xd3, + 0x98, 0x7d, 0xb2, 0x69, 0x0c, 0x2c, 0x32, 0x12, 0x43, 0x4e, 0x63, 0xd1, 0x40, 0x05, 0x48, 0x91, + 0xd3, 0xc9, 0xc8, 0x32, 0x2c, 0x9a, 0x8f, 0x6e, 0x2b, 0xf7, 0x52, 0x38, 0x68, 0xa3, 0x7b, 0xa0, + 0x5a, 0x9e, 0x36, 0xd1, 0x5d, 0x6a, 0x31, 0x9b, 0xda, 0x31, 0x99, 0xe5, 0x63, 0x5c, 0x66, 0xd5, + 0xf2, 0x3a, 0x3e, 0xf9, 0x19, 0x99, 0xa1, 0xdf, 0x83, 0x55, 0x26, 0xe9, 0x5a, 0x63, 0xdd, 0x9d, + 0x71, 0xb9, 0x38, 0x97, 0xcb, 0x5a, 0x5e, 0x47, 0x10, 0x99, 0xd4, 0x87, 0x90, 0xb6, 0xec, 0x01, + 0x71, 0x89, 0x6d, 0x90, 0x7c, 0x62, 0x5b, 0xb9, 0x97, 0xd9, 0x5d, 0x2b, 0xf3, 0x80, 0x37, 0x7c, + 0xf2, 0x5e, 0xec, 0x8b, 0x57, 0xc5, 0x15, 0x3c, 0x97, 0x43, 0x1f, 0xc1, 0xda, 0x4b, 0xd7, 0xa2, + 0x44, 0x9b, 0xab, 0x26, 0xcf, 0x55, 0xc5, 0xab, 0x5c, 0x2e, 0x68, 0x97, 0xfe, 0x33, 0x01, 0xe9, + 0xa0, 0xc5, 0xa6, 0x4f, 0x67, 0x13, 0xe2, 0xe5, 0x95, 0xed, 0x28, 0x9b, 0x3e, 0x6f, 0xa0, 0x32, + 0x24, 0x3c, 0xea, 0x5a, 0xf6, 0x90, 0x4f, 0x3e, 0xb3, 0x7b, 0xf3, 0x8c, 0xd1, 0x72, 0x97, 0x73, + 0xb1, 0x94, 0xe2, 0x56, 0x2c, 0x3a, 0x22, 0xdc, 0x0f, 0xcc, 0x0a, 0x6b, 0xa0, 0x6d, 0xc8, 0x98, + 0xc4, 0x33, 0x5c, 0x6b, 0xc2, 0x1c, 0xc2, 0xe7, 0x9e, 0xc6, 0x61, 0x12, 0x7a, 0x04, 0x59, 0x93, + 0x0c, 0xf4, 0xe9, 0x88, 0x6a, 0x2f, 0x3c, 0xc7, 0xe6, 0xb3, 0x4f, 0xef, 0xbd, 0xfd, 0xfa, 0x55, + 0x31, 0x4f, 0x6c, 0xc3, 0x31, 0x2d, 0x7b, 0xb8, 0xc3, 0x18, 0x65, 0xac, 0xbf, 0x3c, 0x10, 0x01, + 0xc7, 0x49, 0xa9, 0x81, 0x6e, 0x42, 0xc2, 0x23, 0x86, 0x4b, 0x28, 0x9f, 0x79, 0x0a, 0xcb, 0x16, + 0x1b, 0x3c, 0x39, 0xb5, 0x3c, 0xea, 0xe5, 0x53, 0xdb, 0xca, 0xbd, 0xd5, 0xe5, 0xc1, 0xd7, 0x39, + 0x17, 0x4b, 0x29, 0xf4, 0x4d, 0x48, 0xda, 0x6c, 0xa7, 0x58, 0x46, 0x3e, 0xcd, 0x67, 0x7b, 0xeb, + 0xac, 0x42, 0x4b, 0xb0, 0xb1, 0x2f, 0x87, 0xee, 0x43, 0x5c, 0x77, 0x5d, 0x7d, 0x96, 0x07, 0xae, + 0x70, 0xe3, 0xac, 0x42, 0x85, 0x31, 0xb1, 0x90, 0x29, 0xfc, 0x54, 0x81, 0x84, 0xf0, 0x17, 0xba, + 0x0b, 0x59, 0xc3, 0xb1, 0x29, 0xb1, 0xa9, 0xc6, 0x1c, 0xcd, 0xbd, 0x9b, 0xc6, 0x19, 0x49, 0xeb, + 0xcd, 0x26, 0x84, 0xcd, 0x6a, 0xe0, 0xb8, 0x63, 0x9d, 0x4a, 0x5f, 0xca, 0x16, 0x7a, 0x17, 0x54, + 0x5f, 0xd5, 0x77, 0x0d, 0x9f, 0x77, 0x1a, 0xaf, 0x49, 0x7a, 0x5d, 0x92, 0xd1, 0x1d, 0x80, 0xb1, + 0x7e, 0xaa, 0x8d, 0x88, 0x3d, 0xa4, 0x47, 0xdc, 0xa7, 0x39, 0x9c, 0x1e, 0xeb, 0xa7, 0x4d, 0x4e, + 0x78, 0x1a, 0x4b, 0x29, 0x6a, 0xe4, 0x69, 0x2c, 0x15, 0x51, 0xa3, 0x4f, 0x63, 0xa9, 0xb8, 0x9a, + 0x28, 0xfc, 0x31, 0x24, 0xe5, 0x14, 0x51, 0x11, 0x32, 0x47, 0xba, 0xa7, 0x8d, 0x2d, 0xdb, 0x1a, + 0x4f, 0xc7, 0x7c, 0xa3, 0xa4, 0x30, 0x1c, 0xe9, 0xde, 0x81, 0xa0, 0xa0, 0x3c, 0x24, 0x7d, 0x26, + 0xdb, 0x31, 0x0a, 0xf6, 0x9b, 0x81, 0xaa, 0x7e, 0xca, 0xb9, 0xd1, 0xb9, 0xaa, 0xa0, 0x70, 0x55, + 0xc9, 0x8c, 0x49, 0x55, 0xd1, 0x2c, 0xfc, 0x50, 0x81, 0x38, 0xf7, 0x19, 0x7a, 0x0b, 0xd2, 0x63, + 0xcb, 0xd6, 0x2c, 0x4a, 0xc6, 0x9e, 0x44, 0x96, 0xd4, 0xd8, 0xb2, 0x1b, 0xac, 0x8d, 0x4a, 0x90, + 0x93, 0x3d, 0x48, 0x81, 0x08, 0xef, 0x23, 0x23, 0xfa, 0x10, 0x32, 0xcc, 0x40, 0xc0, 0x8f, 0x4a, + 0x03, 0x3e, 0xf3, 0x0e, 0x00, 0x63, 0x68, 0x62, 0xc9, 0xc7, 0xf8, 0x92, 0x4f, 0x33, 0x0a, 0x73, + 0xbd, 0x57, 0xda, 0x83, 0x84, 0x58, 0x1b, 0x28, 0x03, 0xc9, 0x46, 0xeb, 0xbb, 0x95, 0x66, 0xa3, + 0xa6, 0xae, 0xa0, 0x14, 0xc4, 0x0e, 0x0e, 0xbb, 0x3d, 0x55, 0x41, 0x49, 0x88, 0x1e, 0x54, 0x9e, + 0xab, 0x11, 0x94, 0x85, 0x54, 0xe3, 0xa0, 0xd3, 0x6c, 0x54, 0x1b, 0x3d, 0x35, 0x8a, 0x00, 0x12, + 0xd5, 0x4a, 0xab, 0xd5, 0xee, 0xa9, 0xb1, 0xd2, 0x73, 0xc8, 0xb4, 0x08, 0x7d, 0xe9, 0xb8, 0xc7, + 0x1d, 0xc7, 0xe5, 0x8b, 0xd4, 0x9e, 0x8e, 0xfb, 0xc4, 0x95, 0x93, 0x91, 0x2d, 0x06, 0x30, 0x3e, + 0x1e, 0x4b, 0xe4, 0x09, 0xda, 0x4c, 0x67, 0x32, 0xed, 0x8f, 0x2c, 0x43, 0xfa, 0x50, 0xb6, 0x4a, + 0xff, 0xb6, 0x06, 0xab, 0x55, 0x67, 0x34, 0x12, 0x58, 0xd6, 0x9d, 0x10, 0x03, 0x95, 0x20, 0x66, + 0xeb, 0x63, 0x81, 0xa0, 0xe9, 0xbd, 0xd5, 0xd7, 0xaf, 0x8a, 0x30, 0x97, 0xc0, 0x9c, 0x87, 0x6a, + 0xb0, 0x2e, 0xa0, 0xc2, 0x33, 0x8e, 0xc8, 0x58, 0x17, 0x3b, 0x2d, 0x75, 0x85, 0x9d, 0x96, 0xe1, + 0x6a, 0x5d, 0xae, 0x85, 0xf6, 0x40, 0x75, 0x89, 0x6e, 0x2e, 0x18, 0xc9, 0x5c, 0xc1, 0x08, 0x30, + 0x2d, 0x69, 0x43, 0x85, 0x28, 0x03, 0xc1, 0x28, 0xf7, 0x3b, 0xfb, 0x44, 0xb7, 0x21, 0x35, 0x9d, + 0x5a, 0xa6, 0xc6, 0x40, 0x59, 0xac, 0xf7, 0x24, 0x6b, 0x77, 0xa8, 0xcb, 0x16, 0xfc, 0x1c, 0x63, + 0x39, 0x2a, 0x7b, 0xf9, 0x38, 0xd7, 0x5c, 0x0b, 0xe8, 0xfb, 0x9c, 0x8c, 0x3e, 0x82, 0xcc, 0x24, + 0xc0, 0x78, 0x2f, 0x9f, 0xd8, 0x8e, 0xde, 0xcb, 0xec, 0xaa, 0x62, 0x53, 0xce, 0xc1, 0x5f, 0x82, + 0x68, 0x58, 0x94, 0xf9, 0x46, 0x37, 0x8e, 0x35, 0x4a, 0xc6, 0x93, 0x91, 0x4e, 0x89, 0x98, 0x56, + 0xf2, 0x2a, 0xbe, 0xd1, 0x8d, 0xe3, 0x9e, 0xd4, 0x42, 0x35, 0x40, 0xf3, 0xa1, 0xfa, 0xb6, 0x24, + 0x98, 0xdc, 0x28, 0x07, 0x79, 0xf7, 0xa9, 0x33, 0x75, 0x6d, 0x7d, 0xc4, 0x02, 0x87, 0xd7, 0x03, + 0x85, 0xc0, 0xca, 0x77, 0x00, 0x4c, 0xe2, 0x5a, 0x27, 0x3a, 0x47, 0xcb, 0x2c, 0xd7, 0x2e, 0x8a, + 0x49, 0x2c, 0x46, 0xbd, 0x5c, 0x0b, 0xc4, 0x70, 0x48, 0xa5, 0xf0, 0x0f, 0x19, 0x80, 0x39, 0x0b, + 0xf5, 0x60, 0xd5, 0x70, 0x6c, 0x9b, 0x18, 0xd4, 0x71, 0x05, 0xdc, 0x28, 0x1c, 0x0f, 0xdf, 0xbf, + 0xc4, 0x66, 0xb9, 0xea, 0x6b, 0xb1, 0x5d, 0x81, 0x73, 0x46, 0xb8, 0x89, 0x1e, 0x00, 0x83, 0xab, + 0x81, 0x35, 0x14, 0xbe, 0x8a, 0x5c, 0xc1, 0x57, 0x09, 0xa1, 0x80, 0xda, 0x00, 0xd4, 0xd5, 0x6d, + 0x8f, 0x21, 0x9a, 0xc7, 0x57, 0x41, 0x66, 0xf7, 0xdd, 0xcb, 0x06, 0xd3, 0xf3, 0x35, 0x64, 0xf8, + 0x42, 0x26, 0xd0, 0x27, 0xb0, 0xee, 0x1d, 0x4d, 0x07, 0x83, 0x11, 0x61, 0xc9, 0x35, 0xb4, 0xab, + 0x57, 0x77, 0xef, 0x5f, 0x66, 0xb7, 0x2b, 0x14, 0xf9, 0x14, 0xd7, 0xa4, 0x95, 0x67, 0x64, 0xc6, + 0x81, 0x00, 0x3d, 0x84, 0x55, 0xef, 0x48, 0x77, 0xcd, 0x79, 0x30, 0xe3, 0x3c, 0x1c, 0x1b, 0x65, + 0xbf, 0xac, 0x2a, 0x77, 0x19, 0x9f, 0x87, 0x32, 0xc7, 0x45, 0x83, 0x30, 0x36, 0xe0, 0x86, 0x4b, + 0x0c, 0xe7, 0x84, 0xb8, 0x33, 0x6d, 0xe4, 0x0c, 0xe7, 0x26, 0x12, 0x17, 0xad, 0x87, 0x0d, 0x5f, + 0xa7, 0xe9, 0x0c, 0x03, 0x53, 0xdf, 0x86, 0x9c, 0x2d, 0xb0, 0x44, 0x9b, 0x38, 0x2e, 0xf5, 0xf2, + 0x49, 0xee, 0xb3, 0x75, 0x31, 0xb7, 0x10, 0xcc, 0xe0, 0xac, 0x3d, 0x6f, 0x78, 0xe8, 0x53, 0xd8, + 0xb0, 0x6c, 0xdd, 0xa0, 0xd6, 0x09, 0xd1, 0x42, 0x1e, 0x4f, 0x5d, 0xd3, 0xe3, 0x18, 0xf9, 0x56, + 0x02, 0x92, 0x57, 0xf8, 0x69, 0x1c, 0xd2, 0x41, 0x13, 0xdd, 0x5d, 0xc0, 0x9f, 0xdc, 0xeb, 0x57, + 0xc5, 0x39, 0x53, 0xc2, 0xcf, 0x43, 0x00, 0x23, 0xe8, 0x8b, 0xaf, 0x97, 0xcc, 0xee, 0xe6, 0x79, + 0x63, 0xf0, 0x03, 0x3c, 0x97, 0x46, 0xcd, 0xf0, 0xc6, 0xf2, 0xc8, 0x88, 0x2f, 0x43, 0x59, 0x93, + 0xdc, 0x9a, 0x3b, 0xb2, 0xa9, 0xf7, 0xc9, 0xa8, 0x2b, 0xd9, 0xd2, 0xcc, 0x7c, 0x83, 0xf9, 0x0c, + 0x81, 0xb9, 0x96, 0xe3, 0x5a, 0x54, 0x14, 0x6c, 0x39, 0x1c, 0xb4, 0xd1, 0x37, 0x00, 0x71, 0x78, + 0x33, 0xc9, 0x48, 0x9f, 0x69, 0x1e, 0x31, 0x1c, 0x9b, 0xe3, 0x0d, 0x93, 0xe2, 0xc0, 0x57, 0x63, + 0x8c, 0xae, 0xa0, 0xb3, 0x54, 0x17, 0x5a, 0x78, 0x1c, 0x70, 0xd2, 0x18, 0xe6, 0xab, 0x08, 0x1d, + 0x42, 0xc1, 0x17, 0x18, 0xe9, 0xe3, 0xbe, 0xa9, 0x6b, 0xe1, 0x4d, 0x73, 0x15, 0x80, 0xd9, 0x90, + 0xfa, 0x4d, 0xae, 0x5e, 0x15, 0x3b, 0x68, 0x1f, 0xd0, 0x39, 0xe6, 0xae, 0x82, 0xe5, 0xd9, 0x51, + 0xd8, 0xce, 0x5b, 0x90, 0xe6, 0xb3, 0x75, 0xec, 0xd1, 0x8c, 0xe3, 0x54, 0x0a, 0xa7, 0x18, 0xa1, + 0x6d, 0x8f, 0x66, 0xa8, 0x0c, 0x1b, 0x2f, 0xc4, 0xca, 0xd4, 0x04, 0xe2, 0x4f, 0x07, 0x03, 0xeb, + 0x94, 0x97, 0x3a, 0x69, 0xbc, 0x2e, 0x59, 0x98, 0xa1, 0x3a, 0x67, 0xa0, 0x07, 0x00, 0xb6, 0x43, + 0xb5, 0x3e, 0x19, 0x38, 0x2e, 0xe1, 0x39, 0x21, 0xb3, 0x5b, 0x28, 0x8b, 0x73, 0x44, 0xd9, 0x3f, + 0x47, 0x94, 0x7b, 0xfe, 0x39, 0x02, 0xa7, 0x6d, 0x87, 0xee, 0x71, 0x61, 0xf4, 0xfb, 0xc0, 0x1a, + 0x9a, 0x3e, 0xa0, 0xc4, 0x95, 0x88, 0x77, 0x91, 0x66, 0xca, 0x76, 0x68, 0x85, 0xc9, 0xb2, 0x50, + 0xf6, 0x75, 0xe3, 0x78, 0x60, 0x8d, 0x46, 0xf9, 0x9c, 0x08, 0xa5, 0xdf, 0x2e, 0x7d, 0x1f, 0x72, + 0x0b, 0x08, 0x86, 0x0a, 0x70, 0x53, 0x26, 0x73, 0xad, 0xda, 0x6e, 0xb5, 0xea, 0xd5, 0x5e, 0x1b, + 0x6b, 0xbd, 0xe7, 0x9d, 0xba, 0xba, 0xc2, 0x52, 0x77, 0xf7, 0x0f, 0x9b, 0x8d, 0x5e, 0x5d, 0x55, + 0xd0, 0x2a, 0x00, 0xa3, 0x76, 0xab, 0xb8, 0xd1, 0xe9, 0xa9, 0x11, 0x94, 0x86, 0x78, 0xe3, 0xa0, + 0xf2, 0xa4, 0xae, 0x46, 0xd9, 0x67, 0xb3, 0x5d, 0xad, 0x34, 0xd5, 0x58, 0xe9, 0x00, 0x32, 0x21, + 0xec, 0x40, 0x79, 0xd8, 0xf4, 0x8d, 0x77, 0x3f, 0x3e, 0xdc, 0xdf, 0x6f, 0xd6, 0x7d, 0xd3, 0x19, + 0x48, 0xee, 0xb5, 0xdb, 0xcd, 0x7a, 0xa5, 0xa5, 0x2a, 0xa2, 0xa0, 0xe8, 0xd5, 0x9f, 0xd4, 0xb1, + 0x1a, 0xe1, 0x9d, 0xf6, 0x70, 0xa3, 0xf5, 0x44, 0x8d, 0x96, 0x7e, 0x1c, 0x81, 0x55, 0x9e, 0xc6, + 0xc4, 0x32, 0x65, 0xab, 0x1e, 0x41, 0xec, 0x98, 0xcc, 0xfc, 0x92, 0x9c, 0x7f, 0xb3, 0x9a, 0xe0, + 0x44, 0x1f, 0x4d, 0x09, 0xab, 0x79, 0x18, 0x55, 0xb6, 0x98, 0x23, 0x4c, 0xc7, 0x98, 0x8e, 0x89, + 0x4d, 0x65, 0x35, 0x19, 0xb4, 0xd1, 0x09, 0xdc, 0xe0, 0x79, 0x33, 0xbc, 0x58, 0xb4, 0xb1, 0x3e, + 0xe1, 0x10, 0x99, 0xd9, 0xfd, 0x86, 0xd8, 0x84, 0x8b, 0x9d, 0x8b, 0xa6, 0x58, 0x26, 0x4f, 0x3d, + 0xc7, 0x3e, 0xd0, 0x27, 0x75, 0x9b, 0xba, 0xb3, 0xbd, 0xb7, 0x3f, 0xfb, 0xf2, 0xa2, 0x74, 0x38, + 0x98, 0xab, 0x15, 0xea, 0x70, 0xeb, 0x0d, 0x56, 0xfc, 0x0a, 0x40, 0x9e, 0xbf, 0x58, 0x05, 0xb0, + 0x09, 0x71, 0x3e, 0x15, 0xff, 0xfc, 0xc5, 0x1b, 0x0f, 0x23, 0x1f, 0x29, 0xa5, 0x9f, 0x27, 0x20, + 0x53, 0xd5, 0x27, 0x74, 0xea, 0x12, 0x5e, 0xeb, 0x14, 0x17, 0xb0, 0x26, 0xf3, 0xfa, 0x55, 0x31, + 0x29, 0xd9, 0x12, 0x69, 0xf6, 0x97, 0x12, 0x5e, 0x84, 0x27, 0x3c, 0x3f, 0x89, 0xce, 0x6d, 0x5d, + 0x2b, 0xc5, 0x45, 0xaf, 0x91, 0xe2, 0xbe, 0x05, 0xa9, 0xbe, 0x65, 0x33, 0x11, 0x4f, 0x7a, 0xf9, + 0xf6, 0x72, 0xe7, 0x7b, 0x42, 0x02, 0x07, 0xa2, 0xac, 0xd6, 0xb1, 0x6c, 0x4a, 0xdc, 0x13, 0x7d, + 0x74, 0x06, 0x7b, 0xd6, 0x7c, 0xba, 0x0f, 0x3d, 0xcb, 0xa9, 0x29, 0xf1, 0x9b, 0xa7, 0xa6, 0xe4, + 0x6f, 0x9e, 0x9a, 0x52, 0x57, 0x4b, 0x4d, 0xfb, 0xb0, 0x1e, 0xa4, 0xa6, 0xc0, 0x53, 0xe9, 0xcb, + 0x3c, 0xa5, 0xfa, 0x3a, 0x92, 0xe0, 0x15, 0xfe, 0x57, 0x81, 0xa4, 0x6c, 0xa0, 0xa7, 0xb0, 0xe9, + 0x12, 0xcf, 0x99, 0xba, 0x06, 0x59, 0xc0, 0x45, 0xe5, 0x0a, 0x81, 0x5b, 0xf5, 0x35, 0x25, 0x32, + 0x7e, 0x0d, 0x72, 0x81, 0xad, 0x89, 0x4e, 0x8f, 0xe4, 0x76, 0xcb, 0xfa, 0xc4, 0x8e, 0x4e, 0x8f, + 0xce, 0xa4, 0xb4, 0xe8, 0xb5, 0x52, 0x5a, 0x18, 0xb9, 0x62, 0x8b, 0xc8, 0xc5, 0x60, 0xd9, 0xa3, + 0xac, 0x0c, 0xf5, 0xaf, 0x0a, 0xd2, 0x38, 0xc5, 0x09, 0xcf, 0xc8, 0xac, 0xb4, 0x7b, 0x16, 0xd6, + 0x16, 0xce, 0x28, 0x01, 0x56, 0x25, 0xe7, 0x58, 0x95, 0x2a, 0xfd, 0x5d, 0x06, 0x36, 0x0e, 0x74, + 0x4a, 0x5c, 0x4b, 0x1f, 0x59, 0x3f, 0xd0, 0x83, 0x63, 0xc3, 0x3b, 0x0b, 0x5b, 0x69, 0xe3, 0xf5, + 0xab, 0xe2, 0xda, 0x19, 0x31, 0xb9, 0xa5, 0x5a, 0x6f, 0xd8, 0x52, 0xef, 0x88, 0xd9, 0x9e, 0x63, + 0xfb, 0xff, 0x6d, 0x6b, 0x3d, 0x5e, 0xda, 0x5a, 0x77, 0xdf, 0x3c, 0x88, 0xe5, 0x2d, 0xf6, 0x3b, + 0x5e, 0xd2, 0xb5, 0xde, 0xbc, 0x6f, 0xae, 0xe0, 0x86, 0xe5, 0xfd, 0xf3, 0xaf, 0x89, 0xdf, 0xd1, + 0xfd, 0x73, 0x7e, 0x49, 0x98, 0xfc, 0x2d, 0x94, 0x84, 0xe9, 0x33, 0x25, 0x61, 0x15, 0xd6, 0x44, + 0xfa, 0xf4, 0xfc, 0xe4, 0xc8, 0x37, 0x6c, 0x30, 0xd4, 0xc5, 0xc4, 0x29, 0xfb, 0x58, 0x1d, 0x2c, + 0xe6, 0xf2, 0xaf, 0x41, 0xce, 0x24, 0x23, 0xaa, 0x6b, 0xd3, 0x89, 0xa9, 0x53, 0xe2, 0xf9, 0x37, + 0x80, 0x9c, 0x78, 0x28, 0x68, 0xa8, 0x0f, 0xc8, 0x24, 0x13, 0x97, 0x18, 0x3a, 0x25, 0xa6, 0x26, + 0x0b, 0x3f, 0xb9, 0xb8, 0x3e, 0xbc, 0x34, 0xba, 0xe5, 0x5a, 0xa0, 0x2b, 0x0b, 0x13, 0xbc, 0x6e, + 0x9e, 0x25, 0xbd, 0xa9, 0xaa, 0x4b, 0x5d, 0xad, 0xaa, 0x83, 0xaf, 0x5c, 0xd5, 0x65, 0xbe, 0x62, + 0x55, 0x97, 0xbd, 0x08, 0x1b, 0x73, 0x8b, 0xd8, 0x58, 0xf8, 0x13, 0x05, 0xd6, 0x97, 0xbc, 0x80, + 0xee, 0x00, 0x0c, 0x5d, 0x67, 0x3a, 0xd1, 0xe6, 0x58, 0x87, 0xd3, 0x9c, 0xd2, 0x62, 0xd8, 0xf6, + 0x5b, 0x3d, 0x5c, 0x94, 0x1e, 0x5f, 0x0d, 0x9e, 0x53, 0x73, 0x78, 0x4e, 0xb3, 0xcf, 0x5a, 0xfd, + 0x59, 0x65, 0x5f, 0x85, 0xd2, 0xff, 0xa4, 0x21, 0xd1, 0xae, 0x4c, 0xe9, 0xd1, 0xae, 0xbc, 0x1a, + 0x3a, 0xb1, 0x4c, 0xe2, 0x5f, 0x54, 0x07, 0x6d, 0xf4, 0x1e, 0xac, 0xeb, 0x53, 0x7a, 0xa4, 0x4d, + 0xdd, 0xd1, 0x1c, 0x85, 0x44, 0xe5, 0xb4, 0xc6, 0x18, 0x87, 0xee, 0x28, 0x80, 0x9a, 0x07, 0x70, + 0x5b, 0x37, 0x0c, 0xe2, 0x79, 0x1a, 0x75, 0x8e, 0x89, 0xbd, 0xa8, 0x23, 0x6a, 0xc5, 0x9b, 0x42, + 0xa0, 0xc7, 0xf8, 0x61, 0xd5, 0x32, 0x6c, 0x2c, 0xa8, 0x8e, 0x09, 0x3d, 0x72, 0x4c, 0x71, 0xdf, + 0x83, 0xd7, 0x43, 0x4a, 0x07, 0x9c, 0xc1, 0x87, 0x15, 0x96, 0xef, 0x3b, 0xe6, 0x4c, 0xde, 0xe7, + 0xac, 0x85, 0xa4, 0xf7, 0x1c, 0x73, 0x86, 0x7e, 0xa8, 0xc0, 0x9d, 0x05, 0xe1, 0x23, 0xa2, 0x9b, + 0xc4, 0xf5, 0xe6, 0xe5, 0x69, 0x9c, 0xc3, 0x9a, 0x3c, 0xc1, 0x0b, 0xa7, 0x94, 0x2b, 0x73, 0x2b, + 0x1f, 0x0b, 0xf9, 0x6b, 0x54, 0xa7, 0x48, 0x5f, 0xd2, 0x46, 0x9f, 0x29, 0xb0, 0xb5, 0x30, 0x0c, + 0x97, 0x78, 0x13, 0xc7, 0xf6, 0xc8, 0x7c, 0x1c, 0x89, 0x70, 0x99, 0xbc, 0x3c, 0x0e, 0x2c, 0x15, + 0xae, 0x31, 0x90, 0xb0, 0xbf, 0x7d, 0xf5, 0x03, 0x7d, 0x82, 0x1e, 0x41, 0xc1, 0x25, 0x03, 0x97, + 0x78, 0x47, 0xe7, 0xc5, 0x4a, 0x5c, 0xf3, 0xde, 0x92, 0x12, 0x4b, 0xc1, 0xfa, 0x80, 0xc1, 0x77, + 0x58, 0x59, 0x46, 0x2b, 0xcb, 0xd5, 0x50, 0x58, 0x4d, 0x86, 0x8b, 0x1f, 0x76, 0xc3, 0x1a, 0x3c, + 0x5e, 0x02, 0x0a, 0xd4, 0xb0, 0x3c, 0x0f, 0xd8, 0x9f, 0x29, 0xb0, 0xb5, 0x28, 0xbe, 0x14, 0xb1, + 0xf4, 0x39, 0x9e, 0xc2, 0x21, 0x3b, 0xd7, 0x0f, 0xd9, 0x86, 0xbb, 0xac, 0x8e, 0xfe, 0x42, 0x81, + 0xe2, 0xe2, 0x48, 0x96, 0x83, 0x06, 0x7c, 0x28, 0xef, 0xbf, 0x71, 0x28, 0x5f, 0x21, 0x6a, 0x0b, + 0x9e, 0x0f, 0x85, 0xad, 0xd0, 0x84, 0xad, 0x8b, 0xd7, 0xe5, 0x75, 0xce, 0x3b, 0x85, 0x03, 0x28, + 0x5e, 0xb2, 0xba, 0xae, 0x6b, 0xee, 0x92, 0x10, 0x5c, 0xcb, 0x5c, 0x0b, 0xb6, 0x2f, 0x73, 0xe3, + 0xb5, 0x4e, 0x77, 0xff, 0x18, 0x85, 0x54, 0x8f, 0x78, 0x94, 0xd7, 0xa3, 0x28, 0x5c, 0x8f, 0xca, + 0xd2, 0x73, 0x07, 0xe2, 0x1e, 0x25, 0x13, 0x71, 0xe0, 0x65, 0x75, 0x1a, 0x8f, 0xa8, 0xaf, 0x52, + 0xee, 0x52, 0x32, 0x91, 0x68, 0x2c, 0xe4, 0x0a, 0xbf, 0x8a, 0x40, 0x8c, 0x51, 0xd1, 0xb7, 0x58, + 0xaa, 0x20, 0x93, 0xf0, 0x9d, 0x67, 0xfe, 0x1c, 0xed, 0x32, 0x2f, 0x50, 0x53, 0x4c, 0x94, 0x03, + 0xf6, 0x1d, 0x00, 0xae, 0x66, 0xd9, 0x26, 0x39, 0x95, 0x2f, 0x98, 0xdc, 0x50, 0x83, 0x11, 0xce, + 0xbe, 0x66, 0x45, 0x97, 0x5f, 0xb3, 0x7c, 0x03, 0x9e, 0xe1, 0x4c, 0xfc, 0xa7, 0x30, 0x6e, 0xa0, + 0xcb, 0x08, 0xa8, 0xbc, 0x50, 0xf5, 0xc4, 0xcf, 0xbd, 0xb1, 0x0f, 0x57, 0x3a, 0x0f, 0x20, 0x67, + 0x3a, 0x86, 0xdc, 0x65, 0x27, 0xc4, 0x10, 0xd7, 0x4c, 0x97, 0xd4, 0x63, 0x31, 0xa6, 0x82, 0x1e, + 0x03, 0x04, 0x19, 0xca, 0xbb, 0x5a, 0x71, 0x14, 0x52, 0x28, 0x6d, 0x41, 0x8c, 0x7b, 0x04, 0x20, + 0xd1, 0x68, 0x3d, 0xa9, 0x77, 0x7b, 0xe2, 0xa2, 0xe4, 0xbb, 0x75, 0xdc, 0xd8, 0x7f, 0xae, 0x2a, + 0xa5, 0x1f, 0x2b, 0x90, 0xc6, 0xba, 0x3d, 0x14, 0xe7, 0xf2, 0xb7, 0x20, 0x7d, 0x4c, 0x66, 0x9a, + 0x78, 0x0c, 0x66, 0x6e, 0x4b, 0xe2, 0xd4, 0x31, 0x99, 0xed, 0xf1, 0xf7, 0xe0, 0x5b, 0x90, 0x64, + 0x4c, 0x62, 0x9b, 0xdc, 0x63, 0x49, 0x9c, 0x38, 0x26, 0xb3, 0xba, 0x6d, 0xa2, 0x12, 0xe4, 0x5c, + 0x8d, 0xbf, 0xda, 0x4a, 0xcd, 0x18, 0x67, 0x67, 0x5c, 0xfe, 0xa0, 0x2b, 0x94, 0xb7, 0x20, 0xe3, + 0xcb, 0x30, 0x03, 0x71, 0x2e, 0x91, 0x16, 0x12, 0x75, 0xdb, 0x7c, 0xa8, 0xfe, 0xe4, 0xf3, 0xe2, + 0xca, 0x99, 0xe7, 0xe5, 0xd5, 0x20, 0x0b, 0x77, 0x59, 0x75, 0x80, 0x1e, 0x41, 0x56, 0x94, 0x5e, + 0xe6, 0xd5, 0x8b, 0xda, 0xa4, 0xd4, 0x40, 0x45, 0xc8, 0x8c, 0x89, 0x3b, 0xe4, 0xa5, 0xac, 0x71, + 0x24, 0x9f, 0x9b, 0x80, 0x93, 0x3a, 0x8c, 0x52, 0xfa, 0x5b, 0x05, 0xd2, 0x5d, 0xe2, 0x76, 0x9c, + 0x91, 0x65, 0xf0, 0x4b, 0x44, 0x8f, 0xba, 0x1a, 0x75, 0xa7, 0x36, 0x2b, 0x44, 0x64, 0x05, 0x24, + 0x1e, 0x7e, 0x54, 0x8f, 0xba, 0x3d, 0xc9, 0x10, 0xd5, 0xce, 0x03, 0xb8, 0x6d, 0x13, 0x8f, 0x0d, + 0xcc, 0xe9, 0xbf, 0x38, 0xab, 0x24, 0x96, 0xdf, 0x4d, 0x21, 0xd0, 0xee, 0xbf, 0x58, 0x54, 0xfd, + 0x00, 0x36, 0xf9, 0xdb, 0xe2, 0x59, 0x2d, 0xf1, 0xde, 0x85, 0x38, 0x6f, 0x41, 0xa3, 0xf4, 0x37, + 0x11, 0x80, 0xfa, 0x29, 0x75, 0x75, 0x83, 0x56, 0x3a, 0x8d, 0xc2, 0x3f, 0x2b, 0x90, 0x90, 0x15, + 0x7b, 0xf8, 0x09, 0x46, 0x59, 0x7c, 0x82, 0x79, 0x0c, 0x99, 0xf0, 0x73, 0xcf, 0x55, 0xee, 0xfa, + 0x41, 0x28, 0x30, 0x58, 0x60, 0xfb, 0x41, 0x14, 0xd0, 0x13, 0xea, 0x7a, 0xf2, 0xd5, 0x27, 0xcd, + 0x29, 0x1d, 0xea, 0x2e, 0xbd, 0xda, 0xc4, 0xae, 0xfc, 0x6a, 0x53, 0x7a, 0x0e, 0xb1, 0xaa, 0x63, + 0x9e, 0x29, 0xb1, 0x72, 0x90, 0xae, 0xb6, 0x5b, 0xfb, 0x8d, 0x27, 0x87, 0xb8, 0x2e, 0x2e, 0xdc, + 0xea, 0xdf, 0xeb, 0xe1, 0x4a, 0xb5, 0xa7, 0x46, 0x10, 0x82, 0x55, 0xd9, 0xa8, 0xd7, 0xb4, 0xc3, + 0xc3, 0x46, 0x4d, 0x8d, 0xa2, 0x4d, 0x50, 0xe7, 0xb4, 0xfd, 0x46, 0xbd, 0x59, 0xeb, 0xaa, 0xb1, + 0xd2, 0xaf, 0x14, 0x48, 0xed, 0x4d, 0xad, 0x91, 0xc9, 0xbc, 0xf4, 0x1f, 0x0b, 0x5e, 0xea, 0x33, + 0xb2, 0x66, 0x99, 0xbe, 0x97, 0x78, 0xbb, 0x61, 0xce, 0x59, 0x66, 0x5f, 0xc2, 0x9c, 0x60, 0xd5, + 0xfa, 0xfc, 0x89, 0x9a, 0x1f, 0x7b, 0x24, 0x5c, 0xc8, 0x16, 0xda, 0x85, 0x8c, 0x3c, 0x23, 0x71, + 0x8c, 0x8a, 0x71, 0x8c, 0x5a, 0xf7, 0x4f, 0x40, 0xc1, 0x63, 0x30, 0x06, 0x21, 0xc5, 0x37, 0xe3, + 0x7d, 0x58, 0x9f, 0x1f, 0xc5, 0xe5, 0xd9, 0x50, 0x5e, 0x12, 0xa8, 0x01, 0x43, 0x1e, 0x20, 0xd1, + 0x5d, 0xc8, 0x4a, 0x87, 0x69, 0xae, 0xe3, 0x50, 0xf1, 0xb0, 0x1e, 0x38, 0x11, 0x3b, 0x0e, 0x2d, + 0x6d, 0xc0, 0x3a, 0x26, 0x1e, 0xa1, 0x7c, 0x9b, 0x60, 0xf2, 0x47, 0x53, 0xe2, 0xd1, 0xd2, 0x26, + 0xa0, 0x30, 0x51, 0x60, 0x7c, 0xe9, 0x31, 0xa0, 0x8a, 0x79, 0xa2, 0xdb, 0x06, 0x61, 0x55, 0xbb, + 0x94, 0x45, 0xef, 0xc0, 0x9a, 0x2e, 0xa8, 0xc1, 0x9d, 0x15, 0xf3, 0x4c, 0x0c, 0xaf, 0x4a, 0xb2, + 0xbc, 0xb2, 0x2a, 0xdd, 0x80, 0x8d, 0x05, 0x75, 0x69, 0xf5, 0x23, 0xc8, 0xd6, 0x1c, 0xc3, 0xab, + 0xd8, 0xe6, 0xde, 0x8c, 0x9d, 0x82, 0x10, 0x70, 0xf0, 0x92, 0xfb, 0x45, 0x00, 0xd9, 0x26, 0xc4, + 0xfb, 0x8c, 0xc9, 0x1d, 0x1b, 0xc3, 0xa2, 0x51, 0xfa, 0x2b, 0x05, 0x72, 0x0d, 0x7b, 0x48, 0x3c, + 0xea, 0x8f, 0x65, 0x11, 0x5b, 0x95, 0x4b, 0xb1, 0x35, 0x1c, 0xce, 0xc8, 0x62, 0x38, 0x97, 0x60, + 0x37, 0x7a, 0x55, 0xd8, 0x2d, 0xfd, 0x7b, 0x04, 0x56, 0xfd, 0x71, 0x89, 0x49, 0xa2, 0x7f, 0x52, + 0xe6, 0xe7, 0x2e, 0xf1, 0x0a, 0xcb, 0x8a, 0x27, 0x71, 0xb7, 0x1b, 0xd4, 0xb8, 0x8b, 0x3a, 0xfe, + 0x25, 0xc2, 0x27, 0x4c, 0x9c, 0xa5, 0x6d, 0x4f, 0x14, 0x29, 0xdf, 0xff, 0xd3, 0x2f, 0x8b, 0xf7, + 0xcf, 0xfb, 0x2b, 0xcc, 0x99, 0xbf, 0x02, 0xf9, 0xfa, 0x9f, 0x7d, 0x59, 0x7c, 0xef, 0x2a, 0xe2, + 0xed, 0xc1, 0xc0, 0x23, 0x34, 0x38, 0x03, 0xce, 0xbb, 0x45, 0x7f, 0x00, 0x59, 0x7f, 0xec, 0x84, + 0x1a, 0xa6, 0x7c, 0xbc, 0x09, 0x5d, 0x77, 0x88, 0xa2, 0xa2, 0x5c, 0xa7, 0x86, 0xe9, 0xef, 0x53, + 0xa9, 0xc0, 0x48, 0x85, 0x1a, 0xdc, 0x3c, 0x7f, 0x2e, 0x97, 0x55, 0x0a, 0xd1, 0x70, 0xa5, 0xf0, + 0xf7, 0x11, 0xb8, 0xd5, 0xd3, 0xbd, 0x63, 0xff, 0x72, 0xc4, 0x75, 0x4e, 0x67, 0x7e, 0xdc, 0xbf, + 0x0d, 0x31, 0x67, 0x42, 0x44, 0xc4, 0x33, 0xbb, 0x25, 0x99, 0xe5, 0xcf, 0x17, 0x2e, 0xb7, 0x27, + 0xc4, 0xc6, 0x5c, 0x9e, 0xaf, 0x35, 0x9d, 0xea, 0xbc, 0xb3, 0x2c, 0xe6, 0xdf, 0x85, 0x9f, 0x2b, + 0x10, 0x63, 0x22, 0xe8, 0x1e, 0x24, 0x44, 0x85, 0x2b, 0xcd, 0xaa, 0x67, 0x27, 0x8c, 0x25, 0x1f, + 0x35, 0x21, 0x25, 0x2e, 0x95, 0xfc, 0x65, 0xb4, 0xf7, 0xcd, 0xd7, 0xaf, 0x8a, 0xef, 0x9f, 0xe7, + 0xf5, 0xa5, 0xff, 0x63, 0x89, 0xeb, 0xa6, 0x46, 0x0d, 0x27, 0xb9, 0x89, 0x06, 0xcf, 0x36, 0x54, + 0x77, 0x87, 0x84, 0xf2, 0xab, 0x21, 0x09, 0xe6, 0x20, 0x48, 0xfc, 0xcf, 0x04, 0x45, 0xc8, 0x18, + 0x23, 0x8b, 0xd8, 0x54, 0xd3, 0x4d, 0xd3, 0x7f, 0x30, 0x07, 0x41, 0xaa, 0x98, 0xa6, 0x5b, 0xfa, + 0xf3, 0x28, 0xe4, 0x97, 0x67, 0x2f, 0x97, 0x62, 0x07, 0x72, 0x6c, 0xee, 0x41, 0xcd, 0x2c, 0x67, + 0x77, 0xff, 0x4d, 0x4e, 0x93, 0xab, 0x91, 0x7b, 0x4d, 0x36, 0x70, 0xd6, 0x09, 0xb5, 0xce, 0xf5, + 0xe2, 0x4b, 0xc8, 0x86, 0x35, 0xd0, 0x63, 0x48, 0xb0, 0x63, 0xfa, 0xd4, 0x93, 0x95, 0xd8, 0xd7, + 0x2f, 0xe9, 0xae, 0xcb, 0x85, 0xb1, 0x54, 0x0a, 0xc5, 0x22, 0x72, 0x71, 0x2c, 0x4a, 0x3f, 0xe2, + 0x7f, 0xb3, 0xe1, 0x4a, 0x09, 0x88, 0xb4, 0x9f, 0xa9, 0x2b, 0x68, 0x03, 0xd6, 0xba, 0x1f, 0x57, + 0x70, 0x4d, 0x6b, 0xb5, 0x7b, 0xda, 0x7e, 0xfb, 0xb0, 0x55, 0x53, 0x15, 0x86, 0xfb, 0xad, 0xb6, + 0x26, 0xe8, 0x1d, 0xdc, 0x38, 0xa8, 0xe0, 0xe7, 0x6a, 0x04, 0xdd, 0x80, 0x75, 0x26, 0xb4, 0x48, + 0x8e, 0xb2, 0xc4, 0xd1, 0x68, 0xf5, 0xea, 0xb8, 0x55, 0x69, 0x6a, 0x75, 0x8c, 0xdb, 0x58, 0x8d, + 0xa1, 0x75, 0xc8, 0x09, 0xb1, 0x6e, 0xaf, 0xdd, 0xe9, 0xd4, 0x6b, 0x6a, 0x1c, 0xdd, 0x00, 0xb5, + 0xd3, 0xc6, 0x3d, 0xde, 0x4f, 0xa5, 0xd9, 0x6c, 0x7f, 0x52, 0xaf, 0xa9, 0xbf, 0x4e, 0xbe, 0xd7, + 0x82, 0x4c, 0x08, 0xcd, 0x59, 0x4a, 0xaa, 0x56, 0x7a, 0x95, 0x66, 0xfb, 0x89, 0xba, 0x82, 0xd6, + 0x20, 0xf3, 0xb4, 0xdb, 0x6e, 0x69, 0xdd, 0xea, 0xc7, 0xf5, 0x83, 0x8a, 0xaa, 0xf0, 0x3f, 0x91, + 0xf0, 0xfc, 0xa5, 0xc6, 0xd8, 0x68, 0x6a, 0xed, 0xea, 0xe1, 0x41, 0xbd, 0xd5, 0xeb, 0x6a, 0xfb, + 0x8d, 0xef, 0xf5, 0x58, 0x4e, 0x8b, 0xef, 0xfe, 0x8b, 0x02, 0x49, 0x56, 0xc2, 0x5a, 0xf6, 0x10, + 0x7d, 0x07, 0x60, 0x8e, 0xd4, 0x48, 0xfe, 0x65, 0x69, 0x09, 0xd0, 0x0b, 0xf9, 0x65, 0x86, 0x0c, + 0xcc, 0x1e, 0x64, 0x42, 0xa8, 0x8c, 0xa4, 0xe0, 0x32, 0xce, 0x17, 0x6e, 0x9f, 0xc3, 0x91, 0x36, + 0x3e, 0x84, 0x84, 0xc0, 0x2e, 0xb4, 0xb1, 0x88, 0x64, 0x42, 0x73, 0xf3, 0x3c, 0x78, 0xdb, 0xfd, + 0x14, 0xb2, 0xe1, 0xc8, 0xa3, 0xa7, 0x10, 0x17, 0x1f, 0x77, 0x2e, 0xdc, 0xbe, 0x85, 0xad, 0x8b, + 0x57, 0xce, 0x3d, 0xe5, 0x03, 0x65, 0xef, 0xd1, 0x17, 0xff, 0xb5, 0xb5, 0xf2, 0xc5, 0x2f, 0xb7, + 0x94, 0x5f, 0xfc, 0x72, 0x4b, 0xf9, 0xfc, 0xbf, 0xb7, 0x94, 0x4f, 0xdf, 0x1d, 0x5a, 0xf4, 0x68, + 0xda, 0x2f, 0x1b, 0xce, 0x78, 0x87, 0x78, 0x74, 0xaa, 0xbb, 0x33, 0xf1, 0x97, 0xca, 0xa5, 0x3f, + 0x59, 0xf6, 0x13, 0xbc, 0xfd, 0xe1, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x43, 0x43, 0xfd, 0x31, + 0x80, 0x29, 0x00, 0x00, } func (this *UUIDParts) Equal(that interface{}) bool { @@ -3474,6 +3487,20 @@ func (m *CollectionSpec_Derivation) MarshalToSizedBuffer(dAtA []byte) (int, erro i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.InactiveTransforms) > 0 { + for iNdEx := len(m.InactiveTransforms) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.InactiveTransforms[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintFlow(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + } if len(m.NetworkPorts) > 0 { for iNdEx := len(m.NetworkPorts) - 1; iNdEx >= 0; iNdEx-- { { @@ -3787,6 +3814,20 @@ func (m *CaptureSpec) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.InactiveBindings) > 0 { + for iNdEx := len(m.InactiveBindings) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.InactiveBindings[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintFlow(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } + } if len(m.NetworkPorts) > 0 { for iNdEx := len(m.NetworkPorts) - 1; iNdEx >= 0; iNdEx-- { { @@ -3955,6 +3996,20 @@ func (m *MaterializationSpec) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.InactiveBindings) > 0 { + for iNdEx := len(m.InactiveBindings) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.InactiveBindings[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintFlow(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } + } if len(m.NetworkPorts) > 0 { for iNdEx := len(m.NetworkPorts) - 1; iNdEx >= 0; iNdEx-- { { @@ -5588,6 +5643,12 @@ func (m *CollectionSpec_Derivation) ProtoSize() (n int) { n += 1 + l + sovFlow(uint64(l)) } } + if len(m.InactiveTransforms) > 0 { + for _, e := range m.InactiveTransforms { + l = e.ProtoSize() + n += 1 + l + sovFlow(uint64(l)) + } + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -5728,6 +5789,12 @@ func (m *CaptureSpec) ProtoSize() (n int) { n += 1 + l + sovFlow(uint64(l)) } } + if len(m.InactiveBindings) > 0 { + for _, e := range m.InactiveBindings { + l = e.ProtoSize() + n += 1 + l + sovFlow(uint64(l)) + } + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -5802,6 +5869,12 @@ func (m *MaterializationSpec) ProtoSize() (n int) { n += 1 + l + sovFlow(uint64(l)) } } + if len(m.InactiveBindings) > 0 { + for _, e := range m.InactiveBindings { + l = e.ProtoSize() + n += 1 + l + sovFlow(uint64(l)) + } + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -8320,6 +8393,40 @@ func (m *CollectionSpec_Derivation) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InactiveTransforms", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowFlow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthFlow + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthFlow + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.InactiveTransforms = append(m.InactiveTransforms, &CollectionSpec_Derivation_Transform{}) + if err := m.InactiveTransforms[len(m.InactiveTransforms)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipFlow(dAtA[iNdEx:]) @@ -9313,6 +9420,40 @@ func (m *CaptureSpec) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InactiveBindings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowFlow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthFlow + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthFlow + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.InactiveBindings = append(m.InactiveBindings, &CaptureSpec_Binding{}) + if err := m.InactiveBindings[len(m.InactiveBindings)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipFlow(dAtA[iNdEx:]) @@ -9786,6 +9927,40 @@ func (m *MaterializationSpec) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InactiveBindings", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowFlow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthFlow + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthFlow + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.InactiveBindings = append(m.InactiveBindings, &MaterializationSpec_Binding{}) + if err := m.InactiveBindings[len(m.InactiveBindings)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipFlow(dAtA[iNdEx:]) diff --git a/go/protocols/flow/flow.proto b/go/protocols/flow/flow.proto index 932bc1755c..5485a32009 100644 --- a/go/protocols/flow/flow.proto +++ b/go/protocols/flow/flow.proto @@ -232,7 +232,7 @@ message CollectionSpec { message Transform { // Next tag: 14. - // Stable name of this transform. + // Stable and unique name of this transform. string name = 1 [ (gogoproto.casttype) = "Transform" ]; // Source collection which is read by this transform. CollectionSpec collection = 2 [ (gogoproto.nullable) = false ]; @@ -285,6 +285,10 @@ message CollectionSpec { protocol.JournalSpec recovery_log_template = 6; // Network ports of this derivation. repeated NetworkPort network_ports = 7; + // Transforms which were previously active for this task, but are no longer. + // Inactive transforms are unique by their transform name, and have no + // overlap with active transforms. + repeated Transform inactive_transforms = 8; } Derivation derivation = 12; } @@ -324,9 +328,8 @@ message CaptureSpec { (gogoproto.casttype) = "encoding/json.RawMessage", json_name = "config" ]; - // Bindings of endpoint resources and collections into which they're captured. - // Bindings are ordered and unique on the bound collection name, - // and are also unique on the resource path. + // Bindings between endpoint resources, uniquely identified by their + // resource path, and the collections into which they're captured. message Binding { // Next tag: 6. @@ -355,6 +358,10 @@ message CaptureSpec { protocol.JournalSpec recovery_log_template = 7; // Network ports of this capture. repeated NetworkPort network_ports = 8; + // Bindings which were previously active for this task, but are no longer. + // Inactive bindings are unique by their resource path, and have no + // overlap with active bindings. + repeated Binding inactive_bindings = 9; } // MaterializationSpec describes a collection and its materialization to an @@ -375,9 +382,8 @@ message MaterializationSpec { (gogoproto.casttype) = "encoding/json.RawMessage", json_name = "config" ]; - // Bindings of endpoint resources and collections from which they're - // materialized. Bindings are ordered and unique on the bound collection name, - // and are also unique on the resource path. + // Bindings between endpoint resources, uniquely identified by their + // resource path, and the collections from which they're materialized. message Binding { // Next tag: 14. @@ -426,6 +432,10 @@ message MaterializationSpec { protocol.JournalSpec recovery_log_template = 6; // Network ports of this materialization. repeated NetworkPort network_ports = 7; + // Bindings which were previously active for this task, but are no longer. + // Inactive bindings are unique by their resource path, and have no + // overlap with active bindings. + repeated Binding inactive_bindings = 9; } // OAuth2 describes an OAuth2 provider diff --git a/go/protocols/materialize/materialize.pb.go b/go/protocols/materialize/materialize.pb.go index 50c119713f..b78d99f8d5 100644 --- a/go/protocols/materialize/materialize.pb.go +++ b/go/protocols/materialize/materialize.pb.go @@ -284,11 +284,14 @@ func (m *Request_Validate_Binding) XXX_DiscardUnknown() { var xxx_messageInfo_Request_Validate_Binding proto.InternalMessageInfo -// Apply a materialization configuration and bindings to its endpoint. -// Apply is run out-of-band with ongoing connector invocations, -// and may be run many times for a single materialization name, -// where each invocation has varying bindings, or even no bindings. -// The connector performs any required setup or cleanup. +// Apply an updated materialization specification to its endpoint, +// in preparation for an Open of a materialization session. +// Apply is run by the leader shard of a materialization task +// (having key_begin: 0) while the materialization is quiescent. +// Apply may be called multiple times for a given `version` and +// `last_version`, even if a prior call succeeded from the connector's +// perspective, so implementations must be idempotent. However, the next +// session will not Open until it's preceding Apply has durably completed. type Request_Apply struct { // Materialization to be applied. Materialization *flow.MaterializationSpec `protobuf:"bytes,1,opt,name=materialization,proto3" json:"materialization,omitempty"` @@ -297,10 +300,14 @@ type Request_Apply struct { // Last CaptureSpec which was successfully applied. LastMaterialization *flow.MaterializationSpec `protobuf:"bytes,4,opt,name=last_materialization,json=lastMaterialization,proto3" json:"last_materialization,omitempty"` // Version of the last applied MaterializationSpec. - LastVersion string `protobuf:"bytes,5,opt,name=last_version,json=lastVersion,proto3" json:"last_version,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + LastVersion string `protobuf:"bytes,5,opt,name=last_version,json=lastVersion,proto3" json:"last_version,omitempty"` + // Last-persisted connector checkpoint state from a previous session. + // The Apply RPC may use this state to perform a post-commit apply + // of files staged under the `last_materialization` specification. + StateJson encoding_json.RawMessage `protobuf:"bytes,6,opt,name=state_json,json=state,proto3,casttype=encoding/json.RawMessage" json:"state_json,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *Request_Apply) Reset() { *m = Request_Apply{} } @@ -360,9 +367,9 @@ type Request_Open struct { // provided to a successful Apply RPC. It's possible that it won't, // due to expected propagation races in Flow's distributed runtime. Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"` - // Range of documents to be processed by this invocation. + // Range of documents to be processed by this session. Range *flow.RangeSpec `protobuf:"bytes,3,opt,name=range,proto3" json:"range,omitempty"` - // Last-persisted connector checkpoint state from a previous invocation. + // Last-persisted connector checkpoint state from a previous session. StateJson encoding_json.RawMessage `protobuf:"bytes,4,opt,name=state_json,json=state,proto3,casttype=encoding/json.RawMessage" json:"state_json,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -689,15 +696,22 @@ type Response_Spec struct { Protocol uint32 `protobuf:"varint,1,opt,name=protocol,proto3" json:"protocol,omitempty"` // JSON schema of the connector's configuration. ConfigSchemaJson encoding_json.RawMessage `protobuf:"bytes,2,opt,name=config_schema_json,json=configSchema,proto3,casttype=encoding/json.RawMessage" json:"config_schema_json,omitempty"` - // JSON schema of the connecor's resource configuration. + // JSON schema of the connector's resource configuration. ResourceConfigSchemaJson encoding_json.RawMessage `protobuf:"bytes,3,opt,name=resource_config_schema_json,json=resourceConfigSchema,proto3,casttype=encoding/json.RawMessage" json:"resource_config_schema_json,omitempty"` - // URL for connector's documention. + // URL for connector's documentation. DocumentationUrl string `protobuf:"bytes,4,opt,name=documentation_url,json=documentationUrl,proto3" json:"documentation_url,omitempty"` // Optional OAuth2 configuration. - Oauth2 *flow.OAuth2 `protobuf:"bytes,5,opt,name=oauth2,proto3" json:"oauth2,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Oauth2 *flow.OAuth2 `protobuf:"bytes,5,opt,name=oauth2,proto3" json:"oauth2,omitempty"` + // One or more JSON pointers, which are used to extract resource paths + // from resource configurations of this connector. For example, + // a database connector might have a resource config like: + // {"schema": "foo", "table": "bar", "other": "config", "answer": 42} + // The connector would specify `resource_path_pointers: ["/schema", "/table"]`, + // which would result in a `resource_path` of `["foo", "bar"]`. + ResourcePathPointers []string `protobuf:"bytes,6,rep,name=resource_path_pointers,json=resourcePathPointers,proto3" json:"resource_path_pointers,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *Response_Spec) Reset() { *m = Response_Spec{} } @@ -889,10 +903,14 @@ var xxx_messageInfo_Response_Validated_Binding proto.InternalMessageInfo type Response_Applied struct { // Human-readable description of the action that the connector took. // If empty, this Apply is to be considered a "no-op". - ActionDescription string `protobuf:"bytes,1,opt,name=action_description,json=actionDescription,proto3" json:"action_description,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + ActionDescription string `protobuf:"bytes,1,opt,name=action_description,json=actionDescription,proto3" json:"action_description,omitempty"` + // Optional *transactional* update to ConnectorState. + // This update commits atomically with the Flow recovery log checkpoint + // which marks the current specification as having been applied. + State *flow.ConnectorState `protobuf:"bytes,2,opt,name=state,proto3" json:"state,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *Response_Applied) Reset() { *m = Response_Applied{} } @@ -1333,114 +1351,115 @@ func init() { } var fileDescriptor_3e8b62b327f34bc6 = []byte{ - // 1699 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0xcf, 0x6f, 0xdb, 0xc8, - 0x15, 0x0e, 0x65, 0xfd, 0x7c, 0x92, 0x6d, 0x79, 0xa2, 0xb4, 0x2c, 0x37, 0x9b, 0x38, 0xde, 0x5d, - 0xac, 0x91, 0x62, 0xe5, 0xc0, 0x69, 0xb1, 0x49, 0x16, 0x29, 0x2a, 0xc9, 0x12, 0xa0, 0x54, 0xb6, - 0xbc, 0xe3, 0x24, 0x0b, 0xec, 0x45, 0x98, 0x90, 0x63, 0x99, 0x31, 0xc5, 0x61, 0xc9, 0x51, 0xb2, - 0xea, 0xa9, 0x28, 0x50, 0x14, 0xe8, 0xad, 0x40, 0xd1, 0x4b, 0x51, 0xa0, 0xb7, 0xfe, 0x05, 0xbd, - 0xf4, 0x5a, 0x14, 0xc8, 0xb1, 0xb7, 0xde, 0x52, 0x74, 0xfb, 0x17, 0xf4, 0xba, 0x87, 0xa2, 0x98, - 0x1f, 0xa4, 0x28, 0x47, 0x92, 0xb5, 0x40, 0x9a, 0x8b, 0xc1, 0x79, 0xf3, 0x7d, 0x8f, 0x8f, 0x6f, - 0xde, 0xbc, 0xf7, 0xc9, 0x70, 0x7b, 0xc8, 0xf6, 0x82, 0x90, 0x71, 0x66, 0x33, 0x2f, 0xda, 0x1b, - 0x11, 0x4e, 0x43, 0x97, 0x78, 0xee, 0xcf, 0x68, 0xfa, 0xb9, 0x2e, 0x11, 0xa8, 0x9c, 0x32, 0x59, - 0xdb, 0x36, 0xf3, 0xa3, 0xf1, 0x88, 0x86, 0x09, 0x3d, 0x79, 0x50, 0x70, 0xeb, 0xfa, 0x8c, 0xeb, - 0x53, 0x8f, 0xbd, 0x94, 0x7f, 0xf4, 0x6e, 0x6d, 0xc8, 0x86, 0x4c, 0x3e, 0xee, 0x89, 0x27, 0x65, - 0xdd, 0xf9, 0xc5, 0x55, 0x28, 0x60, 0xfa, 0xd3, 0x31, 0x8d, 0x38, 0xfa, 0x04, 0xb2, 0x51, 0x40, - 0x6d, 0xd3, 0xd8, 0x36, 0x76, 0xcb, 0xfb, 0xdf, 0xab, 0xa7, 0x03, 0xd2, 0x98, 0xfa, 0x49, 0x40, - 0x6d, 0x2c, 0x61, 0xe8, 0x3e, 0x14, 0x5f, 0x10, 0xcf, 0x75, 0x08, 0xa7, 0x66, 0x46, 0x52, 0xde, - 0x9f, 0x4b, 0x79, 0xaa, 0x41, 0x38, 0x81, 0xa3, 0x3b, 0x90, 0x23, 0x41, 0xe0, 0x4d, 0xcc, 0x35, - 0xc9, 0xb3, 0xe6, 0xf2, 0x1a, 0x02, 0x81, 0x15, 0x50, 0xc4, 0xc6, 0x02, 0xea, 0x9b, 0xd9, 0x25, - 0xb1, 0xf5, 0x03, 0xea, 0x63, 0x09, 0x13, 0x70, 0x8f, 0x11, 0xc7, 0xcc, 0x2d, 0x81, 0xf7, 0x18, - 0x71, 0xb0, 0x84, 0x89, 0x78, 0x4e, 0xbd, 0x71, 0x74, 0x66, 0xe6, 0x97, 0xc4, 0xd3, 0x11, 0x08, - 0xac, 0x80, 0x82, 0x11, 0x71, 0x16, 0x52, 0xb3, 0xb0, 0x84, 0x71, 0x22, 0x10, 0x58, 0x01, 0x51, - 0x0b, 0x2a, 0x11, 0x27, 0x21, 0x1f, 0xd8, 0x6c, 0x34, 0x72, 0xb9, 0x59, 0x94, 0xc4, 0xed, 0x05, - 0x44, 0x12, 0xf2, 0x96, 0xc4, 0xe1, 0x72, 0x34, 0x5d, 0xa0, 0x26, 0x94, 0x89, 0x7d, 0xee, 0xb3, - 0x97, 0x1e, 0x75, 0x86, 0xd4, 0x2c, 0x2d, 0xf1, 0xd1, 0x98, 0xe2, 0x70, 0x9a, 0x84, 0xde, 0x83, - 0xa2, 0xeb, 0x73, 0x1a, 0xfa, 0xc4, 0x33, 0x9d, 0x6d, 0x63, 0xb7, 0x82, 0x4b, 0x1f, 0xc6, 0x06, - 0xeb, 0x37, 0x06, 0x64, 0xc5, 0x19, 0xa3, 0x23, 0xd8, 0xb0, 0x99, 0xef, 0x53, 0x9b, 0xb3, 0x70, - 0xc0, 0x27, 0x01, 0x95, 0x65, 0xb1, 0xb1, 0xff, 0x71, 0x5d, 0xd6, 0xd4, 0x61, 0xf2, 0x46, 0xc2, - 0x5d, 0xe6, 0x0b, 0x4a, 0xbd, 0x15, 0xe3, 0x1f, 0x4f, 0x02, 0x8a, 0xd7, 0xed, 0xf4, 0x12, 0xdd, - 0x87, 0xb2, 0xcd, 0xfc, 0x53, 0x77, 0x38, 0x78, 0x1e, 0x31, 0x5f, 0x16, 0x4c, 0xa9, 0x79, 0xfd, - 0x9b, 0xd7, 0x37, 0x4d, 0xea, 0xdb, 0xcc, 0x71, 0xfd, 0xe1, 0x9e, 0xd8, 0xa8, 0x63, 0xf2, 0xf2, - 0x90, 0x46, 0x11, 0x19, 0x52, 0x9c, 0x57, 0x04, 0xeb, 0x0f, 0x79, 0x28, 0xc6, 0x45, 0x84, 0x3e, - 0x87, 0xac, 0x4f, 0x46, 0x2a, 0x9a, 0x52, 0xf3, 0xe1, 0x37, 0xaf, 0x6f, 0xde, 0x1f, 0xba, 0xfc, - 0x6c, 0xfc, 0xac, 0x6e, 0xb3, 0xd1, 0x1e, 0x8d, 0xf8, 0x98, 0x84, 0x13, 0x55, 0xfc, 0x6f, 0x5c, - 0x87, 0x8b, 0x51, 0x63, 0xe9, 0x6a, 0xce, 0xa7, 0x66, 0xde, 0xe6, 0xa7, 0xae, 0xad, 0xfe, 0xa9, - 0xa8, 0x01, 0xc5, 0x67, 0xae, 0x2f, 0x20, 0x91, 0x99, 0xdd, 0x5e, 0xdb, 0x2d, 0xef, 0x7f, 0xb4, - 0xf4, 0x4e, 0xd5, 0x9b, 0x0a, 0x8d, 0x13, 0x1a, 0xea, 0x41, 0xcd, 0x23, 0x11, 0x1f, 0x8c, 0x66, - 0xc3, 0x4e, 0xae, 0xc2, 0xa2, 0x6f, 0xc2, 0x57, 0x05, 0xed, 0xc2, 0x06, 0xba, 0x05, 0x15, 0xe9, - 0xed, 0x05, 0x0d, 0x23, 0xe1, 0x45, 0x5c, 0x90, 0x12, 0x2e, 0x0b, 0xdb, 0x53, 0x65, 0xb2, 0xfe, - 0x9b, 0x81, 0x82, 0x0e, 0x03, 0x3d, 0x82, 0x5a, 0x48, 0x23, 0x36, 0x0e, 0x6d, 0x3a, 0x48, 0xe7, - 0xc0, 0x58, 0x21, 0x07, 0x1b, 0x31, 0xb3, 0xa5, 0x72, 0xf1, 0x00, 0xc0, 0x66, 0x9e, 0x47, 0x6d, - 0x19, 0xbe, 0xea, 0x30, 0x35, 0x15, 0x7e, 0x2b, 0xb1, 0x8b, 0xc8, 0x9b, 0xd9, 0x57, 0xaf, 0x6f, - 0x5e, 0xc1, 0x29, 0x34, 0xfa, 0x95, 0x01, 0xd7, 0x4e, 0x5d, 0xea, 0x39, 0xe9, 0x28, 0x06, 0x23, - 0x12, 0x98, 0x6b, 0x32, 0xab, 0x0f, 0x57, 0xca, 0x6a, 0xbd, 0x23, 0x5c, 0xa8, 0x70, 0x1e, 0x45, - 0xcc, 0x3f, 0x24, 0x41, 0xdb, 0xe7, 0xe1, 0xa4, 0x79, 0xfd, 0xd7, 0xff, 0x5c, 0xf2, 0x21, 0xe5, - 0xd3, 0x29, 0x0d, 0x59, 0x50, 0x7c, 0x46, 0xec, 0xf3, 0x53, 0xd7, 0xf3, 0x64, 0xf3, 0x5a, 0xc7, - 0xc9, 0xda, 0x6a, 0xc3, 0x77, 0x17, 0xbc, 0x01, 0x55, 0x61, 0xed, 0x9c, 0x4e, 0x54, 0xde, 0xb0, - 0x78, 0x44, 0x35, 0xc8, 0xbd, 0x20, 0xde, 0x58, 0x15, 0x67, 0x09, 0xab, 0xc5, 0x83, 0xcc, 0x3d, - 0xc3, 0xfa, 0x87, 0x01, 0x39, 0xd9, 0x2c, 0x51, 0x0b, 0x36, 0x2f, 0x1e, 0xbb, 0x71, 0xd9, 0xb1, - 0x5f, 0x64, 0x20, 0x13, 0x0a, 0xf1, 0x69, 0xab, 0x57, 0xc5, 0xcb, 0x85, 0xa5, 0x95, 0x7d, 0x2b, - 0xa5, 0x95, 0x7b, 0xb3, 0xb4, 0xfe, 0x66, 0x40, 0x56, 0x74, 0xf5, 0xff, 0xf7, 0x87, 0x7d, 0x04, - 0xb9, 0x90, 0xf8, 0x43, 0xaa, 0xe7, 0xd1, 0xa6, 0x72, 0x8a, 0x85, 0x49, 0xba, 0x52, 0xbb, 0xe8, - 0x53, 0x80, 0x88, 0x13, 0x4e, 0x55, 0x4d, 0x67, 0x57, 0xa8, 0xe9, 0x9c, 0xc4, 0x5b, 0x1c, 0xb2, - 0x62, 0xda, 0x88, 0x08, 0xf4, 0x3d, 0x95, 0xe1, 0xaf, 0xe3, 0x78, 0x89, 0xee, 0x42, 0xf1, 0x9c, - 0x4e, 0x56, 0xef, 0x8d, 0xb2, 0x24, 0xde, 0x07, 0x10, 0xa4, 0x80, 0xd8, 0xe7, 0xd4, 0x91, 0xb1, - 0x57, 0x70, 0xe9, 0x9c, 0x4e, 0x8e, 0xa5, 0xc1, 0x2a, 0x40, 0x4e, 0xce, 0x2c, 0xeb, 0x2f, 0x19, - 0xc8, 0xc9, 0x59, 0xf4, 0x6e, 0x03, 0x10, 0x8d, 0x50, 0x56, 0x69, 0xb4, 0x7a, 0xc2, 0xf2, 0x8a, - 0x80, 0x3e, 0x80, 0x75, 0x4d, 0xd5, 0xce, 0x73, 0xd2, 0x79, 0x45, 0x19, 0xb5, 0xff, 0xbb, 0x50, - 0x74, 0x98, 0xad, 0x9c, 0xe7, 0x57, 0x89, 0xd9, 0x61, 0x36, 0xfa, 0x0e, 0xe4, 0xe9, 0x57, 0x6e, - 0xc4, 0x23, 0x39, 0xba, 0x8b, 0x58, 0xaf, 0x84, 0xdd, 0xa1, 0x1e, 0xe5, 0x54, 0x4e, 0xe6, 0x22, - 0xd6, 0x2b, 0x0b, 0x43, 0x39, 0x35, 0x8e, 0x51, 0x0b, 0x50, 0x38, 0xf6, 0xb9, 0x3b, 0xa2, 0x03, - 0xfb, 0x8c, 0xda, 0xe7, 0x01, 0x73, 0x7d, 0xae, 0x8b, 0xb1, 0x56, 0x8f, 0x35, 0x5a, 0xbd, 0x95, - 0xec, 0xe1, 0x2d, 0x8d, 0x9f, 0x9a, 0xac, 0x75, 0x28, 0xa7, 0xc6, 0xf3, 0xce, 0x9f, 0x36, 0xa0, - 0x88, 0x69, 0x14, 0x30, 0x3f, 0xa2, 0xa8, 0x3e, 0xa3, 0xc2, 0x2e, 0x0a, 0x0b, 0x05, 0x4a, 0xcb, - 0xb0, 0x87, 0x50, 0x8a, 0x75, 0x95, 0xa3, 0xbb, 0xe4, 0xcd, 0xf9, 0xa4, 0xb8, 0xbd, 0x39, 0x78, - 0xca, 0x40, 0x9f, 0x42, 0x41, 0x28, 0x2c, 0x57, 0x9f, 0xdf, 0x9b, 0x22, 0x4e, 0x93, 0x1b, 0x0a, - 0x84, 0x63, 0x34, 0xfa, 0x01, 0xe4, 0x85, 0xd4, 0xa2, 0x8e, 0xbe, 0xfe, 0xd7, 0xe7, 0xf3, 0xfa, - 0x12, 0x83, 0x35, 0x56, 0xb0, 0x84, 0xe2, 0xa2, 0xb1, 0x34, 0x5b, 0xc0, 0xea, 0x49, 0x0c, 0xd6, - 0x58, 0x11, 0xa4, 0x94, 0x5d, 0xd4, 0xd1, 0x0a, 0x6d, 0x41, 0x90, 0x1d, 0x05, 0xc2, 0x31, 0x1a, - 0x3d, 0x82, 0x0d, 0x29, 0x9f, 0xa8, 0x13, 0xcb, 0x2e, 0xa5, 0xd7, 0x3e, 0x58, 0x90, 0x56, 0x85, - 0xd5, 0xca, 0x6b, 0x3d, 0x4a, 0x2f, 0x51, 0x07, 0x2a, 0x29, 0x19, 0xe5, 0x68, 0x01, 0xb7, 0xb3, - 0x20, 0x5d, 0x29, 0x24, 0x9e, 0xe1, 0x2d, 0xd7, 0x5f, 0xbf, 0xcb, 0x68, 0xfd, 0x65, 0x41, 0x31, - 0x16, 0x2f, 0xfa, 0xaa, 0x26, 0x6b, 0xd4, 0x01, 0xa4, 0xc7, 0x5a, 0x64, 0x9f, 0xd1, 0x11, 0x59, - 0xfd, 0xd6, 0x56, 0x14, 0xef, 0x44, 0xd2, 0xd0, 0x17, 0xf0, 0xde, 0xc5, 0x69, 0x9d, 0x76, 0xb8, - 0x8a, 0x70, 0xa9, 0xcd, 0x0e, 0x6d, 0xed, 0xf8, 0xfb, 0xb0, 0xe5, 0x30, 0x7b, 0x3c, 0xa2, 0x3e, - 0x97, 0xad, 0x77, 0x30, 0x0e, 0xd5, 0xf4, 0x2b, 0xe1, 0xea, 0xcc, 0xc6, 0x93, 0xd0, 0x43, 0x1f, - 0x42, 0x9e, 0x91, 0x31, 0x3f, 0xdb, 0xd7, 0x25, 0x51, 0x51, 0xdd, 0xb7, 0xdf, 0x10, 0x36, 0xac, - 0xf7, 0xac, 0xff, 0x64, 0xa1, 0x94, 0x14, 0x30, 0x6a, 0xa5, 0x74, 0x92, 0x21, 0x27, 0xfa, 0xc7, - 0x97, 0xd4, 0xfc, 0x9b, 0x4a, 0xc9, 0xfa, 0x79, 0x06, 0xa0, 0xc5, 0xfc, 0x88, 0x87, 0xc4, 0xf5, - 0xc5, 0xcd, 0xce, 0xa6, 0xc4, 0xdf, 0xde, 0x65, 0xfe, 0xa6, 0xcc, 0xba, 0x14, 0x81, 0x92, 0x2c, - 0xba, 0x48, 0x48, 0x49, 0x92, 0x3d, 0xac, 0x57, 0x3b, 0xbf, 0x35, 0x20, 0x2b, 0xc5, 0x61, 0x19, - 0x0a, 0xdd, 0xa3, 0xa7, 0x8d, 0x5e, 0xf7, 0xa0, 0x7a, 0x05, 0x21, 0xd8, 0xe8, 0x74, 0xdb, 0xbd, - 0x83, 0x01, 0x6e, 0x7f, 0xfe, 0xa4, 0x8b, 0xdb, 0x07, 0x55, 0x03, 0x5d, 0x83, 0xad, 0x5e, 0xbf, - 0xd5, 0x78, 0xdc, 0xed, 0x1f, 0x4d, 0xcd, 0x19, 0x64, 0x42, 0x2d, 0x65, 0x6e, 0xf5, 0x0f, 0x0f, - 0xdb, 0x47, 0x07, 0xed, 0x83, 0xea, 0xda, 0xd4, 0x49, 0xff, 0x58, 0xec, 0x36, 0x7a, 0xd5, 0x2c, - 0xba, 0x0a, 0x9b, 0xca, 0xd6, 0xe9, 0xe3, 0x66, 0xf7, 0xe0, 0xa0, 0x7d, 0x54, 0xcd, 0xa1, 0x2d, - 0x58, 0x7f, 0x72, 0x74, 0xd2, 0x78, 0xdc, 0x3d, 0xe9, 0x74, 0x1b, 0xcd, 0x5e, 0xbb, 0x9a, 0xb7, - 0x7e, 0x9f, 0xd2, 0x6e, 0x5f, 0x4a, 0xd9, 0xaa, 0xbf, 0x29, 0x4e, 0xeb, 0xbd, 0x15, 0xd3, 0x9a, - 0x4a, 0x47, 0x24, 0x15, 0x0c, 0x4e, 0x3b, 0x13, 0xed, 0x3c, 0xa9, 0xb4, 0x80, 0xf0, 0x33, 0x33, - 0xb3, 0xbd, 0xb6, 0x5b, 0xc2, 0x95, 0xd8, 0x78, 0x4c, 0xf8, 0x99, 0x00, 0x39, 0xd4, 0xe3, 0x64, - 0x30, 0x0e, 0x84, 0xef, 0x48, 0xa6, 0xb0, 0x88, 0x2b, 0xd2, 0xf8, 0x44, 0xd9, 0xac, 0xe7, 0x50, - 0xbd, 0xf8, 0xaa, 0x39, 0x62, 0xe9, 0xc7, 0x69, 0xb1, 0x54, 0xde, 0xbf, 0xbd, 0xfa, 0x61, 0xa6, - 0x85, 0xd5, 0x3d, 0x28, 0xe8, 0xb6, 0x87, 0x3e, 0x01, 0x44, 0xa4, 0xb4, 0x1c, 0x38, 0x34, 0xb2, - 0x43, 0x37, 0x48, 0x34, 0x48, 0x09, 0x6f, 0xa9, 0x9d, 0x83, 0xe9, 0x86, 0x75, 0x08, 0x79, 0xd5, - 0xf8, 0xde, 0xce, 0xbc, 0xf8, 0x02, 0xf2, 0xaa, 0x23, 0x2e, 0x1f, 0xe0, 0xc9, 0x30, 0xcc, 0xac, - 0x38, 0x0c, 0xad, 0x1f, 0x42, 0x41, 0xf7, 0x4c, 0x74, 0x1b, 0x94, 0x58, 0x49, 0x62, 0xd3, 0x4a, - 0x5b, 0xff, 0xb2, 0x39, 0x11, 0x7b, 0xb1, 0x9e, 0xf9, 0x0c, 0xd6, 0x67, 0x5a, 0xe5, 0xb7, 0x22, - 0x3f, 0x80, 0x4a, 0xba, 0x3b, 0x7e, 0x1b, 0xee, 0xce, 0x2f, 0xb3, 0x90, 0x6b, 0x7f, 0xc5, 0x43, - 0x62, 0xfd, 0xd5, 0x80, 0x5b, 0xf1, 0xf9, 0xb5, 0xc5, 0x04, 0x77, 0xfd, 0xe1, 0x71, 0xc8, 0x9e, - 0xab, 0x1f, 0x00, 0xf1, 0xbf, 0x34, 0x7a, 0x50, 0xa5, 0x7a, 0x73, 0x90, 0xce, 0x5b, 0x79, 0xff, - 0xd6, 0xe2, 0x1f, 0x77, 0x71, 0xa7, 0xd8, 0x8c, 0xa9, 0xf1, 0x0d, 0x39, 0x86, 0x6a, 0x10, 0xb2, - 0x80, 0x45, 0xd4, 0x49, 0xbc, 0xa9, 0x02, 0x5b, 0xf1, 0x57, 0xda, 0x66, 0x4c, 0xd7, 0x06, 0xeb, - 0xcf, 0x99, 0xe9, 0x57, 0x68, 0x5b, 0x63, 0x48, 0x5c, 0x3f, 0xe2, 0xa9, 0x22, 0x47, 0x9f, 0xcd, - 0x1e, 0xfa, 0x4a, 0xc1, 0x27, 0x75, 0x31, 0x9c, 0xbd, 0xd6, 0x19, 0x79, 0xad, 0xdb, 0x33, 0xf1, - 0xca, 0x8c, 0xd6, 0x2f, 0x8d, 0x63, 0xf9, 0x1d, 0x7f, 0x97, 0x37, 0x73, 0xff, 0x27, 0x50, 0x4a, - 0x0a, 0x04, 0xfd, 0x08, 0xca, 0xd3, 0x4c, 0x50, 0x54, 0x9b, 0x77, 0x16, 0xd6, 0xb5, 0xb9, 0x2f, - 0xda, 0x35, 0xee, 0x18, 0xcd, 0xe6, 0xab, 0x7f, 0xdd, 0xb8, 0xf2, 0xea, 0xeb, 0x1b, 0xc6, 0xdf, - 0xbf, 0xbe, 0x61, 0xfc, 0xf1, 0xdf, 0x37, 0x8c, 0x2f, 0xef, 0xac, 0xf4, 0x2f, 0x85, 0x94, 0xc3, - 0x67, 0x79, 0x69, 0xbe, 0xfb, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x4b, 0x65, 0x75, 0xc0, 0xdf, - 0x13, 0x00, 0x00, + // 1725 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x58, 0x4f, 0x6f, 0x1b, 0xc7, + 0x15, 0xf7, 0x52, 0xfc, 0xfb, 0x48, 0x49, 0xd4, 0x98, 0x4e, 0xb7, 0x1b, 0xc7, 0x96, 0x95, 0x04, + 0x11, 0x5c, 0x84, 0x32, 0xe4, 0x14, 0x89, 0x1d, 0xb8, 0x28, 0x49, 0x51, 0x80, 0x5c, 0xea, 0x4f, + 0x46, 0xb6, 0x03, 0xe4, 0x42, 0x8c, 0x77, 0x47, 0xd4, 0x5a, 0xcb, 0x9d, 0xed, 0xce, 0xd0, 0x8e, + 0x7a, 0xea, 0xa5, 0x28, 0xda, 0x53, 0x0b, 0xf4, 0x56, 0x14, 0xe8, 0x97, 0xe8, 0xa5, 0x3d, 0x16, + 0x05, 0x7c, 0xe8, 0xa1, 0x9f, 0xc0, 0x45, 0xd3, 0x4f, 0xd0, 0x6b, 0x0e, 0x45, 0x31, 0x7f, 0x76, + 0xb9, 0xa4, 0x45, 0x8a, 0x01, 0x5c, 0x5f, 0x04, 0xce, 0x9b, 0xdf, 0xef, 0xf1, 0xcd, 0x9b, 0x37, + 0xef, 0xfd, 0x28, 0xb8, 0x3d, 0x60, 0x5b, 0x51, 0xcc, 0x04, 0x73, 0x59, 0xc0, 0xb7, 0x86, 0x44, + 0xd0, 0xd8, 0x27, 0x81, 0xff, 0x33, 0x9a, 0xfd, 0xdc, 0x54, 0x08, 0x54, 0xcd, 0x98, 0x9c, 0x75, + 0x97, 0x85, 0x7c, 0x34, 0xa4, 0x71, 0x4a, 0x4f, 0x3f, 0x68, 0xb8, 0x73, 0x7d, 0xc2, 0xf5, 0x49, + 0xc0, 0x5e, 0xa8, 0x3f, 0x66, 0xb7, 0x31, 0x60, 0x03, 0xa6, 0x3e, 0x6e, 0xc9, 0x4f, 0xda, 0xba, + 0xf1, 0x97, 0xab, 0x50, 0xc2, 0xf4, 0xa7, 0x23, 0xca, 0x05, 0xfa, 0x18, 0xf2, 0x3c, 0xa2, 0xae, + 0x6d, 0xad, 0x5b, 0x9b, 0xd5, 0xed, 0xef, 0x37, 0xb3, 0x01, 0x19, 0x4c, 0xf3, 0x38, 0xa2, 0x2e, + 0x56, 0x30, 0x74, 0x0f, 0xca, 0xcf, 0x49, 0xe0, 0x7b, 0x44, 0x50, 0x3b, 0xa7, 0x28, 0xef, 0x5d, + 0x48, 0x79, 0x62, 0x40, 0x38, 0x85, 0xa3, 0x3b, 0x50, 0x20, 0x51, 0x14, 0x9c, 0xdb, 0x4b, 0x8a, + 0xe7, 0x5c, 0xc8, 0x6b, 0x49, 0x04, 0xd6, 0x40, 0x19, 0x1b, 0x8b, 0x68, 0x68, 0xe7, 0xe7, 0xc4, + 0x76, 0x18, 0xd1, 0x10, 0x2b, 0x98, 0x84, 0x07, 0x8c, 0x78, 0x76, 0x61, 0x0e, 0xbc, 0xc7, 0x88, + 0x87, 0x15, 0x4c, 0xc6, 0x73, 0x12, 0x8c, 0xf8, 0xa9, 0x5d, 0x9c, 0x13, 0xcf, 0xae, 0x44, 0x60, + 0x0d, 0x94, 0x0c, 0x2e, 0x58, 0x4c, 0xed, 0xd2, 0x1c, 0xc6, 0xb1, 0x44, 0x60, 0x0d, 0x44, 0x1d, + 0xa8, 0x71, 0x41, 0x62, 0xd1, 0x77, 0xd9, 0x70, 0xe8, 0x0b, 0xbb, 0xac, 0x88, 0xeb, 0x33, 0x88, + 0x24, 0x16, 0x1d, 0x85, 0xc3, 0x55, 0x3e, 0x5e, 0xa0, 0x36, 0x54, 0x89, 0x7b, 0x16, 0xb2, 0x17, + 0x01, 0xf5, 0x06, 0xd4, 0xae, 0xcc, 0xf1, 0xd1, 0x1a, 0xe3, 0x70, 0x96, 0x84, 0xde, 0x85, 0xb2, + 0x1f, 0x0a, 0x1a, 0x87, 0x24, 0xb0, 0xbd, 0x75, 0x6b, 0xb3, 0x86, 0x2b, 0x1f, 0x24, 0x06, 0xe7, + 0xb7, 0x16, 0xe4, 0xe5, 0x1d, 0xa3, 0x03, 0x58, 0x71, 0x59, 0x18, 0x52, 0x57, 0xb0, 0xb8, 0x2f, + 0xce, 0x23, 0xaa, 0xca, 0x62, 0x65, 0xfb, 0xa3, 0xa6, 0xaa, 0xa9, 0xfd, 0xf4, 0x1b, 0x89, 0xf0, + 0x59, 0x28, 0x29, 0xcd, 0x4e, 0x82, 0x7f, 0x74, 0x1e, 0x51, 0xbc, 0xec, 0x66, 0x97, 0xe8, 0x1e, + 0x54, 0x5d, 0x16, 0x9e, 0xf8, 0x83, 0xfe, 0x33, 0xce, 0x42, 0x55, 0x30, 0x95, 0xf6, 0xf5, 0x6f, + 0x5f, 0xdd, 0xb4, 0x69, 0xe8, 0x32, 0xcf, 0x0f, 0x07, 0x5b, 0x72, 0xa3, 0x89, 0xc9, 0x8b, 0x7d, + 0xca, 0x39, 0x19, 0x50, 0x5c, 0xd4, 0x04, 0xe7, 0x0f, 0x45, 0x28, 0x27, 0x45, 0x84, 0xbe, 0x80, + 0x7c, 0x48, 0x86, 0x3a, 0x9a, 0x4a, 0xfb, 0xc1, 0xb7, 0xaf, 0x6e, 0xde, 0x1b, 0xf8, 0xe2, 0x74, + 0xf4, 0xb4, 0xe9, 0xb2, 0xe1, 0x16, 0xe5, 0x62, 0x44, 0xe2, 0x73, 0x5d, 0xfc, 0xaf, 0x3d, 0x87, + 0xe9, 0xa8, 0xb1, 0x72, 0x75, 0xc1, 0x51, 0x73, 0x6f, 0xf2, 0xa8, 0x4b, 0x8b, 0x1f, 0x15, 0xb5, + 0xa0, 0xfc, 0xd4, 0x0f, 0x25, 0x84, 0xdb, 0xf9, 0xf5, 0xa5, 0xcd, 0xea, 0xf6, 0x87, 0x73, 0xdf, + 0x54, 0xb3, 0xad, 0xd1, 0x38, 0xa5, 0xa1, 0x1e, 0x34, 0x02, 0xc2, 0x45, 0x7f, 0x38, 0x19, 0x76, + 0xfa, 0x14, 0x66, 0x9d, 0x09, 0x5f, 0x95, 0xb4, 0xa9, 0x0d, 0x74, 0x0b, 0x6a, 0xca, 0xdb, 0x73, + 0x1a, 0x73, 0xe9, 0x45, 0x3e, 0x90, 0x0a, 0xae, 0x4a, 0xdb, 0x13, 0x6d, 0x72, 0xfe, 0x9b, 0x83, + 0x92, 0x09, 0x03, 0x3d, 0x84, 0x46, 0x4c, 0x39, 0x1b, 0xc5, 0x2e, 0xed, 0x67, 0x73, 0x60, 0x2d, + 0x90, 0x83, 0x95, 0x84, 0xd9, 0xd1, 0xb9, 0xb8, 0x0f, 0xe0, 0xb2, 0x20, 0xa0, 0xae, 0x0a, 0x5f, + 0x77, 0x98, 0x86, 0x0e, 0xbf, 0x93, 0xda, 0x65, 0xe4, 0xed, 0xfc, 0xcb, 0x57, 0x37, 0xaf, 0xe0, + 0x0c, 0x1a, 0xfd, 0xd2, 0x82, 0x6b, 0x27, 0x3e, 0x0d, 0xbc, 0x6c, 0x14, 0xfd, 0x21, 0x89, 0xec, + 0x25, 0x95, 0xd5, 0x07, 0x0b, 0x65, 0xb5, 0xb9, 0x2b, 0x5d, 0xe8, 0x70, 0x1e, 0x72, 0x16, 0xee, + 0x93, 0xa8, 0x1b, 0x8a, 0xf8, 0xbc, 0x7d, 0xfd, 0xd7, 0xff, 0x9c, 0x73, 0x90, 0xea, 0xc9, 0x98, + 0x86, 0x1c, 0x28, 0x3f, 0x25, 0xee, 0xd9, 0x89, 0x1f, 0x04, 0xaa, 0x79, 0x2d, 0xe3, 0x74, 0xed, + 0x74, 0xe1, 0x7b, 0x33, 0xbe, 0x01, 0xd5, 0x61, 0xe9, 0x8c, 0x9e, 0xeb, 0xbc, 0x61, 0xf9, 0x11, + 0x35, 0xa0, 0xf0, 0x9c, 0x04, 0x23, 0x5d, 0x9c, 0x15, 0xac, 0x17, 0xf7, 0x73, 0x9f, 0x59, 0xce, + 0x6f, 0x72, 0x50, 0x50, 0xcd, 0x12, 0x75, 0x60, 0x75, 0xfa, 0xda, 0xad, 0xcb, 0xae, 0x7d, 0x9a, + 0x81, 0x6c, 0x28, 0x25, 0xb7, 0xad, 0xbf, 0x2a, 0x59, 0xce, 0x2c, 0xad, 0xfc, 0x1b, 0x29, 0xad, + 0xc2, 0x6b, 0xa5, 0x85, 0x3e, 0x05, 0xe0, 0x82, 0x08, 0xaa, 0x8b, 0xa8, 0xb8, 0x40, 0x11, 0x15, + 0x14, 0xde, 0xf9, 0x9b, 0x05, 0x79, 0x39, 0x0e, 0xfe, 0xdf, 0x19, 0xf9, 0x10, 0x0a, 0x31, 0x09, + 0x07, 0xd4, 0x0c, 0xb2, 0x55, 0xed, 0x14, 0x4b, 0x93, 0x72, 0xa5, 0x77, 0xa7, 0xce, 0x91, 0x5f, + 0xfc, 0x1c, 0x02, 0xf2, 0x72, 0x4c, 0xc9, 0x08, 0xcc, 0x03, 0x57, 0xe1, 0x2f, 0xe3, 0x64, 0x89, + 0xee, 0x42, 0xf9, 0x8c, 0x9e, 0x2f, 0xde, 0x54, 0x55, 0x2d, 0xbd, 0x07, 0x20, 0x49, 0x11, 0x71, + 0xcf, 0xa8, 0xa7, 0x62, 0xaf, 0xe1, 0xca, 0x19, 0x3d, 0x3f, 0x52, 0x06, 0xa7, 0x04, 0x05, 0x35, + 0xec, 0x9c, 0x3f, 0xe7, 0xa0, 0xa0, 0x86, 0xd8, 0xdb, 0x0d, 0x40, 0x76, 0x50, 0x55, 0xde, 0x7c, + 0xf1, 0x84, 0x15, 0x35, 0x01, 0xbd, 0x0f, 0xcb, 0x86, 0x6a, 0x9c, 0x17, 0x94, 0xf3, 0x9a, 0x36, + 0x1a, 0xff, 0x77, 0xa1, 0xec, 0x31, 0x77, 0xf1, 0xaa, 0x5a, 0xf2, 0x98, 0x8b, 0xde, 0x81, 0x22, + 0xfd, 0xda, 0xe7, 0x82, 0xab, 0x99, 0x5f, 0xc6, 0x66, 0x25, 0xed, 0x1e, 0x0d, 0xa8, 0xa0, 0x6a, + 0xa4, 0x97, 0xb1, 0x59, 0x39, 0x18, 0xaa, 0x99, 0x39, 0x8e, 0x3a, 0x80, 0xe2, 0x51, 0x28, 0xfc, + 0x21, 0xed, 0xbb, 0xa7, 0xd4, 0x3d, 0x8b, 0x98, 0x1f, 0x0a, 0x53, 0x8c, 0x8d, 0x66, 0x22, 0xee, + 0x9a, 0x9d, 0x74, 0x0f, 0xaf, 0x19, 0xfc, 0xd8, 0xe4, 0x2c, 0x43, 0x35, 0x33, 0xd7, 0x37, 0x7e, + 0xb5, 0x0a, 0x65, 0x4c, 0x79, 0xc4, 0x42, 0x4e, 0x51, 0x73, 0x42, 0xbe, 0x4d, 0x2b, 0x12, 0x0d, + 0xca, 0xea, 0xb7, 0x07, 0x50, 0x49, 0x04, 0x99, 0x67, 0xda, 0xeb, 0xcd, 0x8b, 0x49, 0x49, 0x5f, + 0xf4, 0xf0, 0x98, 0x81, 0x3e, 0x85, 0x92, 0x94, 0x66, 0xbe, 0xb9, 0xbf, 0xd7, 0xd5, 0x9f, 0x21, + 0xb7, 0x34, 0x08, 0x27, 0x68, 0xf4, 0x09, 0x14, 0xa5, 0x46, 0xa3, 0x9e, 0xe9, 0x1b, 0xd7, 0x2f, + 0xe6, 0x1d, 0x2a, 0x0c, 0x36, 0x58, 0xc9, 0x92, 0x52, 0x8d, 0x26, 0x9a, 0x6e, 0x06, 0xab, 0xa7, + 0x30, 0xd8, 0x60, 0x65, 0x90, 0x4a, 0xaf, 0x51, 0xcf, 0x48, 0xbb, 0x19, 0x41, 0xee, 0x6a, 0x10, + 0x4e, 0xd0, 0xe8, 0x21, 0xac, 0x28, 0xdd, 0x45, 0xbd, 0x44, 0xaf, 0x69, 0xa1, 0xf7, 0xfe, 0x8c, + 0xb4, 0x6a, 0xac, 0x91, 0x6c, 0xcb, 0x3c, 0xbb, 0x44, 0xbb, 0x50, 0xcb, 0xe8, 0x2f, 0xcf, 0x28, + 0xbf, 0x8d, 0x19, 0xe9, 0xca, 0x20, 0xf1, 0x04, 0x6f, 0xbe, 0x70, 0xfb, 0x7b, 0xce, 0x08, 0x37, + 0x07, 0xca, 0x89, 0xea, 0x31, 0x4f, 0x35, 0x5d, 0xa3, 0x5d, 0x40, 0x66, 0x1e, 0x72, 0xf7, 0x94, + 0x0e, 0xc9, 0xe2, 0xaf, 0xb6, 0xa6, 0x79, 0xc7, 0x8a, 0x86, 0xbe, 0x84, 0x77, 0xa7, 0xc7, 0x7c, + 0xd6, 0xe1, 0x22, 0x8a, 0xa7, 0x31, 0x39, 0xed, 0x8d, 0xe3, 0x1f, 0xc0, 0x9a, 0xc7, 0xdc, 0xd1, + 0x90, 0x86, 0x42, 0xb5, 0xde, 0xfe, 0x28, 0xd6, 0x63, 0xb3, 0x82, 0xeb, 0x13, 0x1b, 0x8f, 0xe3, + 0x00, 0x7d, 0x00, 0x45, 0x46, 0x46, 0xe2, 0x74, 0xdb, 0x94, 0x44, 0x4d, 0x77, 0xdf, 0xc3, 0x96, + 0xb4, 0x61, 0xb3, 0x87, 0x3e, 0x81, 0x77, 0xd2, 0x58, 0x23, 0x22, 0x4e, 0xfb, 0xea, 0x25, 0xd1, + 0x98, 0xdb, 0xc5, 0xf5, 0xa5, 0xcd, 0xca, 0x38, 0x90, 0x23, 0x22, 0x4e, 0x8f, 0xcc, 0x9e, 0xf3, + 0x9f, 0x3c, 0x54, 0xd2, 0xb2, 0x47, 0x9d, 0x8c, 0x2c, 0xb3, 0x94, 0x80, 0xf8, 0xe8, 0x92, 0x97, + 0xf2, 0xba, 0x30, 0x73, 0x7e, 0x9e, 0x03, 0xe8, 0xb0, 0x90, 0x8b, 0x98, 0xf8, 0xa1, 0xec, 0x07, + 0xf9, 0x8c, 0xd6, 0xdc, 0xba, 0xcc, 0xdf, 0x98, 0xd9, 0x54, 0x9a, 0x53, 0x91, 0x65, 0xef, 0x89, + 0x29, 0x49, 0x73, 0x8e, 0xcd, 0x6a, 0xe3, 0x77, 0x16, 0xe4, 0x95, 0x16, 0xad, 0x42, 0x69, 0xef, + 0xe0, 0x49, 0xab, 0xb7, 0xb7, 0x53, 0xbf, 0x82, 0x10, 0xac, 0xec, 0xee, 0x75, 0x7b, 0x3b, 0x7d, + 0xdc, 0xfd, 0xe2, 0xf1, 0x1e, 0xee, 0xee, 0xd4, 0x2d, 0x74, 0x0d, 0xd6, 0x7a, 0x87, 0x9d, 0xd6, + 0xa3, 0xbd, 0xc3, 0x83, 0xb1, 0x39, 0x87, 0x6c, 0x68, 0x64, 0xcc, 0x9d, 0xc3, 0xfd, 0xfd, 0xee, + 0xc1, 0x4e, 0x77, 0xa7, 0xbe, 0x34, 0x76, 0x72, 0x78, 0x24, 0x77, 0x5b, 0xbd, 0x7a, 0x1e, 0x5d, + 0x85, 0x55, 0x6d, 0xdb, 0x3d, 0xc4, 0xed, 0xbd, 0x9d, 0x9d, 0xee, 0x41, 0xbd, 0x80, 0xd6, 0x60, + 0xf9, 0xf1, 0xc1, 0x71, 0xeb, 0xd1, 0xde, 0xf1, 0xee, 0x5e, 0xab, 0xdd, 0xeb, 0xd6, 0x8b, 0xce, + 0xef, 0x33, 0x52, 0xf1, 0x2b, 0xa5, 0x92, 0xcd, 0x99, 0x92, 0xb4, 0x7e, 0xb6, 0x60, 0x5a, 0x33, + 0xe9, 0xe0, 0x4a, 0x30, 0xe1, 0xac, 0x33, 0x39, 0x04, 0x26, 0xee, 0xdc, 0xce, 0xa9, 0xab, 0xae, + 0x65, 0xaf, 0x5a, 0x82, 0x3c, 0x1a, 0x08, 0xd2, 0x1f, 0x45, 0xd2, 0x37, 0x57, 0x29, 0x2c, 0xe3, + 0x9a, 0x32, 0x3e, 0xd6, 0x36, 0xe7, 0x19, 0xd4, 0xa7, 0xbf, 0xea, 0x02, 0x6d, 0xf6, 0xe3, 0xac, + 0x36, 0xab, 0x6e, 0xdf, 0x5e, 0xfc, 0x32, 0xb3, 0x3a, 0xce, 0x83, 0x92, 0x69, 0x96, 0xe8, 0x63, + 0x40, 0x44, 0x29, 0xd9, 0xbe, 0x47, 0xb9, 0x1b, 0xfb, 0x51, 0xaa, 0x5c, 0x2a, 0x78, 0x4d, 0xef, + 0xec, 0x8c, 0x37, 0xd0, 0x6d, 0xd0, 0x7a, 0x61, 0x5a, 0x25, 0x9b, 0x5f, 0x25, 0xc7, 0x72, 0x2f, + 0x91, 0x14, 0xfb, 0x50, 0xd4, 0xad, 0xf5, 0xcd, 0x4c, 0xa4, 0x2f, 0xa1, 0xa8, 0x7b, 0xee, 0x7c, + 0x89, 0x90, 0x8e, 0xdb, 0xdc, 0x82, 0xe3, 0xd6, 0xf9, 0x21, 0x94, 0x4c, 0x57, 0x1e, 0x1f, 0xcf, + 0xba, 0xfc, 0x78, 0x9f, 0xc3, 0xf2, 0x44, 0x33, 0xfe, 0x4e, 0xe4, 0xfb, 0x50, 0xcb, 0xf6, 0xdf, + 0xef, 0xc2, 0xdd, 0xf8, 0x45, 0x1e, 0x0a, 0xdd, 0xaf, 0x45, 0x4c, 0x9c, 0xbf, 0x5a, 0x70, 0x2b, + 0xb9, 0xeb, 0xae, 0xd4, 0x08, 0x7e, 0x38, 0x38, 0x8a, 0xd9, 0x33, 0xfd, 0xdb, 0x24, 0xf9, 0x6f, + 0x4b, 0x0f, 0xea, 0xd4, 0x6c, 0xf6, 0xb3, 0x79, 0xab, 0x6e, 0xdf, 0x9a, 0xfd, 0xbb, 0x33, 0xe9, + 0x2a, 0xab, 0x09, 0x35, 0x79, 0x4d, 0x47, 0x50, 0x8f, 0x62, 0x16, 0x31, 0x4e, 0xbd, 0xd4, 0x9b, + 0x2e, 0x86, 0x05, 0x7f, 0x40, 0xae, 0x26, 0x74, 0x63, 0x70, 0xfe, 0x94, 0x1b, 0x9f, 0xc2, 0xd8, + 0x5a, 0x03, 0xe2, 0x87, 0x5c, 0x64, 0x1e, 0x04, 0xfa, 0x7c, 0xf2, 0xd2, 0x17, 0x0a, 0x3e, 0xad, + 0x8b, 0xc1, 0x64, 0x0b, 0xc8, 0xa9, 0x16, 0xd0, 0x9d, 0x88, 0x57, 0x65, 0xb4, 0x79, 0x69, 0x1c, + 0xf3, 0xfb, 0xc1, 0xdb, 0x7c, 0xc5, 0xdb, 0x3f, 0x81, 0x4a, 0x5a, 0x20, 0xe8, 0x47, 0x50, 0x1d, + 0x67, 0x82, 0xa2, 0xc6, 0x45, 0x77, 0xe1, 0x5c, 0xbb, 0xf0, 0x8b, 0x36, 0xad, 0x3b, 0x56, 0xbb, + 0xfd, 0xf2, 0x5f, 0x37, 0xae, 0xbc, 0xfc, 0xe6, 0x86, 0xf5, 0x8f, 0x6f, 0x6e, 0x58, 0x7f, 0xfc, + 0xf7, 0x0d, 0xeb, 0xab, 0x3b, 0x0b, 0xfd, 0xb7, 0x23, 0xe3, 0xf0, 0x69, 0x51, 0x99, 0xef, 0xfe, + 0x2f, 0x00, 0x00, 0xff, 0xff, 0x8f, 0x60, 0x7a, 0xd0, 0x7a, 0x14, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -1909,6 +1928,13 @@ func (m *Request_Apply) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.StateJson) > 0 { + i -= len(m.StateJson) + copy(dAtA[i:], m.StateJson) + i = encodeVarintMaterialize(dAtA, i, uint64(len(m.StateJson))) + i-- + dAtA[i] = 0x32 + } if len(m.LastVersion) > 0 { i -= len(m.LastVersion) copy(dAtA[i:], m.LastVersion) @@ -2397,6 +2423,15 @@ func (m *Response_Spec) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.ResourcePathPointers) > 0 { + for iNdEx := len(m.ResourcePathPointers) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ResourcePathPointers[iNdEx]) + copy(dAtA[i:], m.ResourcePathPointers[iNdEx]) + i = encodeVarintMaterialize(dAtA, i, uint64(len(m.ResourcePathPointers[iNdEx]))) + i-- + dAtA[i] = 0x32 + } + } if m.Oauth2 != nil { { size, err := m.Oauth2.MarshalToSizedBuffer(dAtA[:i]) @@ -2614,6 +2649,18 @@ func (m *Response_Applied) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.State != nil { + { + size, err := m.State.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintMaterialize(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } if len(m.ActionDescription) > 0 { i -= len(m.ActionDescription) copy(dAtA[i:], m.ActionDescription) @@ -3132,6 +3179,10 @@ func (m *Request_Apply) ProtoSize() (n int) { if l > 0 { n += 1 + l + sovMaterialize(uint64(l)) } + l = len(m.StateJson) + if l > 0 { + n += 1 + l + sovMaterialize(uint64(l)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -3343,6 +3394,12 @@ func (m *Response_Spec) ProtoSize() (n int) { l = m.Oauth2.ProtoSize() n += 1 + l + sovMaterialize(uint64(l)) } + if len(m.ResourcePathPointers) > 0 { + for _, s := range m.ResourcePathPointers { + l = len(s) + n += 1 + l + sovMaterialize(uint64(l)) + } + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -3430,6 +3487,10 @@ func (m *Response_Applied) ProtoSize() (n int) { if l > 0 { n += 1 + l + sovMaterialize(uint64(l)) } + if m.State != nil { + l = m.State.ProtoSize() + n += 1 + l + sovMaterialize(uint64(l)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -4760,6 +4821,38 @@ func (m *Request_Apply) Unmarshal(dAtA []byte) error { } m.LastVersion = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateJson", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaterialize + } + 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 ErrInvalidLengthMaterialize + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthMaterialize + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateJson = encoding_json.RawMessage(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipMaterialize(dAtA[iNdEx:]) @@ -6121,6 +6214,38 @@ func (m *Response_Spec) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResourcePathPointers", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaterialize + } + 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 ErrInvalidLengthMaterialize + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthMaterialize + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ResourcePathPointers = append(m.ResourcePathPointers, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipMaterialize(dAtA[iNdEx:]) @@ -6623,6 +6748,42 @@ func (m *Response_Applied) Unmarshal(dAtA []byte) error { } m.ActionDescription = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMaterialize + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMaterialize + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthMaterialize + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.State == nil { + m.State = &flow.ConnectorState{} + } + if err := m.State.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipMaterialize(dAtA[iNdEx:]) diff --git a/go/protocols/materialize/materialize.proto b/go/protocols/materialize/materialize.proto index 45f4fc435b..dda1b7f75f 100644 --- a/go/protocols/materialize/materialize.proto +++ b/go/protocols/materialize/materialize.proto @@ -78,11 +78,14 @@ message Request { } Validate validate = 2; - // Apply a materialization configuration and bindings to its endpoint. - // Apply is run out-of-band with ongoing connector invocations, - // and may be run many times for a single materialization name, - // where each invocation has varying bindings, or even no bindings. - // The connector performs any required setup or cleanup. + // Apply an updated materialization specification to its endpoint, + // in preparation for an Open of a materialization session. + // Apply is run by the leader shard of a materialization task + // (having key_begin: 0) while the materialization is quiescent. + // Apply may be called multiple times for a given `version` and + // `last_version`, even if a prior call succeeded from the connector's + // perspective, so implementations must be idempotent. However, the next + // session will not Open until it's preceding Apply has durably completed. message Apply { // Materialization to be applied. flow.MaterializationSpec materialization = 1; @@ -92,6 +95,13 @@ message Request { flow.MaterializationSpec last_materialization = 4; // Version of the last applied MaterializationSpec. string last_version = 5; + // Last-persisted connector checkpoint state from a previous session. + // The Apply RPC may use this state to perform a post-commit apply + // of files staged under the `last_materialization` specification. + string state_json = 6 [ + (gogoproto.casttype) = "encoding/json.RawMessage", + json_name = "state" + ]; } Apply apply = 3; @@ -119,9 +129,9 @@ message Request { // provided to a successful Apply RPC. It's possible that it won't, // due to expected propagation races in Flow's distributed runtime. string version = 2; - // Range of documents to be processed by this invocation. + // Range of documents to be processed by this session. flow.RangeSpec range = 3; - // Last-persisted connector checkpoint state from a previous invocation. + // Last-persisted connector checkpoint state from a previous session. string state_json = 4 [ (gogoproto.casttype) = "encoding/json.RawMessage", json_name = "state" @@ -210,15 +220,22 @@ message Response { (gogoproto.casttype) = "encoding/json.RawMessage", json_name = "configSchema" ]; - // JSON schema of the connecor's resource configuration. + // JSON schema of the connector's resource configuration. string resource_config_schema_json = 3 [ (gogoproto.casttype) = "encoding/json.RawMessage", json_name = "resourceConfigSchema" ]; - // URL for connector's documention. + // URL for connector's documentation. string documentation_url = 4; // Optional OAuth2 configuration. flow.OAuth2 oauth2 = 5; + // One or more JSON pointers, which are used to extract resource paths + // from resource configurations of this connector. For example, + // a database connector might have a resource config like: + // {"schema": "foo", "table": "bar", "other": "config", "answer": 42} + // The connector would specify `resource_path_pointers: ["/schema", "/table"]`, + // which would result in a `resource_path` of `["foo", "bar"]`. + repeated string resource_path_pointers = 6; } Spec spec = 1; @@ -288,6 +305,10 @@ message Response { // Human-readable description of the action that the connector took. // If empty, this Apply is to be considered a "no-op". string action_description = 1; + // Optional *transactional* update to ConnectorState. + // This update commits atomically with the Flow recovery log checkpoint + // which marks the current specification as having been applied. + flow.ConnectorState state = 2; } Applied applied = 3;