From fd041a38834dd0aef8d886ecf0d337b9a7ce2092 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Wed, 18 Sep 2024 17:22:23 -0400 Subject: [PATCH 01/26] dekaf: Introduce `MaterializationEndpoint::Dekaf` A materialization of this kind is purely descriptive, and cannot be directly started. It is used to convey the intent to expose a set of bindings through Dekaf, which will look it up to determine things like binding names and field selection/projections. --- crates/flowctl/src/generate/mod.rs | 8 + crates/models/src/connector.rs | 11 + crates/models/src/lib.rs | 6 +- crates/models/src/materializations.rs | 10 +- crates/proto-flow/src/flow.rs | 3 + crates/proto-flow/src/flow.serde.rs | 3 + crates/runtime/src/materialize/connector.rs | 12 +- crates/sources/src/indirect.rs | 4 + crates/sources/src/inline.rs | 11 + crates/sources/src/loader.rs | 3 + ...a_generation__catalog_schema_snapshot.snap | 18 + crates/validation/src/materialization.rs | 10 +- crates/validation/tests/scenario_tests.rs | 30 + ...scenario_tests__dekaf_materialization.snap | 647 ++++++++++++++++++ go/protocols/flow/flow.pb.go | 455 ++++++------ go/protocols/flow/flow.proto | 1 + 16 files changed, 996 insertions(+), 236 deletions(-) create mode 100644 crates/validation/tests/snapshots/scenario_tests__dekaf_materialization.snap diff --git a/crates/flowctl/src/generate/mod.rs b/crates/flowctl/src/generate/mod.rs index 1476ed4182..5ce69cf8cc 100644 --- a/crates/flowctl/src/generate/mod.rs +++ b/crates/flowctl/src/generate/mod.rs @@ -287,6 +287,14 @@ async fn generate_missing_materialization_configs( }, serde_json::from_str::(config.config.get()).ok(), ), + models::MaterializationEndpoint::Dekaf(config) => ( + materialize::request::Spec { + connector_type: flow::materialization_spec::ConnectorType::Dekaf as i32, + config_json: serde_json::to_string(config).unwrap(), + }, + // Dekaf isn't a pluggable connector, and so does not have dynamic config. + None, + ), }; let missing_resource_urls: Vec<(url::Url, models::Collection)> = bindings .iter() diff --git a/crates/models/src/connector.rs b/crates/models/src/connector.rs index 0491801c43..d97bd98484 100644 --- a/crates/models/src/connector.rs +++ b/crates/models/src/connector.rs @@ -65,3 +65,14 @@ impl LocalConfig { } } } + +/// Dekaf configuration. Currently empty, but present to enable easy addition +/// of config options when they show up in the future. +#[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] +pub struct DekafConfig {} + +impl DekafConfig { + pub fn example() -> Self { + Self {} + } +} diff --git a/crates/models/src/lib.rs b/crates/models/src/lib.rs index 5cbae6c18d..9cb965508b 100644 --- a/crates/models/src/lib.rs +++ b/crates/models/src/lib.rs @@ -12,21 +12,20 @@ mod derive_typescript; mod id; mod journals; mod labels; -mod source_capture; mod materializations; mod raw_value; mod references; mod schemas; mod shards; mod source; +mod source_capture; mod tests; pub use crate::labels::{Label, LabelSelector, LabelSet}; pub use captures::{AutoDiscover, CaptureBinding, CaptureDef, CaptureEndpoint}; pub use catalogs::{Capability, Catalog, CatalogType}; pub use collections::{CollectionDef, Projection}; -pub use connector::{split_image_tag, ConnectorConfig, LocalConfig}; -pub use source_capture::{SourceCaptureDef, SourceCapture, SourceCaptureSchemaMode}; +pub use connector::{split_image_tag, ConnectorConfig, DekafConfig, LocalConfig}; pub use derivation::{Derivation, DeriveUsing, Shuffle, ShuffleType, TransformDef}; pub use derive_sqlite::DeriveUsingSqlite; pub use derive_typescript::DeriveUsingTypescript; @@ -48,6 +47,7 @@ pub use references::{ pub use schemas::Schema; pub use shards::ShardTemplate; pub use source::{FullSource, OnIncompatibleSchemaChange, PartitionSelector, Source}; +pub use source_capture::{SourceCapture, SourceCaptureDef, SourceCaptureSchemaMode}; pub use tests::{TestDef, TestDocuments, TestStep, TestStepIngest, TestStepVerify}; /// ModelDef is the common trait of top-level Flow specifications. diff --git a/crates/models/src/materializations.rs b/crates/models/src/materializations.rs index d4e5f7ca8f..2c338fe60f 100644 --- a/crates/models/src/materializations.rs +++ b/crates/models/src/materializations.rs @@ -1,11 +1,9 @@ use crate::Capture; -use crate::{source::OnIncompatibleSchemaChange, Collection, Id}; +use crate::{connector::DekafConfig, source::OnIncompatibleSchemaChange, Collection, Id}; use crate::source_capture::SourceCapture; -use super::{ - ConnectorConfig, Field, LocalConfig, RawValue, RelativeUrl, ShardTemplate, Source, -}; +use super::{ConnectorConfig, Field, LocalConfig, RawValue, RelativeUrl, ShardTemplate, Source}; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; use serde_json::json; @@ -53,6 +51,8 @@ pub enum MaterializationEndpoint { Connector(ConnectorConfig), /// # A local command (development only). Local(LocalConfig), + /// # A Dekaf connection + Dekaf(DekafConfig), } #[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] @@ -208,7 +208,7 @@ impl super::ModelDef for MaterializationDef { match &self.source_capture { Some(SourceCapture::Simple(capture_name)) => Some(capture_name), Some(SourceCapture::Configured(sc)) => Some(&sc.capture), - None => None + None => None, } } diff --git a/crates/proto-flow/src/flow.rs b/crates/proto-flow/src/flow.rs index 9fd1d9b6fb..4211d33f71 100644 --- a/crates/proto-flow/src/flow.rs +++ b/crates/proto-flow/src/flow.rs @@ -651,6 +651,7 @@ pub mod materialization_spec { Invalid = 0, Image = 8, Local = 9, + Dekaf = 10, } impl ConnectorType { /// String value of the enum field names used in the ProtoBuf definition. @@ -662,6 +663,7 @@ pub mod materialization_spec { ConnectorType::Invalid => "INVALID", ConnectorType::Image => "IMAGE", ConnectorType::Local => "LOCAL", + ConnectorType::Dekaf => "DEKAF", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -670,6 +672,7 @@ pub mod materialization_spec { "INVALID" => Some(Self::Invalid), "IMAGE" => Some(Self::Image), "LOCAL" => Some(Self::Local), + "DEKAF" => Some(Self::Dekaf), _ => None, } } diff --git a/crates/proto-flow/src/flow.serde.rs b/crates/proto-flow/src/flow.serde.rs index c8e5db89b7..174d897227 100644 --- a/crates/proto-flow/src/flow.serde.rs +++ b/crates/proto-flow/src/flow.serde.rs @@ -4000,6 +4000,7 @@ impl serde::Serialize for materialization_spec::ConnectorType { Self::Invalid => "INVALID", Self::Image => "IMAGE", Self::Local => "LOCAL", + Self::Dekaf => "DEKAF", }; serializer.serialize_str(variant) } @@ -4014,6 +4015,7 @@ impl<'de> serde::Deserialize<'de> for materialization_spec::ConnectorType { "INVALID", "IMAGE", "LOCAL", + "DEKAF", ]; struct GeneratedVisitor; @@ -4057,6 +4059,7 @@ impl<'de> serde::Deserialize<'de> for materialization_spec::ConnectorType { "INVALID" => Ok(materialization_spec::ConnectorType::Invalid), "IMAGE" => Ok(materialization_spec::ConnectorType::Image), "LOCAL" => Ok(materialization_spec::ConnectorType::Local), + "DEKAF" => Ok(materialization_spec::ConnectorType::Dekaf), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/crates/runtime/src/materialize/connector.rs b/crates/runtime/src/materialize/connector.rs index fb3671f239..6d1c215830 100644 --- a/crates/runtime/src/materialize/connector.rs +++ b/crates/runtime/src/materialize/connector.rs @@ -1,5 +1,5 @@ use crate::{unseal, LogHandler, Runtime}; -use anyhow::Context; +use anyhow::{bail, Context}; use futures::{channel::mpsc, stream::BoxStream, FutureExt, StreamExt}; use proto_flow::{ flow::materialization_spec::ConnectorType, @@ -87,6 +87,9 @@ pub async fn start( )? .boxed() } + models::MaterializationEndpoint::Dekaf(_) => { + bail!("Dekaf endpoint types are purely descriptive and cannot be started.") + } }; Ok((connector_tx, connector_rx)) @@ -140,6 +143,13 @@ fn extract_endpoint<'r>( ), config_json, )) + } else if connector_type == ConnectorType::Dekaf as i32 { + Ok(( + models::MaterializationEndpoint::Dekaf( + serde_json::from_str(config_json).context("parsing local config")?, + ), + config_json, + )) } else { anyhow::bail!("invalid connector type: {connector_type}"); } diff --git a/crates/sources/src/indirect.rs b/crates/sources/src/indirect.rs index 1015ca1fea..60986e4ae7 100644 --- a/crates/sources/src/indirect.rs +++ b/crates/sources/src/indirect.rs @@ -498,6 +498,10 @@ fn indirect_materialization( resources, threshold, ), + // Dekaf isn't a pluggable connector, and so does not have dynamic config. + // All of its config is defined directly within models::DekafConfig, and so + // should not be indirected. + models::MaterializationEndpoint::Dekaf(_) => {} } for (index, models::MaterializationBinding { resource, .. }) in bindings.iter_mut().enumerate() diff --git a/crates/sources/src/inline.rs b/crates/sources/src/inline.rs index 0fc7fbd9dd..9634b24fda 100644 --- a/crates/sources/src/inline.rs +++ b/crates/sources/src/inline.rs @@ -266,6 +266,17 @@ fn inline_materialization( resources, ) } + models::MaterializationEndpoint::Dekaf(models::DekafConfig { config, .. }) => { + inline_config( + Scope::new(scope) + .push_prop("endpoint") + .push_prop("dekaf") + .push_prop("config"), + config, + imports, + resources, + ) + } } for (index, models::MaterializationBinding { resource, .. }) in bindings.iter_mut().enumerate() diff --git a/crates/sources/src/loader.rs b/crates/sources/src/loader.rs index 07d81a4951..be6ad06acf 100644 --- a/crates/sources/src/loader.rs +++ b/crates/sources/src/loader.rs @@ -687,6 +687,9 @@ impl Loader { .boxed(), ); } + // Dekaf isn't a pluggable connector, and so does not have dynamic config to possibly + // load from a reference. All of its config is defined directly within models::DekafConfig. + models::MaterializationEndpoint::Dekaf(_) => {} }; for (index, binding) in spec.bindings.iter().enumerate() { diff --git a/crates/sources/tests/snapshots/schema_generation__catalog_schema_snapshot.snap b/crates/sources/tests/snapshots/schema_generation__catalog_schema_snapshot.snap index 5d201e7603..057046a356 100644 --- a/crates/sources/tests/snapshots/schema_generation__catalog_schema_snapshot.snap +++ b/crates/sources/tests/snapshots/schema_generation__catalog_schema_snapshot.snap @@ -1,5 +1,6 @@ --- source: crates/sources/tests/schema_generation.rs +assertion_line: 4 expression: "&schema" --- { @@ -460,6 +461,10 @@ expression: "&schema" } } }, + "DekafConfig": { + "description": "Dekaf configuration. Currently empty, but present to enable easy addition of config options when they show up in the future.", + "type": "object" + }, "Derivation": { "description": "Derive specifies how a collection is derived from other collections.", "type": "object", @@ -917,6 +922,19 @@ expression: "&schema" } }, "additionalProperties": false + }, + { + "title": "A Dekaf connection", + "type": "object", + "required": [ + "dekaf" + ], + "properties": { + "dekaf": { + "$ref": "#/definitions/DekafConfig" + } + }, + "additionalProperties": false } ] }, diff --git a/crates/validation/src/materialization.rs b/crates/validation/src/materialization.rs index 6d3553aa09..ba4cf7175f 100644 --- a/crates/validation/src/materialization.rs +++ b/crates/validation/src/materialization.rs @@ -122,6 +122,10 @@ async fn walk_materialization( flow::materialization_spec::ConnectorType::Local as i32, serde_json::to_string(config).unwrap(), ), + models::MaterializationEndpoint::Dekaf(config) => ( + flow::materialization_spec::ConnectorType::Dekaf as i32, + serde_json::to_string(config).unwrap(), + ), }; // We only validate and build enabled bindings, in their declaration order. @@ -187,7 +191,11 @@ async fn walk_materialization( }); // If shards are disabled, then don't ask the connector to validate. - let response = if shard_template.disable { + let response = if shard_template.disable + // TODO(jshearer): Are we sure that Dekaf is okay with _any_ projection? + // Currently, `NoOpConnectors` report every field as `FieldOptional` + || connector_type == flow::materialization_spec::ConnectorType::Dekaf as i32 + { NoOpConnectors.validate_materialization(wrapped_request, data_plane) } else { connectors.validate_materialization(wrapped_request, data_plane) diff --git a/crates/validation/tests/scenario_tests.rs b/crates/validation/tests/scenario_tests.rs index 4ea38f84a7..90366507ef 100644 --- a/crates/validation/tests/scenario_tests.rs +++ b/crates/validation/tests/scenario_tests.rs @@ -8,6 +8,36 @@ fn test_golden_all_visits() { insta::assert_debug_snapshot!(outcome); } +#[test] +fn test_dekaf_materialization() { + let fixture = r##" +test://example/catalog.yaml: + collections: + testing/schema_with_properties: + schema: + type: object + properties: + id: { type: string } + required: [id] + key: [/id] + materializations: + testing/test_dekaf: + endpoint: + dekaf: {} + bindings: + - source: testing/schema_with_properties + resource: {} +driver: + dataPlanes: + "1d:1d:1d:1d:1d:1d:1d:1d": + default: true +"##; + + let outcome = common::run(fixture, "{}"); + // Expect not to see any projections for the empty properties + insta::assert_debug_snapshot!(outcome); +} + #[test] fn test_projection_not_created_for_empty_properties() { let fixture = r##" diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization.snap new file mode 100644 index 0000000000..61615d4d03 --- /dev/null +++ b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization.snap @@ -0,0 +1,647 @@ +--- +source: crates/validation/tests/scenario_tests.rs +assertion_line: 38 +expression: outcome +--- +Outcome { + built_captures: [], + built_collections: [ + BuiltCollection { + collection: testing/schema_with_properties, + scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, + control_id: "0000000000000000", + data_plane_id: "1d1d1d1d1d1d1d1d", + expect_pub_id: "0000000000000000", + expect_build_id: "0000000000000000", + model: { + "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, + "key": [ + "/id" + ] + }, + validated: NULL, + spec: CollectionSpec { + name: "testing/schema_with_properties", + write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", + read_schema_json: "", + key: [ + "/id", + ], + uuid_ptr: "/_meta/uuid", + partition_fields: [], + projections: [ + Projection { + ptr: "/_meta/flow_truncated", + field: "_meta/flow_truncated", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "boolean", + ], + string: None, + title: "Flow truncation indicator", + description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "", + field: "flow_document", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "object", + ], + string: None, + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/_meta/uuid", + field: "flow_published_at", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "date-time", + content_encoding: "uuid", + max_length: 0, + }, + ), + title: "Flow Publication Time", + description: "Flow publication date-time of this document", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/id", + field: "id", + explicit: false, + is_partition_key: false, + is_primary_key: true, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "", + content_encoding: "", + max_length: 0, + }, + ), + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + ], + ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", + partition_template: Some( + JournalSpec { + name: "testing/schema_with_properties/2020202020202020", + replication: 3, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "content-type", + value: "application/x-ndjson", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/collection", + value: "testing/schema_with_properties", + prefix: false, + }, + ], + }, + ), + fragment: Some( + Fragment { + length: 536870912, + compression_codec: Gzip, + stores: [ + "s3://a-bucket/", + ], + refresh_interval: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + retention: None, + flush_interval: None, + path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", + }, + ), + flags: 4, + max_append_rate: 4194304, + }, + ), + derivation: None, + }, + previous_spec: NULL, + is_touch: 0, + dependency_hash: NULL, + }, + ], + built_materializations: [ + BuiltMaterialization { + materialization: testing/test_dekaf, + scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, + control_id: "0000000000000000", + data_plane_id: "1d1d1d1d1d1d1d1d", + expect_pub_id: "0000000000000000", + expect_build_id: "0000000000000000", + model: { + "endpoint": { + "dekaf": {} + }, + "bindings": [ + { + "resource": {}, + "source": "testing/schema_with_properties", + "fields": { + "recommended": true + } + } + ] + }, + validated: Validated { + bindings: [ + Binding { + constraints: { + "_meta/flow_truncated": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + "flow_document": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + "flow_published_at": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + "id": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + }, + resource_path: [ + "binding-0", + ], + delta_updates: true, + }, + ], + }, + spec: MaterializationSpec { + name: "testing/test_dekaf", + connector_type: Dekaf, + config_json: "{}", + bindings: [ + Binding { + resource_config_json: "{}", + resource_path: [ + "binding-0", + ], + collection: Some( + CollectionSpec { + name: "testing/schema_with_properties", + write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", + read_schema_json: "", + key: [ + "/id", + ], + uuid_ptr: "/_meta/uuid", + partition_fields: [], + projections: [ + Projection { + ptr: "/_meta/flow_truncated", + field: "_meta/flow_truncated", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "boolean", + ], + string: None, + title: "Flow truncation indicator", + description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "", + field: "flow_document", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "object", + ], + string: None, + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/_meta/uuid", + field: "flow_published_at", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "date-time", + content_encoding: "uuid", + max_length: 0, + }, + ), + title: "Flow Publication Time", + description: "Flow publication date-time of this document", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/id", + field: "id", + explicit: false, + is_partition_key: false, + is_primary_key: true, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "", + content_encoding: "", + max_length: 0, + }, + ), + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + ], + ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", + partition_template: Some( + JournalSpec { + name: "testing/schema_with_properties/2020202020202020", + replication: 3, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "content-type", + value: "application/x-ndjson", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/collection", + value: "testing/schema_with_properties", + prefix: false, + }, + ], + }, + ), + fragment: Some( + Fragment { + length: 536870912, + compression_codec: Gzip, + stores: [ + "s3://a-bucket/", + ], + refresh_interval: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + retention: None, + flush_interval: None, + path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", + }, + ), + flags: 4, + max_append_rate: 4194304, + }, + ), + derivation: None, + }, + ), + partition_selector: Some( + LabelSelector { + include: Some( + LabelSet { + labels: [ + Label { + name: "estuary.dev/collection", + value: "testing/schema_with_properties", + prefix: false, + }, + ], + }, + ), + exclude: Some( + LabelSet { + labels: [], + }, + ), + }, + ), + priority: 0, + field_selection: Some( + FieldSelection { + keys: [], + values: [], + document: "", + field_config_json_map: {}, + }, + ), + delta_updates: true, + deprecated_shuffle: None, + journal_read_suffix: "materialize/testing/test_dekaf/binding-0", + not_before: None, + not_after: None, + backfill: 0, + state_key: "binding-0", + }, + ], + shard_template: Some( + ShardSpec { + id: "materialize/testing/test_dekaf/2020202020202020", + sources: [], + recovery_log_prefix: "recovery", + hint_prefix: "/estuary/flow/hints", + hint_backups: 2, + max_txn_duration: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + min_txn_duration: Some( + Duration { + seconds: 0, + nanos: 0, + }, + ), + disable: false, + hot_standbys: 0, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/log-level", + value: "info", + prefix: false, + }, + Label { + name: "estuary.dev/task-name", + value: "testing/test_dekaf", + prefix: false, + }, + Label { + name: "estuary.dev/task-type", + value: "materialization", + prefix: false, + }, + ], + }, + ), + disable_wait_for_ack: false, + ring_buffer_size: 65536, + read_channel_size: 4096, + }, + ), + recovery_log_template: Some( + JournalSpec { + name: "recovery/materialize/testing/test_dekaf/2020202020202020", + replication: 3, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "content-type", + value: "application/x-gazette-recoverylog", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/task-name", + value: "testing/test_dekaf", + prefix: false, + }, + Label { + name: "estuary.dev/task-type", + value: "materialization", + prefix: false, + }, + ], + }, + ), + fragment: Some( + Fragment { + length: 268435456, + compression_codec: Snappy, + stores: [ + "s3://a-bucket/", + ], + refresh_interval: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + retention: None, + flush_interval: None, + path_postfix_template: "", + }, + ), + flags: 4, + max_append_rate: 4194304, + }, + ), + network_ports: [], + }, + previous_spec: NULL, + is_touch: 0, + dependency_hash: 32182596aeb1e4a0, + }, + ], + built_tests: [], + captures: [], + collections: [ + DraftCollection { + collection: testing/schema_with_properties, + scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, + expect_pub_id: NULL, + model: { + "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, + "key": [ + "/id" + ] + }, + is_touch: 0, + }, + ], + errors: [], + errors_draft: [], + fetches: [ + Fetch { + depth: 1, + resource: test://example/catalog.yaml, + }, + ], + imports: [], + materializations: [ + DraftMaterialization { + materialization: testing/test_dekaf, + scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, + expect_pub_id: NULL, + model: { + "endpoint": { + "dekaf": {} + }, + "bindings": [ + { + "resource": {}, + "source": "testing/schema_with_properties", + "fields": { + "recommended": true + } + } + ] + }, + is_touch: 0, + }, + ], + resources: [ + Resource { + resource: test://example/catalog.yaml, + content_type: "CATALOG", + content: ".. binary ..", + content_dom: {"collections":{"testing/schema_with_properties":{"key":["/id"],"schema":{"properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}}},"materializations":{"testing/test_dekaf":{"bindings":[{"resource":{},"source":"testing/schema_with_properties"}],"endpoint":{"dekaf":{}}}}}, + }, + ], + storage_mappings: [ + StorageMapping { + catalog_prefix: , + control_id: "0000000000000000", + stores: [ + { + "provider": "S3", + "bucket": "a-bucket", + "prefix": null, + "region": null + } + ], + }, + ], + tests: [], +} diff --git a/go/protocols/flow/flow.pb.go b/go/protocols/flow/flow.pb.go index fd08c1f681..8f71434f45 100644 --- a/go/protocols/flow/flow.pb.go +++ b/go/protocols/flow/flow.pb.go @@ -214,18 +214,21 @@ const ( MaterializationSpec_INVALID MaterializationSpec_ConnectorType = 0 MaterializationSpec_IMAGE MaterializationSpec_ConnectorType = 8 MaterializationSpec_LOCAL MaterializationSpec_ConnectorType = 9 + MaterializationSpec_DEKAF MaterializationSpec_ConnectorType = 10 ) var MaterializationSpec_ConnectorType_name = map[int32]string{ - 0: "INVALID", - 8: "IMAGE", - 9: "LOCAL", + 0: "INVALID", + 8: "IMAGE", + 9: "LOCAL", + 10: "DEKAF", } var MaterializationSpec_ConnectorType_value = map[string]int32{ "INVALID": 0, "IMAGE": 8, "LOCAL": 9, + "DEKAF": 10, } func (x MaterializationSpec_ConnectorType) String() string { @@ -2263,234 +2266,234 @@ func init() { func init() { proto.RegisterFile("go/protocols/flow/flow.proto", fileDescriptor_d0677502142fec31) } var fileDescriptor_d0677502142fec31 = []byte{ - // 3621 bytes of a gzipped FileDescriptorProto + // 3630 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x5a, 0xcb, 0x6f, 0x23, 0xc9, 0x79, 0x57, 0xf3, 0xcd, 0x8f, 0x0f, 0xb5, 0x4a, 0x9a, 0x19, 0x0e, 0xbd, 0x23, 0x6a, 0xe8, 0x18, 0xab, 0xdd, 0xf5, 0x52, 0x6b, 0x2d, 0xec, 0xec, 0xcc, 0x60, 0x62, 0xf0, 0xa5, 0x59, 0xce, 0x50, 0x24, 0xd3, 0xa4, 0xbc, 0x1e, 0x03, 0x46, 0xa3, 0xd9, 0x5d, 0xa4, 0x7a, 0xd4, 0xec, 0x66, 0xba, 0x8b, 0x1a, 0xd1, 0x97, 0x04, 0x41, 0x80, 0x00, 0x8b, 0x04, 0xf0, 0x25, 0xb0, 0x6f, 0xde, 0x18, - 0x48, 0xe0, 0x7f, 0x21, 0xc9, 0x25, 0x40, 0x2e, 0x7b, 0x08, 0x02, 0x03, 0x41, 0x72, 0xc8, 0x61, - 0x16, 0x71, 0x0e, 0xf6, 0x29, 0xb9, 0x05, 0xc1, 0x9c, 0x82, 0x7a, 0x74, 0xb3, 0x29, 0x6a, 0xf4, - 0x58, 0xdb, 0x07, 0x5f, 0x88, 0xaa, 0xef, 0xd5, 0x55, 0xdf, 0x57, 0xf5, 0xfb, 0xbe, 0xaa, 0x22, - 0xbc, 0x35, 0x76, 0xf6, 0xa6, 0xae, 0x43, 0x1c, 0xdd, 0xb1, 0xbc, 0xbd, 0x91, 0xe5, 0xbc, 0x64, - 0x3f, 0x15, 0x46, 0x43, 0x31, 0xda, 0x2e, 0x6e, 0x0f, 0x5d, 0xe7, 0x04, 0xbb, 0x81, 0x5c, 0xd0, - 0xe0, 0x52, 0xc5, 0x1d, 0xdd, 0xb1, 0xbd, 0xd9, 0xe4, 0x12, 0x89, 0xad, 0xb1, 0x33, 0x76, 0x58, - 0x73, 0x8f, 0xb6, 0x04, 0xb5, 0x34, 0x76, 0x9c, 0xb1, 0x85, 0xb9, 0xf0, 0x70, 0x36, 0xda, 0x23, - 0xe6, 0x04, 0x7b, 0x44, 0x9b, 0x4c, 0xb9, 0x40, 0xf9, 0x31, 0xc4, 0xfb, 0x96, 0xa9, 0x63, 0xb4, - 0x05, 0xf1, 0x21, 0x1e, 0x9b, 0x76, 0x41, 0xda, 0x91, 0x76, 0x73, 0x0a, 0xef, 0x20, 0x19, 0xa2, - 0xd8, 0x36, 0x0a, 0x11, 0x46, 0xa3, 0xcd, 0x87, 0xd9, 0x9f, 0xff, 0x75, 0x69, 0xed, 0x87, 0x3f, - 0x2d, 0xad, 0xfd, 0xf8, 0xa7, 0xa5, 0xb5, 0xb2, 0x0d, 0xe9, 0xa3, 0xa3, 0x56, 0xa3, 0xa7, 0xb9, - 0xc4, 0x43, 0x08, 0x62, 0xb6, 0x63, 0x60, 0x66, 0x21, 0xa1, 0xb0, 0x36, 0x7a, 0x04, 0x71, 0xdd, - 0x72, 0xf4, 0x13, 0x66, 0x22, 0x51, 0xfb, 0xda, 0xeb, 0x57, 0xa5, 0xfb, 0x63, 0xa7, 0x32, 0xd6, - 0x7e, 0x80, 0x09, 0xc1, 0x15, 0x03, 0x9f, 0xee, 0xe9, 0x8e, 0x8b, 0xf7, 0x26, 0xd8, 0xf3, 0xb4, - 0x31, 0xae, 0xd4, 0xa9, 0xb0, 0xc2, 0x75, 0x1e, 0xca, 0xbf, 0xfa, 0xac, 0x24, 0x2d, 0x7d, 0xef, - 0xdf, 0x24, 0x80, 0x9e, 0xeb, 0xbc, 0xc0, 0x3a, 0x31, 0x1d, 0x36, 0xbc, 0x29, 0x71, 0xd9, 0x07, - 0xd3, 0x0a, 0x6d, 0xd2, 0x69, 0x8c, 0x4c, 0x6c, 0xf1, 0x21, 0xa7, 0x15, 0xde, 0x41, 0x45, 0x48, - 0xe1, 0xb3, 0xa9, 0x65, 0xea, 0x26, 0x29, 0x44, 0x77, 0xa4, 0xdd, 0x94, 0x12, 0xf4, 0xd1, 0x2e, - 0xc8, 0xa6, 0xa7, 0x4e, 0x35, 0x97, 0x98, 0xd4, 0xa6, 0x7a, 0x82, 0xe7, 0x85, 0x18, 0x93, 0xc9, - 0x9b, 0x5e, 0xcf, 0x27, 0x3f, 0xc3, 0x73, 0xf4, 0x7b, 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, 0xdd, 0xcc, 0xfe, 0x7a, 0x85, 0x05, 0xbc, 0xe5, - 0x93, 0x6b, 0xb1, 0xcf, 0x5f, 0x95, 0xd6, 0x94, 0x85, 0x5c, 0xf9, 0x67, 0x71, 0x48, 0x07, 0x6c, - 0x3a, 0x09, 0x32, 0x9f, 0x62, 0xaf, 0x20, 0xed, 0x44, 0xe9, 0x24, 0x58, 0x07, 0x55, 0x20, 0xe1, - 0x11, 0xd7, 0xb4, 0xc7, 0x6c, 0x0a, 0x99, 0xfd, 0xdb, 0xe7, 0xac, 0x56, 0xfa, 0x8c, 0xab, 0x08, - 0x29, 0x66, 0xc5, 0x24, 0x16, 0x66, 0xb3, 0xa1, 0x56, 0x68, 0x07, 0xed, 0x40, 0xc6, 0xc0, 0x9e, - 0xee, 0x9a, 0x53, 0x3a, 0x2d, 0x36, 0x83, 0xb4, 0x12, 0x26, 0xa1, 0x47, 0x90, 0x35, 0xf0, 0x48, - 0x9b, 0x59, 0x44, 0x7d, 0xe1, 0x39, 0x36, 0x9b, 0x43, 0xba, 0xf6, 0xd6, 0xeb, 0x57, 0xa5, 0x02, - 0xb6, 0x75, 0xc7, 0x30, 0xed, 0xf1, 0x1e, 0x65, 0x54, 0x14, 0xed, 0xe5, 0x21, 0x0f, 0x9b, 0x92, - 0x14, 0x1a, 0xe8, 0x36, 0x24, 0x3c, 0xac, 0xbb, 0x98, 0x14, 0x92, 0xcc, 0x37, 0xa2, 0x47, 0x07, - 0x8f, 0xcf, 0x4c, 0x8f, 0x78, 0x85, 0xd4, 0x8e, 0xb4, 0x9b, 0x5f, 0x1d, 0x7c, 0x93, 0x71, 0x15, - 0x21, 0x85, 0xbe, 0x01, 0x49, 0x9b, 0xae, 0x77, 0x53, 0x2f, 0xa4, 0xd9, 0x6c, 0xef, 0x9c, 0x57, - 0xe8, 0x70, 0xb6, 0xe2, 0xcb, 0x15, 0x7f, 0x22, 0x41, 0x82, 0xbb, 0x00, 0xdd, 0x87, 0xac, 0xee, - 0xd8, 0x04, 0xdb, 0x44, 0xa5, 0xbe, 0x63, 0x0e, 0x4b, 0x2b, 0x19, 0x41, 0x1b, 0xcc, 0xa7, 0x98, - 0x0e, 0x74, 0xe4, 0xb8, 0x13, 0x8d, 0x08, 0xf7, 0x88, 0x1e, 0x7a, 0x07, 0x64, 0x5f, 0xd5, 0x9f, - 0x2d, 0x9b, 0x4a, 0x5a, 0x59, 0x17, 0xf4, 0xa6, 0x20, 0xa3, 0x7b, 0x00, 0x13, 0xed, 0x4c, 0xb5, - 0xb0, 0x3d, 0x26, 0xc7, 0xcc, 0x4d, 0x39, 0x25, 0x3d, 0xd1, 0xce, 0xda, 0x8c, 0xf0, 0x34, 0x96, - 0x92, 0xe4, 0xc8, 0xd3, 0x58, 0x2a, 0x22, 0x47, 0x9f, 0xc6, 0x52, 0x71, 0x39, 0x51, 0xfc, 0x63, - 0x48, 0x8a, 0x51, 0xa3, 0x12, 0x64, 0x8e, 0x35, 0x4f, 0x9d, 0x98, 0xb6, 0x39, 0x99, 0x4d, 0xd8, - 0x0a, 0x4e, 0x29, 0x70, 0xac, 0x79, 0x87, 0x9c, 0x82, 0x0a, 0x90, 0xf4, 0x99, 0x74, 0x29, 0x4b, - 0x8a, 0xdf, 0x0d, 0x54, 0xb5, 0x33, 0xc6, 0x8d, 0x2e, 0x54, 0x39, 0x85, 0xa9, 0x0a, 0x66, 0x4c, - 0xa8, 0xf2, 0x6e, 0xb9, 0x06, 0x09, 0xee, 0x67, 0x94, 0x81, 0x64, 0xab, 0xf3, 0x9d, 0x6a, 0xbb, - 0xd5, 0x90, 0xd7, 0x50, 0x0a, 0x62, 0x87, 0x47, 0xfd, 0x81, 0x2c, 0xa1, 0x24, 0x44, 0x0f, 0xab, - 0xcf, 0xe5, 0x08, 0xca, 0x42, 0xaa, 0x75, 0xd8, 0x6b, 0xb7, 0xea, 0xad, 0x81, 0x1c, 0x45, 0x00, - 0x89, 0x7a, 0xb5, 0xd3, 0xe9, 0x0e, 0xe4, 0x58, 0xf9, 0x39, 0x64, 0x3a, 0x98, 0xbc, 0x74, 0xdc, - 0x93, 0x9e, 0xe3, 0xb2, 0x80, 0xdb, 0xb3, 0xc9, 0x10, 0xbb, 0x02, 0x38, 0x44, 0x8f, 0x6e, 0x39, - 0x1f, 0xa1, 0xc4, 0x5e, 0x0c, 0xfa, 0x54, 0x67, 0x3a, 0x1b, 0x5a, 0xa6, 0x2e, 0x06, 0x2f, 0x7a, - 0xe5, 0xff, 0xcd, 0x43, 0xbe, 0xee, 0x58, 0x16, 0xdf, 0xdd, 0xfd, 0x29, 0xd6, 0x51, 0x19, 0x62, - 0xb6, 0x36, 0xe1, 0x98, 0x92, 0xae, 0xe5, 0x5f, 0xbf, 0x2a, 0xc1, 0x42, 0x42, 0x61, 0x3c, 0xd4, - 0x80, 0x8d, 0x97, 0xae, 0x49, 0xb0, 0xea, 0xe9, 0xc7, 0x78, 0xa2, 0xf1, 0x55, 0x9b, 0xba, 0xc6, - 0xaa, 0xcd, 0x30, 0xb5, 0x3e, 0xd3, 0x42, 0x35, 0x90, 0x5d, 0xac, 0x19, 0x4b, 0x46, 0x32, 0xd7, - 0x30, 0x02, 0x54, 0x4b, 0xd8, 0x90, 0x21, 0x4a, 0x61, 0x21, 0xca, 0xb6, 0x2d, 0x6d, 0xa2, 0xbb, - 0x90, 0x9a, 0xcd, 0x4c, 0x43, 0xa5, 0x30, 0xc5, 0x17, 0x5a, 0x92, 0xf6, 0x7b, 0xc4, 0xa5, 0x2b, - 0x6d, 0x81, 0x3a, 0x0c, 0xa7, 0xbc, 0x42, 0x9c, 0x69, 0xae, 0x07, 0xf4, 0x03, 0x46, 0x46, 0x1f, - 0x41, 0x66, 0x1a, 0xa0, 0x9e, 0x57, 0x48, 0xec, 0x44, 0x77, 0x33, 0xfb, 0x32, 0xdf, 0x11, 0x0b, - 0x38, 0x14, 0xb0, 0x12, 0x16, 0xa5, 0xbe, 0xd1, 0xf4, 0x13, 0x95, 0xe0, 0xc9, 0xd4, 0xd2, 0x08, - 0xe6, 0xd3, 0x4a, 0x5e, 0xc7, 0x37, 0x9a, 0x7e, 0x32, 0x10, 0x5a, 0xa8, 0x01, 0x68, 0x31, 0x54, - 0xdf, 0x96, 0xd8, 0x98, 0xb7, 0x2a, 0x41, 0x26, 0x7a, 0xea, 0xcc, 0x5c, 0x5b, 0xb3, 0x68, 0xe0, - 0x94, 0x8d, 0x40, 0x21, 0xb0, 0xf2, 0x6d, 0x00, 0x03, 0xbb, 0xe6, 0xa9, 0xc6, 0x90, 0x27, 0xcb, - 0xb4, 0x4b, 0x7c, 0x12, 0xcb, 0x51, 0xaf, 0x34, 0x02, 0x31, 0x25, 0xa4, 0x52, 0xfc, 0x77, 0x00, - 0x58, 0xb0, 0xd0, 0x00, 0xf2, 0xba, 0x63, 0xdb, 0x58, 0x27, 0x8e, 0xcb, 0xf7, 0xb9, 0xc4, 0xb0, - 0xe5, 0xfd, 0x2b, 0x6c, 0x56, 0xea, 0xbe, 0x16, 0x45, 0x02, 0x25, 0xa7, 0x87, 0xbb, 0xe8, 0x01, - 0x50, 0x9c, 0x18, 0x99, 0x63, 0xee, 0xab, 0xc8, 0x35, 0x7c, 0x95, 0xe0, 0x0a, 0xa8, 0x0b, 0x40, - 0x5c, 0xcd, 0xf6, 0x28, 0x94, 0x78, 0x6c, 0x15, 0x64, 0xf6, 0xdf, 0xb9, 0x6a, 0x30, 0x03, 0x5f, - 0x43, 0x84, 0x2f, 0x64, 0x02, 0x7d, 0x02, 0x1b, 0xde, 0xf1, 0x6c, 0x34, 0xb2, 0x30, 0x4d, 0x37, - 0x2a, 0x4f, 0x0a, 0xb1, 0x9d, 0xe8, 0x6e, 0x7e, 0xff, 0xbd, 0xab, 0xec, 0xf6, 0xb9, 0x22, 0x9b, - 0xe2, 0xba, 0xb0, 0xf2, 0x0c, 0xcf, 0x07, 0x2c, 0x97, 0x3c, 0x84, 0xbc, 0x77, 0xac, 0xb9, 0xc6, - 0x22, 0x98, 0x71, 0x16, 0x8e, 0xcd, 0x8a, 0x5f, 0x68, 0x54, 0xfa, 0x94, 0xcf, 0x42, 0x99, 0x63, - 0xa2, 0x41, 0x18, 0x5b, 0x70, 0xcb, 0xc5, 0xba, 0x73, 0x8a, 0xdd, 0xb9, 0x6a, 0x39, 0xe3, 0x85, - 0x89, 0xc4, 0x65, 0xeb, 0x61, 0xd3, 0xd7, 0x69, 0x3b, 0xe3, 0xc0, 0xd4, 0xb7, 0x20, 0x67, 0x73, - 0x2c, 0x51, 0xa7, 0x8e, 0x4b, 0xbc, 0x42, 0x92, 0xf9, 0x6c, 0x83, 0xcf, 0x2d, 0x04, 0x33, 0x4a, - 0xd6, 0x5e, 0x74, 0xbc, 0xe2, 0x4f, 0xe2, 0x90, 0x0e, 0xfc, 0x86, 0xee, 0x2f, 0x61, 0x44, 0xee, - 0xf5, 0xab, 0xd2, 0x82, 0x29, 0x20, 0xe2, 0x21, 0x80, 0x1e, 0x78, 0x8a, 0xc5, 0x34, 0xb3, 0xbf, - 0x75, 0x91, 0x07, 0xfd, 0x20, 0x2c, 0xa4, 0x51, 0x3b, 0xbc, 0xf8, 0x3d, 0x6c, 0xb1, 0xa5, 0x22, - 0x72, 0xf0, 0x9d, 0xc5, 0x64, 0xdb, 0xda, 0x10, 0x5b, 0x7d, 0xc1, 0x16, 0x66, 0x16, 0x9b, 0xc0, - 0x67, 0x70, 0x5c, 0x34, 0x1d, 0xd7, 0x24, 0xbc, 0xcc, 0xc8, 0x29, 0x41, 0x1f, 0x7d, 0x1d, 0x10, - 0x83, 0x20, 0x03, 0x5b, 0xda, 0x5c, 0xf5, 0xb0, 0xee, 0xd8, 0x0c, 0x13, 0xa8, 0x14, 0x03, 0xa7, - 0x06, 0x65, 0xf4, 0x39, 0x9d, 0xe6, 0x81, 0xd0, 0xe2, 0x60, 0xa0, 0x90, 0x56, 0x60, 0x11, 0x69, - 0x74, 0x04, 0x45, 0x5f, 0xc0, 0xd2, 0x26, 0x43, 0x43, 0x53, 0xc3, 0x0b, 0xfb, 0x3a, 0x20, 0xb0, - 0x29, 0xf4, 0xdb, 0x4c, 0xbd, 0xce, 0x57, 0xf9, 0x01, 0xa0, 0x0b, 0xcc, 0x5d, 0x07, 0x6f, 0xb3, - 0x56, 0xd8, 0xce, 0x57, 0x20, 0xcd, 0x66, 0xeb, 0xd8, 0xd6, 0x9c, 0x61, 0x49, 0x4a, 0x49, 0x51, - 0x42, 0xd7, 0xb6, 0xe6, 0xa8, 0x02, 0x9b, 0x2f, 0xf8, 0xea, 0x51, 0x39, 0x2a, 0xcf, 0x46, 0x23, - 0xf3, 0xac, 0x00, 0x0c, 0x42, 0x37, 0x04, 0x4b, 0xa1, 0xc8, 0xcb, 0x18, 0xe8, 0x01, 0x80, 0xed, - 0x10, 0x75, 0x88, 0x47, 0x8e, 0x8b, 0x19, 0x6e, 0x67, 0xf6, 0x8b, 0x15, 0x5e, 0xfd, 0x56, 0xfc, - 0xea, 0xb7, 0x32, 0xf0, 0xab, 0x5f, 0x25, 0x6d, 0x3b, 0xa4, 0xc6, 0x84, 0xd1, 0xef, 0x03, 0xed, - 0xa8, 0xda, 0x88, 0x60, 0x57, 0xa0, 0xd2, 0x65, 0x9a, 0x29, 0xdb, 0x21, 0x55, 0x2a, 0x4b, 0x43, - 0x39, 0xd4, 0xf4, 0x93, 0x91, 0x69, 0x59, 0x85, 0x1c, 0x0f, 0xa5, 0xdf, 0x2f, 0x7f, 0x1f, 0x72, - 0x4b, 0x28, 0x83, 0x8a, 0x70, 0x5b, 0x24, 0x5c, 0xb5, 0xde, 0xed, 0x74, 0x9a, 0xf5, 0x41, 0x57, - 0x51, 0x07, 0xcf, 0x7b, 0x4d, 0x79, 0x8d, 0xa6, 0xd7, 0xfe, 0x1f, 0xb6, 0x5b, 0x83, 0xa6, 0x2c, - 0xa1, 0x3c, 0x00, 0xa5, 0xf6, 0xeb, 0x4a, 0xab, 0x37, 0x90, 0x23, 0x28, 0x0d, 0xf1, 0xd6, 0x61, - 0xf5, 0x49, 0x53, 0x8e, 0xd2, 0x66, 0xbb, 0x5b, 0xaf, 0xb6, 0xe5, 0x58, 0xf9, 0x10, 0x32, 0xa1, - 0xfd, 0x8d, 0x0a, 0xb0, 0xe5, 0x1b, 0xef, 0x7f, 0x7c, 0x74, 0x70, 0xd0, 0x6e, 0xfa, 0xa6, 0x33, - 0x90, 0xac, 0x75, 0xbb, 0xed, 0x66, 0xb5, 0x23, 0x4b, 0x3c, 0xe9, 0x0f, 0x9a, 0x4f, 0x9a, 0x8a, - 0x1c, 0x61, 0x1f, 0x1d, 0x28, 0xad, 0xce, 0x13, 0x39, 0x5a, 0xfe, 0x51, 0x04, 0xf2, 0x2c, 0xd5, - 0xf0, 0x65, 0x4a, 0x57, 0x3d, 0x82, 0xd8, 0x09, 0x9e, 0xfb, 0x25, 0x28, 0x6b, 0xd3, 0xbc, 0x7d, - 0xaa, 0x59, 0x33, 0xec, 0x15, 0x22, 0x8c, 0x2a, 0x7a, 0xd4, 0x11, 0x86, 0xa3, 0xcf, 0x26, 0xd8, - 0x26, 0xa2, 0xd4, 0x0a, 0xfa, 0xe8, 0x14, 0x6e, 0xb1, 0xdc, 0x16, 0x5e, 0x2c, 0xea, 0x44, 0x9b, - 0x32, 0x18, 0xcb, 0xec, 0x7f, 0x9d, 0x6f, 0xc2, 0xe5, 0x8f, 0xf3, 0x2e, 0x5f, 0x26, 0x4f, 0x3d, - 0xc7, 0x3e, 0xd4, 0xa6, 0x4d, 0x9b, 0xb8, 0xf3, 0xda, 0x5b, 0x9f, 0x7e, 0x71, 0x59, 0xca, 0x1a, - 0x2d, 0xd4, 0x8a, 0x4d, 0xb8, 0xf3, 0x06, 0x2b, 0x7e, 0x96, 0x16, 0xa7, 0x06, 0x9a, 0xa5, 0xb7, - 0x20, 0xce, 0xa6, 0xe2, 0x9f, 0x1a, 0x58, 0xe7, 0x61, 0xe4, 0x23, 0xa9, 0xfc, 0x7f, 0x71, 0xc8, - 0xd4, 0xb5, 0x29, 0x99, 0xb9, 0x98, 0xd5, 0x23, 0xa5, 0x25, 0xac, 0xc9, 0xbc, 0x7e, 0x55, 0x4a, - 0x0a, 0xb6, 0x40, 0x9a, 0x83, 0x95, 0xa4, 0x14, 0x61, 0x49, 0xc9, 0x4f, 0x74, 0x0b, 0x5b, 0x37, - 0x4a, 0x43, 0xd1, 0x1b, 0xa4, 0xa1, 0x6f, 0x42, 0x6a, 0x68, 0xda, 0x54, 0xc4, 0x13, 0x5e, 0xbe, - 0xbb, 0xfa, 0xf1, 0x1a, 0x97, 0x50, 0x02, 0x51, 0x5a, 0x8f, 0x98, 0x36, 0xc1, 0xee, 0xa9, 0x66, - 0x9d, 0xc3, 0x9e, 0x75, 0x9f, 0xee, 0x43, 0xcf, 0x6a, 0xfa, 0x48, 0xfc, 0xfa, 0xe9, 0x23, 0xf9, - 0xeb, 0xa7, 0x8f, 0xd4, 0xf5, 0xd2, 0xc7, 0xff, 0x48, 0x90, 0x14, 0xf3, 0x47, 0x4f, 0x61, 0xcb, - 0xc5, 0x9e, 0x33, 0x73, 0x75, 0xbc, 0x84, 0x67, 0xd2, 0x35, 0x1c, 0x9e, 0xf7, 0x35, 0x05, 0xa2, - 0x7d, 0x15, 0x72, 0x81, 0xad, 0xa9, 0x46, 0x8e, 0xc5, 0x36, 0xc9, 0xfa, 0xc4, 0x9e, 0x46, 0x8e, - 0xcf, 0xa5, 0xa2, 0xe8, 0x8d, 0x52, 0x51, 0x18, 0x71, 0x62, 0xcb, 0x88, 0x43, 0xe1, 0xd4, 0x23, - 0xb4, 0xc4, 0xf3, 0x0f, 0xa6, 0x69, 0x25, 0xc5, 0x08, 0xcf, 0xf0, 0xbc, 0xbc, 0x7f, 0x1e, 0x8e, - 0x96, 0xea, 0xff, 0x00, 0x63, 0x92, 0x0b, 0x8c, 0x49, 0x95, 0xff, 0x19, 0x60, 0xf3, 0x50, 0x23, - 0xd8, 0x35, 0x35, 0xcb, 0xfc, 0x81, 0x16, 0x94, 0xe4, 0x6f, 0x2f, 0x6d, 0x81, 0xcd, 0xd7, 0xaf, - 0x4a, 0xeb, 0xe7, 0xc4, 0xc4, 0x56, 0xe8, 0xbc, 0x61, 0x2b, 0xbc, 0xcd, 0x67, 0x7b, 0x81, 0xed, - 0xdf, 0xda, 0x96, 0x78, 0xbc, 0xb2, 0x25, 0xee, 0xbf, 0x79, 0x10, 0xab, 0x5b, 0xe3, 0x77, 0xbc, - 0x5c, 0xfa, 0x97, 0xc4, 0xef, 0xe8, 0x7a, 0xbf, 0xb8, 0xf4, 0x4a, 0xfe, 0x06, 0x4a, 0xaf, 0xf4, - 0xb9, 0xd2, 0xab, 0x0e, 0xeb, 0x3c, 0x4d, 0x79, 0x7e, 0x12, 0x62, 0x1b, 0x2c, 0x18, 0xea, 0x72, - 0x82, 0x12, 0xdf, 0xc8, 0x8f, 0x96, 0x73, 0xe6, 0x57, 0x21, 0x67, 0x60, 0x8b, 0x68, 0xea, 0x6c, - 0x6a, 0x68, 0x04, 0x7b, 0xfe, 0xfd, 0x10, 0x23, 0x1e, 0x71, 0x1a, 0x1a, 0x02, 0x32, 0xf0, 0xd4, - 0xc5, 0xba, 0x46, 0xb0, 0xa1, 0x8a, 0x02, 0x4b, 0x2c, 0x86, 0x0f, 0xaf, 0x5c, 0x94, 0x95, 0x46, - 0xa0, 0x2b, 0x0a, 0x00, 0x65, 0xc3, 0x38, 0x4f, 0x7a, 0x53, 0xf5, 0x94, 0xba, 0x5e, 0xf5, 0x04, - 0x5f, 0xba, 0x7a, 0xca, 0x7c, 0xc9, 0xea, 0x29, 0x7b, 0x19, 0x96, 0xe5, 0x96, 0xb1, 0xac, 0xf8, - 0x27, 0x12, 0x6c, 0xac, 0x78, 0x01, 0xdd, 0x03, 0x18, 0xbb, 0xce, 0x6c, 0xaa, 0x2e, 0xb0, 0x49, - 0x49, 0x33, 0x4a, 0x87, 0x62, 0xd1, 0x6f, 0xb4, 0x88, 0xbf, 0x2e, 0x9c, 0xa6, 0x16, 0x70, 0x9a, - 0x2e, 0xff, 0x77, 0x1a, 0x12, 0xdd, 0xea, 0x8c, 0x1c, 0xef, 0x8b, 0xbb, 0x91, 0x53, 0xd3, 0xc0, - 0xfe, 0xdd, 0x65, 0xd0, 0x47, 0xef, 0xc2, 0x86, 0x36, 0x23, 0xc7, 0xea, 0xcc, 0xb5, 0x16, 0x50, - 0xc1, 0xcb, 0x92, 0x75, 0xca, 0x38, 0x72, 0xad, 0x00, 0x0f, 0x1e, 0xc0, 0x5d, 0x4d, 0xd7, 0xb1, - 0xe7, 0xa9, 0xc4, 0x39, 0xc1, 0xf6, 0xb2, 0x0e, 0x2f, 0xc4, 0x6e, 0x73, 0x81, 0x01, 0xe5, 0x87, - 0x55, 0x2b, 0xb0, 0xb9, 0xa4, 0x3a, 0xc1, 0xe4, 0xd8, 0x31, 0xf8, 0x85, 0x87, 0xb2, 0x11, 0x52, - 0x3a, 0x64, 0x0c, 0x36, 0xac, 0xb0, 0xfc, 0xd0, 0x31, 0xe6, 0xe2, 0x42, 0x63, 0x3d, 0x24, 0x5d, - 0x73, 0x8c, 0x39, 0xfa, 0x33, 0x09, 0xee, 0x2d, 0x09, 0x1f, 0x63, 0xcd, 0xc0, 0xae, 0xb7, 0xa8, - 0xfd, 0xe2, 0x0c, 0xb7, 0xc4, 0x11, 0x96, 0x3b, 0xa5, 0x52, 0x5d, 0x58, 0xf9, 0x98, 0xcb, 0xdf, - 0xa0, 0xf4, 0x43, 0xda, 0x8a, 0x36, 0xfa, 0x54, 0x82, 0xed, 0xa5, 0x61, 0xb8, 0xd8, 0x9b, 0x3a, - 0xb6, 0x87, 0x17, 0xe3, 0x48, 0x84, 0x6b, 0xd0, 0xd5, 0x71, 0x28, 0x42, 0xe1, 0x06, 0x03, 0x09, - 0xfb, 0xdb, 0x57, 0x3f, 0xd4, 0xa6, 0xe8, 0x11, 0x14, 0x5d, 0x3c, 0x72, 0xb1, 0x77, 0x7c, 0x51, - 0xac, 0xf8, 0x05, 0xe3, 0x1d, 0x21, 0xb1, 0x12, 0xac, 0x0f, 0x28, 0x66, 0x87, 0x95, 0x45, 0xb4, - 0xb2, 0x4c, 0x0d, 0x85, 0xd5, 0x44, 0xb8, 0xd8, 0x49, 0x32, 0xac, 0xc1, 0xe2, 0xc5, 0xf7, 0xbf, - 0x1c, 0x96, 0x67, 0x01, 0xfb, 0x73, 0x09, 0xb6, 0x97, 0xc5, 0x57, 0x22, 0x96, 0xbe, 0xc0, 0x53, - 0x4a, 0xc8, 0xce, 0xcd, 0x43, 0xb6, 0xe9, 0xae, 0xaa, 0xa3, 0xbf, 0x94, 0xa0, 0xb4, 0x3c, 0x92, - 0xd5, 0xa0, 0x01, 0x1b, 0xca, 0xfb, 0x6f, 0x1c, 0xca, 0x97, 0x88, 0xda, 0x92, 0xe7, 0x43, 0x61, - 0x2b, 0xb6, 0x61, 0xfb, 0xf2, 0x75, 0x79, 0x93, 0xc3, 0x44, 0xf1, 0x10, 0x4a, 0x57, 0xac, 0xae, - 0x9b, 0x9a, 0xbb, 0x22, 0x04, 0x37, 0x32, 0xd7, 0x81, 0x9d, 0xab, 0xdc, 0x78, 0xa3, 0xa3, 0xd3, - 0xdf, 0x45, 0x21, 0x35, 0xc0, 0x1e, 0x61, 0x45, 0x23, 0x0a, 0x17, 0x8d, 0xa2, 0x3e, 0xdc, 0x83, - 0xb8, 0x47, 0xf0, 0x94, 0x9f, 0x26, 0x69, 0x31, 0xc5, 0x22, 0xea, 0xab, 0x54, 0xfa, 0x04, 0x4f, - 0x05, 0x04, 0x73, 0xb9, 0xe2, 0x2f, 0x23, 0x10, 0xa3, 0x54, 0xf4, 0x4d, 0x9a, 0x1f, 0xf0, 0x34, - 0x7c, 0xe9, 0x57, 0xb8, 0x40, 0xbb, 0xc2, 0xaa, 0xc8, 0x14, 0x15, 0x65, 0x28, 0x7d, 0x0f, 0x80, - 0xa9, 0x99, 0xb6, 0x81, 0xcf, 0xc4, 0xa3, 0x16, 0x33, 0xd4, 0xa2, 0x84, 0xf3, 0x4f, 0x23, 0xd1, - 0xd5, 0xa7, 0x11, 0xdf, 0x80, 0xa7, 0x3b, 0x53, 0xff, 0x5d, 0x85, 0x19, 0xe8, 0x53, 0x02, 0xaa, - 0x2c, 0x95, 0x3a, 0xf1, 0x0b, 0xaf, 0xac, 0xc3, 0xe5, 0xcd, 0x03, 0xc8, 0x19, 0x8e, 0x2e, 0x76, - 0xd9, 0x29, 0xd6, 0xf9, 0x1d, 0xce, 0x15, 0x45, 0x58, 0x8c, 0xaa, 0xa0, 0xc7, 0x00, 0x41, 0x5a, - 0xf2, 0xae, 0x57, 0x11, 0x85, 0x14, 0xca, 0xdb, 0x10, 0x63, 0x1e, 0x01, 0x48, 0xb4, 0x3a, 0x4f, - 0x9a, 0xfd, 0x01, 0xbf, 0x85, 0xf8, 0x4e, 0x53, 0x69, 0x1d, 0x3c, 0x97, 0xa5, 0xf2, 0x8f, 0x24, - 0x48, 0x2b, 0x9a, 0x3d, 0xe6, 0x87, 0xde, 0xaf, 0x40, 0xfa, 0x04, 0xcf, 0x55, 0xfe, 0x3e, 0x48, - 0xdd, 0x96, 0x54, 0x52, 0x27, 0x78, 0x5e, 0x63, 0x4f, 0x84, 0x77, 0x20, 0x49, 0x99, 0xd8, 0x36, - 0x98, 0xc7, 0x92, 0x4a, 0xe2, 0x04, 0xcf, 0x9b, 0xb6, 0x81, 0xca, 0x90, 0x73, 0x55, 0xf6, 0x90, - 0x27, 0x34, 0x63, 0x8c, 0x9d, 0x71, 0xd9, 0x1b, 0x1f, 0x57, 0xde, 0x86, 0x8c, 0x2f, 0x43, 0x0d, - 0xc4, 0x99, 0x44, 0x9a, 0x4b, 0x34, 0x6d, 0xe3, 0xa1, 0xfc, 0xe3, 0xcf, 0x4a, 0x6b, 0xe7, 0x5e, - 0x1c, 0xf3, 0x41, 0xea, 0xed, 0xd3, 0x92, 0x00, 0x3d, 0x82, 0x2c, 0xaf, 0xb7, 0x8c, 0xeb, 0x57, - 0xb2, 0x49, 0xa1, 0x81, 0x4a, 0x90, 0x99, 0x60, 0x77, 0xcc, 0xea, 0x57, 0xfd, 0x98, 0x4d, 0x2e, - 0xa5, 0x00, 0x23, 0xf5, 0x28, 0xa5, 0xfc, 0xb7, 0x12, 0xa4, 0xfb, 0xd8, 0xed, 0x39, 0x96, 0xa9, - 0xb3, 0x1b, 0x3a, 0x8f, 0xb8, 0x2a, 0x71, 0x67, 0x36, 0xad, 0x3e, 0x44, 0xd9, 0xc3, 0x5f, 0x3e, - 0x64, 0x8f, 0xb8, 0x03, 0xc1, 0xe0, 0x25, 0xce, 0x03, 0xb8, 0x6b, 0x63, 0x8f, 0x0e, 0xcc, 0x19, - 0xbe, 0x38, 0xaf, 0xc4, 0x97, 0xdf, 0x6d, 0x2e, 0xd0, 0x1d, 0xbe, 0x58, 0x56, 0xfd, 0x00, 0xb6, - 0x34, 0xd7, 0xd5, 0xe6, 0xe7, 0xb5, 0xa2, 0x4c, 0x0b, 0x31, 0xde, 0x92, 0x46, 0xf9, 0x6f, 0x22, - 0x00, 0xcd, 0x33, 0xe2, 0x6a, 0x3a, 0xa9, 0xf6, 0x5a, 0xc5, 0x7f, 0x90, 0x20, 0x21, 0xca, 0xf4, - 0xf0, 0x1b, 0x84, 0xb4, 0xfc, 0x06, 0xf1, 0x18, 0x32, 0xe1, 0xf7, 0x8e, 0xeb, 0x5c, 0x76, 0x03, - 0x57, 0xa0, 0xb0, 0x40, 0xf7, 0x03, 0xaf, 0x9a, 0xa7, 0xc4, 0xf5, 0xc4, 0xb3, 0x47, 0x9a, 0x51, - 0x7a, 0xc4, 0x5d, 0x79, 0xb6, 0x88, 0x5d, 0xfb, 0xd9, 0xa2, 0xfc, 0x1c, 0x62, 0x75, 0xc7, 0x38, - 0x57, 0x57, 0xe5, 0x20, 0x5d, 0xef, 0x76, 0x0e, 0x5a, 0x4f, 0x8e, 0x94, 0x26, 0xbf, 0xcd, 0x6a, - 0x7e, 0x77, 0xa0, 0x54, 0xeb, 0x03, 0x39, 0x82, 0x10, 0xe4, 0x45, 0xa7, 0xd9, 0x50, 0x8f, 0x8e, - 0x5a, 0x0d, 0x39, 0x8a, 0xb6, 0x40, 0x5e, 0xd0, 0x0e, 0x5a, 0xcd, 0x76, 0xa3, 0x2f, 0xc7, 0xca, - 0xbf, 0x94, 0x20, 0x55, 0x9b, 0x99, 0x96, 0x41, 0xbd, 0xf4, 0x1f, 0x4b, 0x5e, 0x1a, 0x52, 0xb2, - 0x6a, 0x1a, 0xbe, 0x97, 0x58, 0xbf, 0x65, 0x2c, 0x58, 0xc6, 0x50, 0xc0, 0x1c, 0x67, 0x35, 0x86, - 0xec, 0xbd, 0x93, 0x9d, 0x75, 0x04, 0x5c, 0x88, 0x1e, 0xda, 0x87, 0x8c, 0x38, 0x18, 0x31, 0x8c, - 0x8a, 0x31, 0x8c, 0xda, 0xf0, 0x8f, 0x3d, 0xc1, 0x33, 0xa4, 0x02, 0x5c, 0x8a, 0x6d, 0xc6, 0xf7, - 0x60, 0x63, 0x71, 0x5e, 0x16, 0x07, 0x38, 0x71, 0x92, 0x97, 0x03, 0x86, 0x38, 0xe5, 0xa1, 0xfb, - 0x90, 0x15, 0x0e, 0x53, 0x5d, 0xc7, 0x21, 0xfc, 0x95, 0x36, 0x70, 0xa2, 0xe2, 0x38, 0xa4, 0xbc, - 0x09, 0x1b, 0x0a, 0xf6, 0x30, 0x61, 0xdb, 0x44, 0xc1, 0x7f, 0x34, 0xc3, 0x1e, 0x29, 0x6f, 0x01, - 0x0a, 0x13, 0x39, 0xc6, 0x97, 0x1f, 0x03, 0xaa, 0x1a, 0xa7, 0x9a, 0xad, 0x63, 0x5a, 0xaa, 0x0b, - 0x59, 0xf4, 0x36, 0xac, 0x6b, 0x9c, 0x1a, 0x5c, 0x08, 0x51, 0xcf, 0xc4, 0x94, 0xbc, 0x20, 0x8b, - 0xfb, 0xa0, 0xf2, 0x2d, 0xd8, 0x5c, 0x52, 0x17, 0x56, 0x3f, 0x82, 0x6c, 0xc3, 0xd1, 0xbd, 0xaa, - 0x6d, 0xd4, 0xe6, 0xf4, 0xe8, 0x83, 0x80, 0x81, 0x97, 0xd8, 0x2f, 0x1c, 0xc8, 0xb6, 0x20, 0x3e, - 0xa4, 0x4c, 0xe6, 0xd8, 0x98, 0xc2, 0x3b, 0xe5, 0xbf, 0x92, 0x20, 0xd7, 0xb2, 0xc7, 0xd8, 0x23, - 0xfe, 0x58, 0x96, 0xb1, 0x55, 0xba, 0x12, 0x5b, 0xc3, 0xe1, 0x8c, 0x2c, 0x87, 0x73, 0x05, 0x76, - 0xa3, 0xd7, 0x85, 0xdd, 0xf2, 0xbf, 0x46, 0x20, 0xef, 0x8f, 0x8b, 0x4f, 0x12, 0xfd, 0xbd, 0xb4, - 0x38, 0x6c, 0xf1, 0x67, 0x48, 0x5a, 0x3c, 0xf1, 0x8b, 0xd3, 0xa0, 0xc6, 0x5d, 0xd6, 0xf1, 0x4f, - 0xfa, 0x9f, 0x50, 0x71, 0x9a, 0xb6, 0x3d, 0x5e, 0xa4, 0x7c, 0xff, 0x4f, 0xbf, 0x28, 0xbd, 0x77, - 0xd1, 0xbf, 0x23, 0xce, 0xfd, 0x3b, 0xc4, 0xd7, 0xff, 0xf4, 0x8b, 0xd2, 0xbb, 0xd7, 0x11, 0xef, - 0x8e, 0x46, 0x1e, 0x26, 0xc1, 0xc1, 0x6f, 0xf1, 0x59, 0xf4, 0x07, 0x90, 0xf5, 0xc7, 0x8e, 0x89, - 0x6e, 0x88, 0x97, 0x91, 0xd0, 0x9d, 0x04, 0x2f, 0x2a, 0x2a, 0x4d, 0xa2, 0x1b, 0xfe, 0x3e, 0x15, - 0x0a, 0x94, 0x54, 0x6c, 0xc0, 0xed, 0x8b, 0xe7, 0x72, 0x55, 0xa5, 0x10, 0x0d, 0x57, 0x0a, 0x3f, - 0x8b, 0xc0, 0x9d, 0x81, 0xe6, 0x9d, 0xf8, 0x37, 0x18, 0xae, 0x73, 0x36, 0xf7, 0xe3, 0xfe, 0x2d, - 0x88, 0x39, 0x53, 0xcc, 0x23, 0x9e, 0xd9, 0x2f, 0x8b, 0x2c, 0x7f, 0xb1, 0x70, 0xa5, 0x3b, 0xc5, - 0xb6, 0xc2, 0xe4, 0xd9, 0x5a, 0xd3, 0x88, 0xc6, 0x3e, 0x96, 0x55, 0x58, 0xbb, 0xf8, 0x8f, 0x12, - 0xc4, 0xa8, 0x08, 0xda, 0x85, 0x04, 0xaf, 0x70, 0x85, 0x59, 0xf9, 0xfc, 0x84, 0x15, 0xc1, 0x47, - 0x6d, 0x48, 0xf1, 0x9b, 0x1f, 0x7f, 0x19, 0xd5, 0xbe, 0xf1, 0xfa, 0x55, 0xe9, 0xfd, 0x8b, 0xbc, - 0xbe, 0xf2, 0x17, 0x1d, 0x7e, 0x27, 0xd4, 0x6a, 0x28, 0x49, 0x66, 0xa2, 0xc5, 0xb2, 0x0d, 0xd1, - 0xdc, 0x31, 0x26, 0xec, 0xfe, 0x46, 0x80, 0x39, 0x70, 0x12, 0x7b, 0x4d, 0x2f, 0x41, 0x46, 0xb7, - 0x4c, 0x6c, 0x13, 0x55, 0x33, 0x0c, 0xff, 0xc5, 0x18, 0x38, 0xa9, 0x6a, 0x18, 0x6e, 0xf9, 0x2f, - 0xa2, 0x50, 0x58, 0x9d, 0xbd, 0x58, 0x8a, 0x3d, 0xc8, 0xd1, 0xb9, 0x07, 0x35, 0xb3, 0x98, 0xdd, - 0x7b, 0x6f, 0x72, 0x9a, 0x58, 0x8d, 0xcc, 0x6b, 0xa2, 0xa3, 0x64, 0x9d, 0x50, 0xef, 0x42, 0x2f, - 0xbe, 0x84, 0x6c, 0x58, 0x03, 0x3d, 0x86, 0x04, 0x3d, 0x9b, 0xcf, 0x3c, 0x51, 0x89, 0x7d, 0xed, - 0x8a, 0xcf, 0xf5, 0x99, 0xb0, 0x22, 0x94, 0x42, 0xb1, 0x88, 0x5c, 0x1e, 0x8b, 0xf2, 0x0f, 0xd9, - 0x1f, 0x3c, 0x98, 0x52, 0x02, 0x22, 0xdd, 0x67, 0xf2, 0x1a, 0xda, 0x84, 0xf5, 0xfe, 0xc7, 0x55, - 0xa5, 0xa1, 0x76, 0xba, 0x03, 0xf5, 0xa0, 0x7b, 0xd4, 0x69, 0xc8, 0x12, 0xc5, 0xfd, 0x4e, 0x57, - 0xe5, 0xf4, 0x9e, 0xd2, 0x3a, 0xac, 0x2a, 0xcf, 0xe5, 0x08, 0xba, 0x05, 0x1b, 0x54, 0x68, 0x99, - 0x1c, 0xa5, 0x89, 0xa3, 0xd5, 0x19, 0x34, 0x95, 0x4e, 0xb5, 0xad, 0x36, 0x15, 0xa5, 0xab, 0xc8, - 0x31, 0xb4, 0x01, 0x39, 0x2e, 0xd6, 0x1f, 0x74, 0x7b, 0xbd, 0x66, 0x43, 0x8e, 0xa3, 0x5b, 0x20, - 0xf7, 0xba, 0xca, 0x80, 0x7d, 0xa7, 0xda, 0x6e, 0x77, 0x3f, 0x69, 0x36, 0xe4, 0x5f, 0x25, 0xdf, - 0xed, 0x40, 0x26, 0x84, 0xe6, 0x34, 0x25, 0xd5, 0xab, 0x83, 0x6a, 0xbb, 0xfb, 0x44, 0x5e, 0x43, - 0xeb, 0x90, 0x79, 0xda, 0xef, 0x76, 0xd4, 0x7e, 0xfd, 0xe3, 0xe6, 0x61, 0x55, 0x96, 0xd8, 0xbf, - 0x28, 0x58, 0xfe, 0x92, 0x63, 0x74, 0x34, 0x8d, 0x6e, 0xfd, 0xe8, 0xb0, 0xd9, 0x19, 0xf4, 0xd5, - 0x83, 0xd6, 0x77, 0x07, 0x34, 0xa7, 0xc5, 0xf7, 0xff, 0x49, 0x82, 0x24, 0x2d, 0x61, 0x4d, 0x7b, - 0x8c, 0xbe, 0x0d, 0xb0, 0x40, 0x6a, 0x24, 0xfe, 0xff, 0xb2, 0x02, 0xe8, 0xc5, 0xc2, 0x2a, 0x43, - 0x04, 0xa6, 0x06, 0x99, 0x10, 0x2a, 0x23, 0x21, 0xb8, 0x8a, 0xf3, 0xc5, 0xbb, 0x17, 0x70, 0x84, - 0x8d, 0x0f, 0x21, 0xc1, 0xb1, 0x0b, 0x6d, 0x2e, 0x23, 0x19, 0xd7, 0xdc, 0xba, 0x08, 0xde, 0xf6, - 0xbf, 0x07, 0xd9, 0x70, 0xe4, 0xd1, 0x53, 0x88, 0xf3, 0xc6, 0xbd, 0x4b, 0xb7, 0x6f, 0x71, 0xfb, - 0xf2, 0x95, 0xb3, 0x2b, 0x7d, 0x20, 0xd5, 0x1e, 0x7d, 0xfe, 0x9f, 0xdb, 0x6b, 0x9f, 0xff, 0x62, - 0x5b, 0xfa, 0xf9, 0x2f, 0xb6, 0xa5, 0xcf, 0xfe, 0x6b, 0x5b, 0xfa, 0xde, 0x3b, 0x63, 0x93, 0x1c, - 0xcf, 0x86, 0x15, 0xdd, 0x99, 0xec, 0x61, 0x8f, 0xcc, 0x34, 0x77, 0xce, 0xff, 0x65, 0xb7, 0xf2, - 0xbf, 0xbb, 0x61, 0x82, 0xf5, 0x3f, 0xfc, 0xff, 0x00, 0x00, 0x00, 0xff, 0xff, 0x3b, 0xfb, 0x89, - 0xba, 0x93, 0x27, 0x00, 0x00, + 0x48, 0xe0, 0x7f, 0x21, 0xc9, 0x25, 0x40, 0x2e, 0x7b, 0x0a, 0x0c, 0x04, 0x49, 0x80, 0x1c, 0x66, + 0x11, 0xe7, 0x60, 0x9f, 0x92, 0x5b, 0x10, 0xcc, 0x29, 0xa8, 0x47, 0x37, 0x9b, 0xa2, 0x46, 0x8f, + 0xb5, 0x7d, 0xf0, 0x85, 0xa8, 0xfa, 0x5e, 0x5d, 0xf5, 0x7d, 0x55, 0xbf, 0xef, 0xab, 0x2a, 0xc2, + 0x5b, 0x63, 0x67, 0x6f, 0xea, 0x3a, 0xc4, 0xd1, 0x1d, 0xcb, 0xdb, 0x1b, 0x59, 0xce, 0x4b, 0xf6, + 0x53, 0x61, 0x34, 0x14, 0xa3, 0xed, 0xe2, 0xf6, 0xd0, 0x75, 0x4e, 0xb0, 0x1b, 0xc8, 0x05, 0x0d, + 0x2e, 0x55, 0xdc, 0xd1, 0x1d, 0xdb, 0x9b, 0x4d, 0x2e, 0x91, 0xd8, 0x1a, 0x3b, 0x63, 0x87, 0x35, + 0xf7, 0x68, 0x4b, 0x50, 0x4b, 0x63, 0xc7, 0x19, 0x5b, 0x98, 0x0b, 0x0f, 0x67, 0xa3, 0x3d, 0x62, + 0x4e, 0xb0, 0x47, 0xb4, 0xc9, 0x94, 0x0b, 0x94, 0x1f, 0x43, 0xbc, 0x6f, 0x99, 0x3a, 0x46, 0x5b, + 0x10, 0x1f, 0xe2, 0xb1, 0x69, 0x17, 0xa4, 0x1d, 0x69, 0x37, 0xa7, 0xf0, 0x0e, 0x92, 0x21, 0x8a, + 0x6d, 0xa3, 0x10, 0x61, 0x34, 0xda, 0x7c, 0x98, 0xfd, 0xf9, 0x5f, 0x97, 0xd6, 0x7e, 0xf8, 0xd3, + 0xd2, 0xda, 0x8f, 0x7f, 0x5a, 0x5a, 0x2b, 0xdb, 0x90, 0x3e, 0x3a, 0x6a, 0x35, 0x7a, 0x9a, 0x4b, + 0x3c, 0x84, 0x20, 0x66, 0x3b, 0x06, 0x66, 0x16, 0x12, 0x0a, 0x6b, 0xa3, 0x47, 0x10, 0xd7, 0x2d, + 0x47, 0x3f, 0x61, 0x26, 0x12, 0xb5, 0xaf, 0xbd, 0x7e, 0x55, 0xba, 0x3f, 0x76, 0x2a, 0x63, 0xed, + 0x07, 0x98, 0x10, 0x5c, 0x31, 0xf0, 0xe9, 0x9e, 0xee, 0xb8, 0x78, 0x6f, 0x82, 0x3d, 0x4f, 0x1b, + 0xe3, 0x4a, 0x9d, 0x0a, 0x2b, 0x5c, 0xe7, 0xa1, 0xfc, 0xab, 0xcf, 0x4a, 0xd2, 0xd2, 0xf7, 0xfe, + 0x55, 0x02, 0xe8, 0xb9, 0xce, 0x0b, 0xac, 0x13, 0xd3, 0x61, 0xc3, 0x9b, 0x12, 0x97, 0x7d, 0x30, + 0xad, 0xd0, 0x26, 0x9d, 0xc6, 0xc8, 0xc4, 0x16, 0x1f, 0x72, 0x5a, 0xe1, 0x1d, 0x54, 0x84, 0x14, + 0x3e, 0x9b, 0x5a, 0xa6, 0x6e, 0x92, 0x42, 0x74, 0x47, 0xda, 0x4d, 0x29, 0x41, 0x1f, 0xed, 0x82, + 0x6c, 0x7a, 0xea, 0x54, 0x73, 0x89, 0x49, 0x6d, 0xaa, 0x27, 0x78, 0x5e, 0x88, 0x31, 0x99, 0xbc, + 0xe9, 0xf5, 0x7c, 0xf2, 0x33, 0x3c, 0x47, 0xbf, 0x07, 0x79, 0x2a, 0xe9, 0x9a, 0x13, 0xcd, 0x9d, + 0x33, 0xb9, 0x38, 0x93, 0xcb, 0x9a, 0x5e, 0x8f, 0x13, 0xa9, 0xd4, 0x87, 0x90, 0x36, 0xed, 0x11, + 0x76, 0xb1, 0xad, 0xe3, 0x42, 0x62, 0x47, 0xda, 0xcd, 0xec, 0xaf, 0x57, 0x58, 0xc0, 0x5b, 0x3e, + 0xb9, 0x16, 0xfb, 0xfc, 0x55, 0x69, 0x4d, 0x59, 0xc8, 0x95, 0x7f, 0x16, 0x87, 0x74, 0xc0, 0xa6, + 0x93, 0x20, 0xf3, 0x29, 0xf6, 0x0a, 0xd2, 0x4e, 0x94, 0x4e, 0x82, 0x75, 0x50, 0x05, 0x12, 0x1e, + 0x71, 0x4d, 0x7b, 0xcc, 0xa6, 0x90, 0xd9, 0xbf, 0x7d, 0xce, 0x6a, 0xa5, 0xcf, 0xb8, 0x8a, 0x90, + 0x62, 0x56, 0x4c, 0x62, 0x61, 0x36, 0x1b, 0x6a, 0x85, 0x76, 0xd0, 0x0e, 0x64, 0x0c, 0xec, 0xe9, + 0xae, 0x39, 0xa5, 0xd3, 0x62, 0x33, 0x48, 0x2b, 0x61, 0x12, 0x7a, 0x04, 0x59, 0x03, 0x8f, 0xb4, + 0x99, 0x45, 0xd4, 0x17, 0x9e, 0x63, 0xb3, 0x39, 0xa4, 0x6b, 0x6f, 0xbd, 0x7e, 0x55, 0x2a, 0x60, + 0x5b, 0x77, 0x0c, 0xd3, 0x1e, 0xef, 0x51, 0x46, 0x45, 0xd1, 0x5e, 0x1e, 0xf2, 0xb0, 0x29, 0x49, + 0xa1, 0x81, 0x6e, 0x43, 0xc2, 0xc3, 0xba, 0x8b, 0x49, 0x21, 0xc9, 0x7c, 0x23, 0x7a, 0x74, 0xf0, + 0xf8, 0xcc, 0xf4, 0x88, 0x57, 0x48, 0xed, 0x48, 0xbb, 0xf9, 0xd5, 0xc1, 0x37, 0x19, 0x57, 0x11, + 0x52, 0xe8, 0x1b, 0x90, 0xb4, 0xe9, 0x7a, 0x37, 0xf5, 0x42, 0x9a, 0xcd, 0xf6, 0xce, 0x79, 0x85, + 0x0e, 0x67, 0x2b, 0xbe, 0x5c, 0xf1, 0x27, 0x12, 0x24, 0xb8, 0x0b, 0xd0, 0x7d, 0xc8, 0xea, 0x8e, + 0x4d, 0xb0, 0x4d, 0x54, 0xea, 0x3b, 0xe6, 0xb0, 0xb4, 0x92, 0x11, 0xb4, 0xc1, 0x7c, 0x8a, 0xe9, + 0x40, 0x47, 0x8e, 0x3b, 0xd1, 0x88, 0x70, 0x8f, 0xe8, 0xa1, 0x77, 0x40, 0xf6, 0x55, 0xfd, 0xd9, + 0xb2, 0xa9, 0xa4, 0x95, 0x75, 0x41, 0x6f, 0x0a, 0x32, 0xba, 0x07, 0x30, 0xd1, 0xce, 0x54, 0x0b, + 0xdb, 0x63, 0x72, 0xcc, 0xdc, 0x94, 0x53, 0xd2, 0x13, 0xed, 0xac, 0xcd, 0x08, 0x4f, 0x63, 0x29, + 0x49, 0x8e, 0x3c, 0x8d, 0xa5, 0x22, 0x72, 0xf4, 0x69, 0x2c, 0x15, 0x97, 0x13, 0xc5, 0x3f, 0x86, + 0xa4, 0x18, 0x35, 0x2a, 0x41, 0xe6, 0x58, 0xf3, 0xd4, 0x89, 0x69, 0x9b, 0x93, 0xd9, 0x84, 0xad, + 0xe0, 0x94, 0x02, 0xc7, 0x9a, 0x77, 0xc8, 0x29, 0xa8, 0x00, 0x49, 0x9f, 0x49, 0x97, 0xb2, 0xa4, + 0xf8, 0xdd, 0x40, 0x55, 0x3b, 0x63, 0xdc, 0xe8, 0x42, 0x95, 0x53, 0x98, 0xaa, 0x60, 0xc6, 0x84, + 0x2a, 0xef, 0x96, 0x6b, 0x90, 0xe0, 0x7e, 0x46, 0x19, 0x48, 0xb6, 0x3a, 0xdf, 0xa9, 0xb6, 0x5b, + 0x0d, 0x79, 0x0d, 0xa5, 0x20, 0x76, 0x78, 0xd4, 0x1f, 0xc8, 0x12, 0x4a, 0x42, 0xf4, 0xb0, 0xfa, + 0x5c, 0x8e, 0xa0, 0x2c, 0xa4, 0x5a, 0x87, 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, + 0xe9, 0x39, 0x2e, 0x0b, 0xb8, 0x3d, 0x9b, 0x0c, 0xb1, 0x2b, 0x80, 0x43, 0xf4, 0xe8, 0x96, 0xf3, + 0x11, 0x4a, 0xec, 0xc5, 0xa0, 0x4f, 0x75, 0xa6, 0xb3, 0xa1, 0x65, 0xea, 0x62, 0xf0, 0xa2, 0x57, + 0xfe, 0xdf, 0x3c, 0xe4, 0xeb, 0x8e, 0x65, 0xf1, 0xdd, 0xdd, 0x9f, 0x62, 0x1d, 0x95, 0x21, 0x66, + 0x6b, 0x13, 0x8e, 0x29, 0xe9, 0x5a, 0xfe, 0xf5, 0xab, 0x12, 0x2c, 0x24, 0x14, 0xc6, 0x43, 0x0d, + 0xd8, 0x78, 0xe9, 0x9a, 0x04, 0xab, 0x9e, 0x7e, 0x8c, 0x27, 0x1a, 0x5f, 0xb5, 0xa9, 0x6b, 0xac, + 0xda, 0x0c, 0x53, 0xeb, 0x33, 0x2d, 0x54, 0x03, 0xd9, 0xc5, 0x9a, 0xb1, 0x64, 0x24, 0x73, 0x0d, + 0x23, 0x40, 0xb5, 0x84, 0x0d, 0x19, 0xa2, 0x14, 0x16, 0xa2, 0x6c, 0xdb, 0xd2, 0x26, 0xba, 0x0b, + 0xa9, 0xd9, 0xcc, 0x34, 0x54, 0x0a, 0x53, 0x7c, 0xa1, 0x25, 0x69, 0xbf, 0x47, 0x5c, 0xba, 0xd2, + 0x16, 0xa8, 0xc3, 0x70, 0xca, 0x2b, 0xc4, 0x99, 0xe6, 0x7a, 0x40, 0x3f, 0x60, 0x64, 0xf4, 0x11, + 0x64, 0xa6, 0x01, 0xea, 0x79, 0x85, 0xc4, 0x4e, 0x74, 0x37, 0xb3, 0x2f, 0xf3, 0x1d, 0xb1, 0x80, + 0x43, 0x01, 0x2b, 0x61, 0x51, 0xea, 0x1b, 0x4d, 0x3f, 0x51, 0x09, 0x9e, 0x4c, 0x2d, 0x8d, 0x60, + 0x3e, 0xad, 0xe4, 0x75, 0x7c, 0xa3, 0xe9, 0x27, 0x03, 0xa1, 0x85, 0x1a, 0x80, 0x16, 0x43, 0xf5, + 0x6d, 0x89, 0x8d, 0x79, 0xab, 0x12, 0x64, 0xa2, 0xa7, 0xce, 0xcc, 0xb5, 0x35, 0x8b, 0x06, 0x4e, + 0xd9, 0x08, 0x14, 0x02, 0x2b, 0xdf, 0x06, 0x30, 0xb0, 0x6b, 0x9e, 0x6a, 0x0c, 0x79, 0xb2, 0x4c, + 0xbb, 0xc4, 0x27, 0xb1, 0x1c, 0xf5, 0x4a, 0x23, 0x10, 0x53, 0x42, 0x2a, 0xc5, 0x7f, 0x03, 0x80, + 0x05, 0x0b, 0x0d, 0x20, 0xaf, 0x3b, 0xb6, 0x8d, 0x75, 0xe2, 0xb8, 0x7c, 0x9f, 0x4b, 0x0c, 0x5b, + 0xde, 0xbf, 0xc2, 0x66, 0xa5, 0xee, 0x6b, 0x51, 0x24, 0x50, 0x72, 0x7a, 0xb8, 0x8b, 0x1e, 0x00, + 0xc5, 0x89, 0x91, 0x39, 0xe6, 0xbe, 0x8a, 0x5c, 0xc3, 0x57, 0x09, 0xae, 0x80, 0xba, 0x00, 0xc4, + 0xd5, 0x6c, 0x8f, 0x42, 0x89, 0xc7, 0x56, 0x41, 0x66, 0xff, 0x9d, 0xab, 0x06, 0x33, 0xf0, 0x35, + 0x44, 0xf8, 0x42, 0x26, 0xd0, 0x27, 0xb0, 0xe1, 0x1d, 0xcf, 0x46, 0x23, 0x0b, 0xd3, 0x74, 0xa3, + 0xf2, 0xa4, 0x10, 0xdb, 0x89, 0xee, 0xe6, 0xf7, 0xdf, 0xbb, 0xca, 0x6e, 0x9f, 0x2b, 0xb2, 0x29, + 0xae, 0x0b, 0x2b, 0xcf, 0xf0, 0x7c, 0xc0, 0x72, 0xc9, 0x43, 0xc8, 0x7b, 0xc7, 0x9a, 0x6b, 0x2c, + 0x82, 0x19, 0x67, 0xe1, 0xd8, 0xac, 0xf8, 0x85, 0x46, 0xa5, 0x4f, 0xf9, 0x2c, 0x94, 0x39, 0x26, + 0x1a, 0x84, 0xb1, 0x05, 0xb7, 0x5c, 0xac, 0x3b, 0xa7, 0xd8, 0x9d, 0xab, 0x96, 0x33, 0x5e, 0x98, + 0x48, 0x5c, 0xb6, 0x1e, 0x36, 0x7d, 0x9d, 0xb6, 0x33, 0x0e, 0x4c, 0x7d, 0x0b, 0x72, 0x36, 0xc7, + 0x12, 0x75, 0xea, 0xb8, 0xc4, 0x2b, 0x24, 0x99, 0xcf, 0x36, 0xf8, 0xdc, 0x42, 0x30, 0xa3, 0x64, + 0xed, 0x45, 0xc7, 0x2b, 0xfe, 0x24, 0x0e, 0xe9, 0xc0, 0x6f, 0xe8, 0xfe, 0x12, 0x46, 0xe4, 0x5e, + 0xbf, 0x2a, 0x2d, 0x98, 0x02, 0x22, 0x1e, 0x02, 0xe8, 0x81, 0xa7, 0x58, 0x4c, 0x33, 0xfb, 0x5b, + 0x17, 0x79, 0xd0, 0x0f, 0xc2, 0x42, 0x1a, 0xb5, 0xc3, 0x8b, 0xdf, 0xc3, 0x16, 0x5b, 0x2a, 0x22, + 0x07, 0xdf, 0x59, 0x4c, 0xb6, 0xad, 0x0d, 0xb1, 0xd5, 0x17, 0x6c, 0x61, 0x66, 0xb1, 0x09, 0x7c, + 0x06, 0xc7, 0x45, 0xd3, 0x71, 0x4d, 0xc2, 0xcb, 0x8c, 0x9c, 0x12, 0xf4, 0xd1, 0xd7, 0x01, 0x31, + 0x08, 0x32, 0xb0, 0xa5, 0xcd, 0x55, 0x0f, 0xeb, 0x8e, 0xcd, 0x30, 0x81, 0x4a, 0x31, 0x70, 0x6a, + 0x50, 0x46, 0x9f, 0xd3, 0x69, 0x1e, 0x08, 0x2d, 0x0e, 0x06, 0x0a, 0x69, 0x05, 0x16, 0x91, 0x46, + 0x47, 0x50, 0xf4, 0x05, 0x2c, 0x6d, 0x32, 0x34, 0x34, 0x35, 0xbc, 0xb0, 0xaf, 0x03, 0x02, 0x9b, + 0x42, 0xbf, 0xcd, 0xd4, 0xeb, 0x7c, 0x95, 0x1f, 0x00, 0xba, 0xc0, 0xdc, 0x75, 0xf0, 0x36, 0x6b, + 0x85, 0xed, 0x7c, 0x05, 0xd2, 0x6c, 0xb6, 0x8e, 0x6d, 0xcd, 0x19, 0x96, 0xa4, 0x94, 0x14, 0x25, + 0x74, 0x6d, 0x6b, 0x8e, 0x2a, 0xb0, 0xf9, 0x82, 0xaf, 0x1e, 0x95, 0xa3, 0xf2, 0x6c, 0x34, 0x32, + 0xcf, 0x0a, 0xc0, 0x20, 0x74, 0x43, 0xb0, 0x14, 0x8a, 0xbc, 0x8c, 0x81, 0x1e, 0x00, 0xd8, 0x0e, + 0x51, 0x87, 0x78, 0xe4, 0xb8, 0x98, 0xe1, 0x76, 0x66, 0xbf, 0x58, 0xe1, 0xd5, 0x6f, 0xc5, 0xaf, + 0x7e, 0x2b, 0x03, 0xbf, 0xfa, 0x55, 0xd2, 0xb6, 0x43, 0x6a, 0x4c, 0x18, 0xfd, 0x3e, 0xd0, 0x8e, + 0xaa, 0x8d, 0x08, 0x76, 0x05, 0x2a, 0x5d, 0xa6, 0x99, 0xb2, 0x1d, 0x52, 0xa5, 0xb2, 0x34, 0x94, + 0x43, 0x4d, 0x3f, 0x19, 0x99, 0x96, 0x55, 0xc8, 0xf1, 0x50, 0xfa, 0xfd, 0xf2, 0xf7, 0x21, 0xb7, + 0x84, 0x32, 0xa8, 0x08, 0xb7, 0x45, 0xc2, 0x55, 0xeb, 0xdd, 0x4e, 0xa7, 0x59, 0x1f, 0x74, 0x15, + 0x75, 0xf0, 0xbc, 0xd7, 0x94, 0xd7, 0x68, 0x7a, 0xed, 0xff, 0x61, 0xbb, 0x35, 0x68, 0xca, 0x12, + 0xca, 0x03, 0x50, 0x6a, 0xbf, 0xae, 0xb4, 0x7a, 0x03, 0x39, 0x82, 0xd2, 0x10, 0x6f, 0x1d, 0x56, + 0x9f, 0x34, 0xe5, 0x28, 0x6d, 0xb6, 0xbb, 0xf5, 0x6a, 0x5b, 0x8e, 0x95, 0x0f, 0x21, 0x13, 0xda, + 0xdf, 0xa8, 0x00, 0x5b, 0xbe, 0xf1, 0xfe, 0xc7, 0x47, 0x07, 0x07, 0xed, 0xa6, 0x6f, 0x3a, 0x03, + 0xc9, 0x5a, 0xb7, 0xdb, 0x6e, 0x56, 0x3b, 0xb2, 0xc4, 0x93, 0xfe, 0xa0, 0xf9, 0xa4, 0xa9, 0xc8, + 0x11, 0xf6, 0xd1, 0x81, 0xd2, 0xea, 0x3c, 0x91, 0xa3, 0xe5, 0x1f, 0x45, 0x20, 0xcf, 0x52, 0x0d, + 0x5f, 0xa6, 0x74, 0xd5, 0x23, 0x88, 0x9d, 0xe0, 0xb9, 0x5f, 0x82, 0xb2, 0x36, 0xcd, 0xdb, 0xa7, + 0x9a, 0x35, 0xc3, 0x5e, 0x21, 0xc2, 0xa8, 0xa2, 0x47, 0x1d, 0x61, 0x38, 0xfa, 0x6c, 0x82, 0x6d, + 0x22, 0x4a, 0xad, 0xa0, 0x8f, 0x4e, 0xe1, 0x16, 0xcb, 0x6d, 0xe1, 0xc5, 0xa2, 0x4e, 0xb4, 0x29, + 0x83, 0xb1, 0xcc, 0xfe, 0xd7, 0xf9, 0x26, 0x5c, 0xfe, 0x38, 0xef, 0xf2, 0x65, 0xf2, 0xd4, 0x73, + 0xec, 0x43, 0x6d, 0xda, 0xb4, 0x89, 0x3b, 0xaf, 0xbd, 0xf5, 0xe9, 0x17, 0x97, 0xa5, 0xac, 0xd1, + 0x42, 0xad, 0xd8, 0x84, 0x3b, 0x6f, 0xb0, 0xe2, 0x67, 0x69, 0x71, 0x6a, 0xa0, 0x59, 0x7a, 0x0b, + 0xe2, 0x6c, 0x2a, 0xfe, 0xa9, 0x81, 0x75, 0x1e, 0x46, 0x3e, 0x92, 0xca, 0xff, 0x17, 0x87, 0x4c, + 0x5d, 0x9b, 0x92, 0x99, 0x8b, 0x59, 0x3d, 0x52, 0x5a, 0xc2, 0x9a, 0xcc, 0xeb, 0x57, 0xa5, 0xa4, + 0x60, 0x0b, 0xa4, 0x39, 0x58, 0x49, 0x4a, 0x11, 0x96, 0x94, 0xfc, 0x44, 0xb7, 0xb0, 0x75, 0xa3, + 0x34, 0x14, 0xbd, 0x41, 0x1a, 0xfa, 0x26, 0xa4, 0x86, 0xa6, 0x4d, 0x45, 0x3c, 0xe1, 0xe5, 0xbb, + 0xab, 0x1f, 0xaf, 0x71, 0x09, 0x25, 0x10, 0xa5, 0xf5, 0x88, 0x69, 0x13, 0xec, 0x9e, 0x6a, 0xd6, + 0x39, 0xec, 0x59, 0xf7, 0xe9, 0x3e, 0xf4, 0xac, 0xa6, 0x8f, 0xc4, 0xaf, 0x9f, 0x3e, 0x92, 0xbf, + 0x7e, 0xfa, 0x48, 0x5d, 0x2f, 0x7d, 0xfc, 0x8f, 0x04, 0x49, 0x31, 0x7f, 0xf4, 0x14, 0xb6, 0x5c, + 0xec, 0x39, 0x33, 0x57, 0xc7, 0x4b, 0x78, 0x26, 0x5d, 0xc3, 0xe1, 0x79, 0x5f, 0x53, 0x20, 0xda, + 0x57, 0x21, 0x17, 0xd8, 0x9a, 0x6a, 0xe4, 0x58, 0x6c, 0x93, 0xac, 0x4f, 0xec, 0x69, 0xe4, 0xf8, + 0x5c, 0x2a, 0x8a, 0xde, 0x28, 0x15, 0x85, 0x11, 0x27, 0xb6, 0x8c, 0x38, 0x14, 0x4e, 0x3d, 0x42, + 0x4b, 0x3c, 0xff, 0x60, 0x9a, 0x56, 0x52, 0x8c, 0xf0, 0x0c, 0xcf, 0xcb, 0xfb, 0xe7, 0xe1, 0x68, + 0xa9, 0xfe, 0x0f, 0x30, 0x26, 0xb9, 0xc0, 0x98, 0x54, 0xf9, 0xdf, 0x01, 0x36, 0x0f, 0x35, 0x82, + 0x5d, 0x53, 0xb3, 0xcc, 0x1f, 0x68, 0x41, 0x49, 0xfe, 0xf6, 0xd2, 0x16, 0xd8, 0x7c, 0xfd, 0xaa, + 0xb4, 0x7e, 0x4e, 0x4c, 0x6c, 0x85, 0xce, 0x1b, 0xb6, 0xc2, 0xdb, 0x7c, 0xb6, 0x17, 0xd8, 0xfe, + 0xad, 0x6d, 0x89, 0xc7, 0x2b, 0x5b, 0xe2, 0xfe, 0x9b, 0x07, 0xb1, 0xba, 0x35, 0x7e, 0xc7, 0xcb, + 0xa5, 0x7f, 0x4e, 0xfc, 0x8e, 0xae, 0xf7, 0x8b, 0x4b, 0xaf, 0xe4, 0x6f, 0xa0, 0xf4, 0x4a, 0x9f, + 0x2b, 0xbd, 0xea, 0xb0, 0xce, 0xd3, 0x94, 0xe7, 0x27, 0x21, 0xb6, 0xc1, 0x82, 0xa1, 0x2e, 0x27, + 0x28, 0xf1, 0x8d, 0xfc, 0x68, 0x39, 0x67, 0x7e, 0x15, 0x72, 0x06, 0xb6, 0x88, 0xa6, 0xce, 0xa6, + 0x86, 0x46, 0xb0, 0xe7, 0xdf, 0x0f, 0x31, 0xe2, 0x11, 0xa7, 0xa1, 0x21, 0x20, 0x03, 0x4f, 0x5d, + 0xac, 0x6b, 0x04, 0x1b, 0xaa, 0x28, 0xb0, 0xc4, 0x62, 0xf8, 0xf0, 0xca, 0x45, 0x59, 0x69, 0x04, + 0xba, 0xa2, 0x00, 0x50, 0x36, 0x8c, 0xf3, 0xa4, 0x37, 0x55, 0x4f, 0xa9, 0xeb, 0x55, 0x4f, 0xf0, + 0xa5, 0xab, 0xa7, 0xcc, 0x97, 0xac, 0x9e, 0xb2, 0x97, 0x61, 0x59, 0x6e, 0x19, 0xcb, 0x8a, 0x7f, + 0x22, 0xc1, 0xc6, 0x8a, 0x17, 0xd0, 0x3d, 0x80, 0xb1, 0xeb, 0xcc, 0xa6, 0xea, 0x02, 0x9b, 0x94, + 0x34, 0xa3, 0x74, 0x28, 0x16, 0xfd, 0x46, 0x8b, 0xf8, 0xf2, 0xe3, 0xeb, 0xc1, 0x69, 0x6a, 0x01, + 0xa7, 0x69, 0xda, 0x6c, 0x34, 0x9f, 0x55, 0x0f, 0x64, 0x28, 0xff, 0x77, 0x1a, 0x12, 0xdd, 0xea, + 0x8c, 0x1c, 0xef, 0x8b, 0x6b, 0x92, 0x53, 0xd3, 0xc0, 0xfe, 0x35, 0x66, 0xd0, 0x47, 0xef, 0xc2, + 0x86, 0x36, 0x23, 0xc7, 0xea, 0xcc, 0xb5, 0x16, 0xa8, 0xc1, 0x2b, 0x94, 0x75, 0xca, 0x38, 0x72, + 0xad, 0x00, 0x1a, 0x1e, 0xc0, 0x5d, 0x4d, 0xd7, 0xb1, 0xe7, 0xa9, 0xc4, 0x39, 0xc1, 0xf6, 0xb2, + 0x0e, 0xaf, 0xc9, 0x6e, 0x73, 0x81, 0x01, 0xe5, 0x87, 0x55, 0x2b, 0xb0, 0xb9, 0xa4, 0x3a, 0xc1, + 0xe4, 0xd8, 0x31, 0xf8, 0xdd, 0x87, 0xb2, 0x11, 0x52, 0x3a, 0x64, 0x0c, 0x36, 0xac, 0xb0, 0xfc, + 0xd0, 0x31, 0xe6, 0xe2, 0x6e, 0x63, 0x3d, 0x24, 0x5d, 0x73, 0x8c, 0x39, 0xfa, 0x33, 0x09, 0xee, + 0x2d, 0x09, 0x1f, 0x63, 0xcd, 0xc0, 0xae, 0xb7, 0x28, 0x03, 0xe3, 0x0c, 0xc2, 0xc4, 0x69, 0x96, + 0x3b, 0xa5, 0x52, 0x5d, 0x58, 0xf9, 0x98, 0xcb, 0xdf, 0xa0, 0x0a, 0x44, 0xda, 0x8a, 0x36, 0xfa, + 0x54, 0x82, 0xed, 0xa5, 0x61, 0xb8, 0xd8, 0x9b, 0x3a, 0xb6, 0x87, 0x17, 0xe3, 0x48, 0x84, 0xcb, + 0xd1, 0xd5, 0x71, 0x28, 0x42, 0xe1, 0x06, 0x03, 0x09, 0xfb, 0xdb, 0x57, 0x3f, 0xd4, 0xa6, 0xe8, + 0x11, 0x14, 0x5d, 0x3c, 0x72, 0xb1, 0x77, 0x7c, 0x51, 0xac, 0xf8, 0x5d, 0xe3, 0x1d, 0x21, 0xb1, + 0x12, 0xac, 0x0f, 0x28, 0x7c, 0x87, 0x95, 0x45, 0xb4, 0xb2, 0x4c, 0x0d, 0x85, 0xd5, 0x44, 0xb8, + 0xd8, 0xa1, 0x32, 0xac, 0xc1, 0xe2, 0xc5, 0xa1, 0x40, 0x0e, 0xcb, 0xb3, 0x80, 0xfd, 0xb9, 0x04, + 0xdb, 0xcb, 0xe2, 0x2b, 0x11, 0x4b, 0x5f, 0xe0, 0x29, 0x25, 0x64, 0xe7, 0xe6, 0x21, 0xdb, 0x74, + 0x57, 0xd5, 0xd1, 0x5f, 0x4a, 0x50, 0x5a, 0x1e, 0xc9, 0x6a, 0xd0, 0x80, 0x0d, 0xe5, 0xfd, 0x37, + 0x0e, 0xe5, 0x4b, 0x44, 0x6d, 0xc9, 0xf3, 0xa1, 0xb0, 0x15, 0xdb, 0xb0, 0x7d, 0xf9, 0xba, 0xbc, + 0xc9, 0xb9, 0xa2, 0x78, 0x08, 0xa5, 0x2b, 0x56, 0xd7, 0x4d, 0xcd, 0x5d, 0x11, 0x82, 0x1b, 0x99, + 0xeb, 0xc0, 0xce, 0x55, 0x6e, 0xbc, 0xd1, 0x29, 0xea, 0xef, 0xa2, 0x90, 0x1a, 0x60, 0x8f, 0xb0, + 0xfa, 0x11, 0x85, 0xeb, 0x47, 0x51, 0x2a, 0xee, 0x41, 0xdc, 0x23, 0x78, 0xca, 0x0f, 0x96, 0xb4, + 0xae, 0x62, 0x11, 0xf5, 0x55, 0x2a, 0x7d, 0x82, 0xa7, 0x02, 0x8d, 0xb9, 0x5c, 0xf1, 0x97, 0x11, + 0x88, 0x51, 0x2a, 0xfa, 0x26, 0x4d, 0x15, 0x78, 0x1a, 0xbe, 0xff, 0x2b, 0x5c, 0xa0, 0x5d, 0x61, + 0x05, 0x65, 0x8a, 0x8a, 0x32, 0xc0, 0xbe, 0x07, 0xc0, 0xd4, 0x4c, 0xdb, 0xc0, 0x67, 0xe2, 0x7d, + 0x8b, 0x19, 0x6a, 0x51, 0xc2, 0xf9, 0x57, 0x92, 0xe8, 0xea, 0x2b, 0x89, 0x6f, 0xc0, 0xd3, 0x9d, + 0xa9, 0xff, 0xc4, 0xc2, 0x0c, 0xf4, 0x29, 0x01, 0x55, 0x96, 0xaa, 0x9e, 0xf8, 0x85, 0xb7, 0xd7, + 0xe1, 0x4a, 0xe7, 0x01, 0xe4, 0x0c, 0x47, 0x17, 0xbb, 0xec, 0x14, 0xeb, 0xfc, 0x3a, 0xe7, 0x8a, + 0x7a, 0x2c, 0x46, 0x55, 0xd0, 0x63, 0x80, 0x20, 0x43, 0x79, 0xd7, 0x2b, 0x8e, 0x42, 0x0a, 0xe5, + 0x6d, 0x88, 0x31, 0x8f, 0x00, 0x24, 0x5a, 0x9d, 0x27, 0xcd, 0xfe, 0x80, 0x5f, 0x48, 0x7c, 0xa7, + 0xa9, 0xb4, 0x0e, 0x9e, 0xcb, 0x52, 0xf9, 0x47, 0x12, 0xa4, 0x15, 0xcd, 0x1e, 0xf3, 0xf3, 0xef, + 0x57, 0x20, 0x7d, 0x82, 0xe7, 0x2a, 0x7f, 0x2a, 0xa4, 0x6e, 0x4b, 0x2a, 0xa9, 0x13, 0x3c, 0xaf, + 0xb1, 0xd7, 0xc2, 0x3b, 0x90, 0xa4, 0x4c, 0x6c, 0x1b, 0xcc, 0x63, 0x49, 0x25, 0x71, 0x82, 0xe7, + 0x4d, 0xdb, 0x40, 0x65, 0xc8, 0xb9, 0x2a, 0x7b, 0xd3, 0x13, 0x9a, 0x31, 0xc6, 0xce, 0xb8, 0xec, + 0xb9, 0x8f, 0x2b, 0x6f, 0x43, 0xc6, 0x97, 0xa1, 0x06, 0xe2, 0x4c, 0x22, 0xcd, 0x25, 0x9a, 0xb6, + 0xf1, 0x50, 0xfe, 0xf1, 0x67, 0xa5, 0xb5, 0x73, 0x8f, 0x8f, 0xf9, 0x20, 0x0b, 0xf7, 0x69, 0x75, + 0x80, 0x1e, 0x41, 0x96, 0x97, 0x5e, 0xc6, 0xf5, 0x8b, 0xda, 0xa4, 0xd0, 0x40, 0x25, 0xc8, 0x4c, + 0xb0, 0x3b, 0x66, 0xa5, 0xac, 0x7e, 0xcc, 0x26, 0x97, 0x52, 0x80, 0x91, 0x7a, 0x94, 0x52, 0xfe, + 0x5b, 0x09, 0xd2, 0x7d, 0xec, 0xf6, 0x1c, 0xcb, 0xd4, 0xd9, 0x65, 0x9d, 0x47, 0x5c, 0x95, 0xb8, + 0x33, 0x9b, 0x16, 0x22, 0xa2, 0x02, 0xe2, 0x8f, 0x20, 0xb2, 0x47, 0xdc, 0x81, 0x60, 0xf0, 0x6a, + 0xe7, 0x01, 0xdc, 0xb5, 0xb1, 0x47, 0x07, 0xe6, 0x0c, 0x5f, 0x9c, 0x57, 0xe2, 0xcb, 0xef, 0x36, + 0x17, 0xe8, 0x0e, 0x5f, 0x2c, 0xab, 0x7e, 0x00, 0x5b, 0x9a, 0xeb, 0x6a, 0xf3, 0xf3, 0x5a, 0x51, + 0xa6, 0x85, 0x18, 0x6f, 0x49, 0xa3, 0xfc, 0x37, 0x11, 0x80, 0xe6, 0x19, 0x71, 0x35, 0x9d, 0x54, + 0x7b, 0xad, 0xe2, 0x3f, 0x48, 0x90, 0x10, 0x15, 0x7b, 0xf8, 0x39, 0x42, 0x5a, 0x7e, 0x8e, 0x78, + 0x0c, 0x99, 0xf0, 0xd3, 0xc7, 0x75, 0xee, 0xbd, 0x81, 0x2b, 0x50, 0x58, 0xa0, 0xfb, 0x81, 0x17, + 0xd0, 0x53, 0xe2, 0x7a, 0xe2, 0x05, 0x24, 0xcd, 0x28, 0x3d, 0xe2, 0xae, 0xbc, 0x60, 0xc4, 0xae, + 0xfd, 0x82, 0x51, 0x7e, 0x0e, 0xb1, 0xba, 0x63, 0x9c, 0x2b, 0xb1, 0x72, 0x90, 0xae, 0x77, 0x3b, + 0x07, 0xad, 0x27, 0x47, 0x4a, 0x93, 0x5f, 0x6c, 0x35, 0xbf, 0x3b, 0x50, 0xaa, 0xf5, 0x81, 0x1c, + 0x41, 0x08, 0xf2, 0xa2, 0xd3, 0x6c, 0xa8, 0x47, 0x47, 0xad, 0x86, 0x1c, 0x45, 0x5b, 0x20, 0x2f, + 0x68, 0x07, 0xad, 0x66, 0xbb, 0xd1, 0x97, 0x63, 0xe5, 0x5f, 0x4a, 0x90, 0xaa, 0xcd, 0x4c, 0xcb, + 0xa0, 0x5e, 0xfa, 0x8f, 0x25, 0x2f, 0x0d, 0x29, 0x59, 0x35, 0x0d, 0xdf, 0x4b, 0xac, 0xdf, 0x32, + 0x16, 0x2c, 0x63, 0x28, 0x60, 0x8e, 0xb3, 0x1a, 0x43, 0xf6, 0xf4, 0xc9, 0x8e, 0x3d, 0x02, 0x2e, + 0x44, 0x0f, 0xed, 0x43, 0x46, 0x9c, 0x91, 0x18, 0x46, 0xc5, 0x18, 0x46, 0x6d, 0xf8, 0x27, 0xa0, + 0xe0, 0x45, 0x52, 0x01, 0x2e, 0xc5, 0x36, 0xe3, 0x7b, 0xb0, 0xb1, 0x38, 0x3a, 0x8b, 0xb3, 0x9c, + 0x38, 0xd4, 0xcb, 0x01, 0x43, 0x1c, 0xf8, 0xd0, 0x7d, 0xc8, 0x0a, 0x87, 0xa9, 0xae, 0xe3, 0x10, + 0xfe, 0x60, 0x1b, 0x38, 0x51, 0x71, 0x1c, 0x52, 0xde, 0x84, 0x0d, 0x05, 0x7b, 0x98, 0xb0, 0x6d, + 0xa2, 0xe0, 0x3f, 0x9a, 0x61, 0x8f, 0x94, 0xb7, 0x00, 0x85, 0x89, 0x1c, 0xe3, 0xcb, 0x8f, 0x01, + 0x55, 0x8d, 0x53, 0xcd, 0xd6, 0x31, 0xad, 0xda, 0x85, 0x2c, 0x7a, 0x1b, 0xd6, 0x35, 0x4e, 0x0d, + 0xee, 0x86, 0xa8, 0x67, 0x62, 0x4a, 0x5e, 0x90, 0xc5, 0xd5, 0x50, 0xf9, 0x16, 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, 0x9d, + 0xf2, 0x5f, 0x49, 0x90, 0x6b, 0xd9, 0x63, 0xec, 0x11, 0x7f, 0x2c, 0xcb, 0xd8, 0x2a, 0x5d, 0x89, + 0xad, 0xe1, 0x70, 0x46, 0x96, 0xc3, 0xb9, 0x02, 0xbb, 0xd1, 0xeb, 0xc2, 0x6e, 0xf9, 0x5f, 0x22, + 0x90, 0xf7, 0xc7, 0xc5, 0x27, 0x89, 0xfe, 0x5e, 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, 0xbe, 0xff, 0xa7, 0x5f, 0x94, 0xde, 0xbb, 0xe8, 0x8f, 0x12, 0xe7, 0xfe, 0x28, 0xe2, + 0xeb, 0x7f, 0xfa, 0x45, 0xe9, 0xdd, 0xeb, 0x88, 0x77, 0x47, 0x23, 0x0f, 0x93, 0xe0, 0x0c, 0xb8, + 0xf8, 0x2c, 0xfa, 0x03, 0xc8, 0xfa, 0x63, 0xc7, 0x44, 0x37, 0xc4, 0x23, 0x49, 0xe8, 0x7a, 0x82, + 0x17, 0x15, 0x95, 0x26, 0xd1, 0x0d, 0x7f, 0x9f, 0x0a, 0x05, 0x4a, 0x2a, 0x36, 0xe0, 0xf6, 0xc5, + 0x73, 0xb9, 0xaa, 0x52, 0x88, 0x86, 0x2b, 0x85, 0x9f, 0x45, 0xe0, 0xce, 0x40, 0xf3, 0x4e, 0xfc, + 0xcb, 0x0c, 0xd7, 0x39, 0x9b, 0xfb, 0x71, 0xff, 0x16, 0xc4, 0x9c, 0x29, 0xe6, 0x11, 0xcf, 0xec, + 0x97, 0x45, 0x96, 0xbf, 0x58, 0xb8, 0xd2, 0x9d, 0x62, 0x5b, 0x61, 0xf2, 0x6c, 0xad, 0x69, 0x44, + 0x63, 0x1f, 0xcb, 0x2a, 0xac, 0x5d, 0xfc, 0x47, 0x09, 0x62, 0x54, 0x04, 0xed, 0x42, 0x82, 0x57, + 0xb8, 0xc2, 0xac, 0x7c, 0x7e, 0xc2, 0x8a, 0xe0, 0xa3, 0x36, 0xa4, 0xf8, 0x25, 0x90, 0xbf, 0x8c, + 0x6a, 0xdf, 0x78, 0xfd, 0xaa, 0xf4, 0xfe, 0x45, 0x5e, 0x5f, 0xf9, 0xb7, 0x0e, 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, 0x17, 0x51, 0x28, 0xac, 0xce, 0x5e, 0x2c, 0xc5, + 0x1e, 0xe4, 0xe8, 0xdc, 0x83, 0x9a, 0x59, 0xcc, 0xee, 0xbd, 0x37, 0x39, 0x4d, 0xac, 0x46, 0xe6, + 0x35, 0xd1, 0x51, 0xb2, 0x4e, 0xa8, 0x77, 0xa1, 0x17, 0x5f, 0x42, 0x36, 0xac, 0x81, 0x1e, 0x43, + 0x82, 0x1e, 0xd3, 0x67, 0x9e, 0xa8, 0xc4, 0xbe, 0x76, 0xc5, 0xe7, 0xfa, 0x4c, 0x58, 0x11, 0x4a, + 0xa1, 0x58, 0x44, 0x2e, 0x8f, 0x45, 0xf9, 0x87, 0xec, 0xbf, 0x1e, 0x4c, 0x29, 0x01, 0x91, 0xee, + 0x33, 0x79, 0x0d, 0x6d, 0xc2, 0x7a, 0xff, 0xe3, 0xaa, 0xd2, 0x50, 0x3b, 0xdd, 0x81, 0x7a, 0xd0, + 0x3d, 0xea, 0x34, 0x64, 0x89, 0xe2, 0x7e, 0xa7, 0xab, 0x72, 0x7a, 0x4f, 0x69, 0x1d, 0x56, 0x95, + 0xe7, 0x72, 0x04, 0xdd, 0x82, 0x0d, 0x2a, 0xb4, 0x4c, 0x8e, 0xd2, 0xc4, 0xd1, 0xea, 0x0c, 0x9a, + 0x4a, 0xa7, 0xda, 0x56, 0x9b, 0x8a, 0xd2, 0x55, 0xe4, 0x18, 0xda, 0x80, 0x1c, 0x17, 0xeb, 0x0f, + 0xba, 0xbd, 0x5e, 0xb3, 0x21, 0xc7, 0xd1, 0x2d, 0x90, 0x7b, 0x5d, 0x65, 0xc0, 0xbe, 0x53, 0x6d, + 0xb7, 0xbb, 0x9f, 0x34, 0x1b, 0xf2, 0xaf, 0x92, 0xef, 0x76, 0x20, 0x13, 0x42, 0x73, 0x9a, 0x92, + 0xea, 0xd5, 0x41, 0xb5, 0xdd, 0x7d, 0x22, 0xaf, 0xa1, 0x75, 0xc8, 0x3c, 0xed, 0x77, 0x3b, 0x6a, + 0xbf, 0xfe, 0x71, 0xf3, 0xb0, 0x2a, 0x4b, 0xec, 0x0f, 0x15, 0x2c, 0x7f, 0xc9, 0x31, 0x3a, 0x9a, + 0x46, 0xb7, 0x7e, 0x74, 0xd8, 0xec, 0x0c, 0xfa, 0xea, 0x41, 0xeb, 0xbb, 0x03, 0x9a, 0xd3, 0xe2, + 0xfb, 0xff, 0x24, 0x41, 0x92, 0x96, 0xb0, 0xa6, 0x3d, 0x46, 0xdf, 0x06, 0x58, 0x20, 0x35, 0x12, + 0x7f, 0x85, 0x59, 0x01, 0xf4, 0x62, 0x61, 0x95, 0x21, 0x02, 0x53, 0x83, 0x4c, 0x08, 0x95, 0x91, + 0x10, 0x5c, 0xc5, 0xf9, 0xe2, 0xdd, 0x0b, 0x38, 0xc2, 0xc6, 0x87, 0x90, 0xe0, 0xd8, 0x85, 0x36, + 0x97, 0x91, 0x8c, 0x6b, 0x6e, 0x5d, 0x04, 0x6f, 0xfb, 0xdf, 0x83, 0x6c, 0x38, 0xf2, 0xe8, 0x29, + 0xc4, 0x79, 0xe3, 0xde, 0xa5, 0xdb, 0xb7, 0xb8, 0x7d, 0xf9, 0xca, 0xd9, 0x95, 0x3e, 0x90, 0x6a, + 0x8f, 0x3e, 0xff, 0xcf, 0xed, 0xb5, 0xcf, 0x7f, 0xb1, 0x2d, 0xfd, 0xfc, 0x17, 0xdb, 0xd2, 0x67, + 0xff, 0xb5, 0x2d, 0x7d, 0xef, 0x9d, 0xb1, 0x49, 0x8e, 0x67, 0xc3, 0x8a, 0xee, 0x4c, 0xf6, 0xb0, + 0x47, 0x66, 0x9a, 0x3b, 0xe7, 0x7f, 0xb8, 0x5b, 0xf9, 0x0b, 0xde, 0x30, 0xc1, 0xfa, 0x1f, 0xfe, + 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xe7, 0xa6, 0x88, 0x39, 0x9e, 0x27, 0x00, 0x00, } func (this *UUIDParts) Equal(that interface{}) bool { diff --git a/go/protocols/flow/flow.proto b/go/protocols/flow/flow.proto index f9f5a0cf3f..07af7e7ad7 100644 --- a/go/protocols/flow/flow.proto +++ b/go/protocols/flow/flow.proto @@ -345,6 +345,7 @@ message MaterializationSpec { INVALID = 0; IMAGE = 8; LOCAL = 9; + DEKAF = 10; } ConnectorType connector_type = 2; // JSON-encoded connector configuration of this materialization. From f141de986269a99878bb4d18332f6e1a00f9689d Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Thu, 19 Sep 2024 12:43:54 -0400 Subject: [PATCH 02/26] crates/activate: Dekaf materializations should not get shards or recovery journals, but they _should_ get ops stats+logs collections --- crates/activate/src/lib.rs | 115 +++++++++++++++++++++++++++---------- 1 file changed, 85 insertions(+), 30 deletions(-) diff --git a/crates/activate/src/lib.rs b/crates/activate/src/lib.rs index e77a74ba4c..fe51c93cdc 100644 --- a/crates/activate/src/lib.rs +++ b/crates/activate/src/lib.rs @@ -1,5 +1,5 @@ use anyhow::Context; -use proto_flow::flow; +use proto_flow::flow::{self, materialization_spec}; use proto_gazette::{ broker::{self, JournalSpec, Label, LabelSelector, LabelSet}, consumer::{self, ShardSpec}, @@ -34,9 +34,12 @@ pub async fn activate_capture( .as_ref() .context("CaptureSpec missing recovery_log_template")?; - Some((shard_template, recovery_template)) + TaskTemplate::UpsertReal { + shard: shard_template, + recovery_journal: recovery_template, + } } else { - None + TaskTemplate::Delete }; let changes = converge_task_changes( @@ -81,14 +84,17 @@ pub async fn activate_collection( .as_ref() .context("CollectionSpec.Derivation missing recovery_log_template")?; - Some((shard_template, recovery_template)) + TaskTemplate::UpsertReal { + shard: shard_template, + recovery_journal: recovery_template, + } } else { - None + TaskTemplate::Delete }; (task_template, Some(partition_template)) } else { - (None, None) + (TaskTemplate::Delete, None) }; let (changes_1, changes_2) = futures::try_join!( @@ -123,20 +129,29 @@ pub async fn activate_materialization( ops_stats_template: Option<&broker::JournalSpec>, initial_splits: usize, ) -> anyhow::Result<()> { - let task_template = if let Some(task_spec) = task_spec { - let shard_template = task_spec - .shard_template - .as_ref() - .context("MaterializationSpec missing shard_template")?; + let task_template = match task_spec { + Some(task_spec) + if task_spec.connector_type == materialization_spec::ConnectorType::Dekaf as i32 => + { + TaskTemplate::UpsertVirtual + } + Some(task_spec) => { + let shard_template = task_spec + .shard_template + .as_ref() + .context("MaterializationSpec missing shard_template")?; - let recovery_template = task_spec - .recovery_log_template - .as_ref() - .context("MaterializationSpec missing recovery_log_template")?; + let recovery_template = task_spec + .recovery_log_template + .as_ref() + .context("MaterializationSpec missing recovery_log_template")?; - Some((shard_template, recovery_template)) - } else { - None + TaskTemplate::UpsertReal { + shard: shard_template, + recovery_journal: recovery_template, + } + } + None => TaskTemplate::Delete, }; let changes = converge_task_changes( @@ -253,15 +268,30 @@ async fn apply_changes( Ok(()) } +/// Describes the desired future state of a task. +/// Virtual tasks get logs and stats journals, +/// but are otherwise purely descriptive and +/// do not get shards and recovery log journals +/// created for them like real tasks do. +#[derive(Clone, Copy, Debug)] +enum TaskTemplate<'a> { + UpsertReal { + shard: &'a ShardSpec, + recovery_journal: &'a JournalSpec, + }, + UpsertVirtual, + Delete, +} + /// Converge a task by listing data-plane ShardSpecs and recovery log /// JournalSpecs, and then applying updates to bring them into alignment /// with the templated task configuration. -async fn converge_task_changes( +async fn converge_task_changes<'a>( journal_client: &gazette::journal::Client, shard_client: &gazette::shard::Client, task_type: ops::TaskType, task_name: &str, - template: Option<(&ShardSpec, &JournalSpec)>, + template: TaskTemplate<'a>, ops_logs_template: Option<&broker::JournalSpec>, ops_stats_template: Option<&broker::JournalSpec>, initial_splits: usize, @@ -295,7 +325,10 @@ async fn converge_task_changes( // If (and only if) the task is being upserted, // then ensure the creation of its ops collection partitions. - if template.is_some() { + if matches!( + template, + TaskTemplate::UpsertVirtual | TaskTemplate::UpsertReal { .. } + ) { changes.extend(ops_logs_change.into_iter()); changes.extend(ops_stats_change.into_iter()); } @@ -401,7 +434,7 @@ fn unpack_journal_listing( /// Determine the consumer shard and broker recovery log changes required to /// converge from current `shards` and `recovery` splits into the desired state. fn task_changes( - template: Option<(&ShardSpec, &JournalSpec)>, + template: TaskTemplate, shards: &[(String, LabelSet, i64)], recovery: &[(String, LabelSet, i64)], initial_splits: usize, @@ -412,7 +445,11 @@ fn task_changes( // If the template is Some and no current shards match its prefix, // then instantiate `initial_splits` new shards to create. - if let Some((shard_template, _)) = template { + if let TaskTemplate::UpsertReal { + shard: shard_template, + .. + } = template + { if !shards .iter() .any(|(id, _, _)| id.starts_with(&shard_template.id)) @@ -442,7 +479,10 @@ fn task_changes( for (id, split, shard_revision) in shards { match template { - Some((shard_template, recovery_template)) if id.starts_with(&shard_template.id) => { + TaskTemplate::UpsertReal { + shard: shard_template, + recovery_journal: recovery_template, + } if id.starts_with(&shard_template.id) => { let mut shard_spec = shard_template.clone(); let mut shard_set = shard_spec.labels.take().unwrap_or_default(); @@ -894,7 +934,10 @@ mod test { let partition_changes = partition_changes(Some(&partition_template), &all_partitions).unwrap(); let task_changes = task_changes( - Some((&shard_template, &recovery_template)), + TaskTemplate::UpsertReal { + shard: shard_template, + recovery_journal: recovery_template, + }, &all_shards, &all_recovery, 4, @@ -910,7 +953,10 @@ mod test { { let partition_changes = partition_changes(Some(&partition_template), &[]).unwrap(); let task_changes = task_changes( - Some((&shard_template, &recovery_template)), + TaskTemplate::UpsertReal { + shard: shard_template, + recovery_journal: recovery_template, + }, &[], &[], 4, @@ -926,7 +972,7 @@ mod test { { let partition_changes = partition_changes(None, &all_partitions).unwrap(); let task_changes = task_changes( - None, + TaskTemplate::Delete, &all_shards, &all_recovery, 4, @@ -961,7 +1007,10 @@ mod test { let partition_changes = partition_changes(Some(&partition_template), &all_partitions).unwrap(); let task_changes = task_changes( - Some((&shard_template, &recovery_template)), + TaskTemplate::UpsertReal { + shard: shard_template, + recovery_journal: recovery_template, + }, &all_shards, &all_recovery, 4, @@ -983,7 +1032,10 @@ mod test { map_shard_to_split(parent_id, parent_set, *parent_revision, false).unwrap(); let key_changes = task_changes( - Some((&shard_template, &recovery_template)), + TaskTemplate::UpsertReal { + shard: shard_template, + recovery_journal: recovery_template, + }, &key_splits, &all_recovery[..1], 4, @@ -993,7 +1045,10 @@ mod test { .unwrap(); let clock_changes = task_changes( - Some((&shard_template, &recovery_template)), + TaskTemplate::UpsertReal { + shard: shard_template, + recovery_journal: recovery_template, + }, &clock_splits, &all_recovery[..1], 4, From 13e45c72ea364c3fb081bb1f26ebd410bee513d8 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Fri, 20 Sep 2024 14:42:46 -0400 Subject: [PATCH 03/26] schema: Update `flow.schema.json` to include `MaterializationEndpoint::Dekaf` --- flow.schema.json | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/flow.schema.json b/flow.schema.json index 16c83fbe35..1573eeef73 100644 --- a/flow.schema.json +++ b/flow.schema.json @@ -456,6 +456,10 @@ } } }, + "DekafConfig": { + "description": "Dekaf configuration. Currently empty, but present to enable easy addition of config options when they show up in the future.", + "type": "object" + }, "Derivation": { "description": "Derive specifies how a collection is derived from other collections.", "type": "object", @@ -913,6 +917,19 @@ } }, "additionalProperties": false + }, + { + "title": "A Dekaf connection", + "type": "object", + "required": [ + "dekaf" + ], + "properties": { + "dekaf": { + "$ref": "#/definitions/DekafConfig" + } + }, + "additionalProperties": false } ] }, From 1576b90e5401d2d50902c207c2b06b5ae4f68da3 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Tue, 24 Sep 2024 16:52:52 -0400 Subject: [PATCH 04/26] dekaf: Allow for indirecting `DekafConfig` if/when desired --- crates/flowctl/src/generate/mod.rs | 8 +- crates/models/src/lib.rs | 4 +- crates/models/src/materializations.rs | 9 +- crates/sources/src/indirect.rs | 27 +- crates/sources/src/loader.rs | 21 +- ...a_generation__catalog_schema_snapshot.snap | 17 +- crates/validation/tests/scenario_tests.rs | 95 ++- ...dekaf_materialization_indirect_config.snap | 661 ++++++++++++++++++ ..._dekaf_materialization_inline_config.snap} | 0 ..._tests__dekaf_materialization_invalid.snap | 51 ++ ...ts__dekaf_materialization_nonexistent.snap | 661 ++++++++++++++++++ flow.schema.json | 12 +- 12 files changed, 1550 insertions(+), 16 deletions(-) create mode 100644 crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap rename crates/validation/tests/snapshots/{scenario_tests__dekaf_materialization.snap => scenario_tests__dekaf_materialization_inline_config.snap} (100%) create mode 100644 crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap create mode 100644 crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap diff --git a/crates/flowctl/src/generate/mod.rs b/crates/flowctl/src/generate/mod.rs index 5ce69cf8cc..b77946b265 100644 --- a/crates/flowctl/src/generate/mod.rs +++ b/crates/flowctl/src/generate/mod.rs @@ -3,6 +3,8 @@ use anyhow::Context; use futures::{FutureExt, StreamExt}; use itertools::Itertools; use proto_flow::{capture, derive, flow, materialize}; +use std::str::FromStr; +use url::Url; #[derive(Debug, clap::Args)] #[clap(rename_all = "kebab-case")] @@ -292,8 +294,10 @@ async fn generate_missing_materialization_configs( connector_type: flow::materialization_spec::ConnectorType::Dekaf as i32, config_json: serde_json::to_string(config).unwrap(), }, - // Dekaf isn't a pluggable connector, and so does not have dynamic config. - None, + match &config { + models::DekafConfigContainer::Indirect(s) => Url::from_str(s.as_str()).ok(), + _ => None, + }, ), }; let missing_resource_urls: Vec<(url::Url, models::Collection)> = bindings diff --git a/crates/models/src/lib.rs b/crates/models/src/lib.rs index 9cb965508b..d9e75534fd 100644 --- a/crates/models/src/lib.rs +++ b/crates/models/src/lib.rs @@ -36,8 +36,8 @@ pub use journals::{ AZURE_STORAGE_ACCOUNT_RE, GCS_BUCKET_RE, S3_BUCKET_RE, }; pub use materializations::{ - MaterializationBinding, MaterializationDef, MaterializationEndpoint, MaterializationFields, - SqliteConfig, + DekafConfigContainer, MaterializationBinding, MaterializationDef, MaterializationEndpoint, + MaterializationFields, SqliteConfig, }; pub use raw_value::RawValue; pub use references::{ diff --git a/crates/models/src/materializations.rs b/crates/models/src/materializations.rs index 2c338fe60f..31b86b9b0f 100644 --- a/crates/models/src/materializations.rs +++ b/crates/models/src/materializations.rs @@ -42,6 +42,13 @@ pub struct MaterializationDef { pub delete: bool, } +#[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] +#[serde(untagged)] +pub enum DekafConfigContainer { + Direct(DekafConfig), + Indirect(String), +} + /// An Endpoint connector used for Flow materializations. #[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] #[serde(deny_unknown_fields, rename_all = "camelCase")] @@ -52,7 +59,7 @@ pub enum MaterializationEndpoint { /// # A local command (development only). Local(LocalConfig), /// # A Dekaf connection - Dekaf(DekafConfig), + Dekaf(DekafConfigContainer), } #[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] diff --git a/crates/sources/src/indirect.rs b/crates/sources/src/indirect.rs index 60986e4ae7..c7a36069a1 100644 --- a/crates/sources/src/indirect.rs +++ b/crates/sources/src/indirect.rs @@ -1,5 +1,6 @@ use super::Format; use crate::Scope; +use models::RawValue; use proto_flow::flow::ContentType; use std::collections::BTreeMap; @@ -498,10 +499,28 @@ fn indirect_materialization( resources, threshold, ), - // Dekaf isn't a pluggable connector, and so does not have dynamic config. - // All of its config is defined directly within models::DekafConfig, and so - // should not be indirected. - models::MaterializationEndpoint::Dekaf(_) => {} + // I don't think this case can ever get hit as `indirect_materialization` is only called by + // `do_discover`, `do_pull_specs`, and `do_develop`, all of which are working with fully + // inlined specs. + models::MaterializationEndpoint::Dekaf(models::DekafConfigContainer::Indirect(_)) => { + tracing::warn!("Unexpectedly tried to indirect an already indirected location (dekaf)"); + } + models::MaterializationEndpoint::Dekaf(models::DekafConfigContainer::Direct(config)) => { + indirect_dom( + Scope::new(scope) + .push_prop("endpoint") + .push_prop("local") + .push_prop("config"), + &mut RawValue::from_value( + &serde_json::to_value(config).expect("Serializing DekafConfig should not fail"), + ), + ContentType::Config, + format!("{base}.config"), + imports, + resources, + threshold, + ) + } } for (index, models::MaterializationBinding { resource, .. }) in bindings.iter_mut().enumerate() diff --git a/crates/sources/src/loader.rs b/crates/sources/src/loader.rs index be6ad06acf..0a0e93b1da 100644 --- a/crates/sources/src/loader.rs +++ b/crates/sources/src/loader.rs @@ -687,9 +687,24 @@ impl Loader { .boxed(), ); } - // Dekaf isn't a pluggable connector, and so does not have dynamic config to possibly - // load from a reference. All of its config is defined directly within models::DekafConfig. - models::MaterializationEndpoint::Dekaf(_) => {} + models::MaterializationEndpoint::Dekaf(models::DekafConfigContainer::Indirect( + location, + )) => { + tasks.push( + async move { + self.load_config( + scope + .push_prop("endpoint") + .push_prop("dekaf") + .push_prop("config"), + location.as_str(), + ) + .await + } + .boxed(), + ); + } + models::MaterializationEndpoint::Dekaf(models::DekafConfigContainer::Direct(_)) => {} }; for (index, binding) in spec.bindings.iter().enumerate() { diff --git a/crates/sources/tests/snapshots/schema_generation__catalog_schema_snapshot.snap b/crates/sources/tests/snapshots/schema_generation__catalog_schema_snapshot.snap index 057046a356..cee3f54fc0 100644 --- a/crates/sources/tests/snapshots/schema_generation__catalog_schema_snapshot.snap +++ b/crates/sources/tests/snapshots/schema_generation__catalog_schema_snapshot.snap @@ -462,8 +462,21 @@ expression: "&schema" } }, "DekafConfig": { - "description": "Dekaf configuration. Currently empty, but present to enable easy addition of config options when they show up in the future.", - "type": "object" + "description": "Dekaf service configuration", + "type": "object", + "required": [ + "config", + "variant" + ], + "properties": { + "config": { + "title": "Dekaf endpoint config." + }, + "variant": { + "title": "Dekaf variant type", + "type": "string" + } + } }, "Derivation": { "description": "Derive specifies how a collection is derived from other collections.", diff --git a/crates/validation/tests/scenario_tests.rs b/crates/validation/tests/scenario_tests.rs index 90366507ef..5836fd7d6a 100644 --- a/crates/validation/tests/scenario_tests.rs +++ b/crates/validation/tests/scenario_tests.rs @@ -9,7 +9,7 @@ fn test_golden_all_visits() { } #[test] -fn test_dekaf_materialization() { +fn test_dekaf_materialization_inline_config() { let fixture = r##" test://example/catalog.yaml: collections: @@ -38,6 +38,99 @@ driver: insta::assert_debug_snapshot!(outcome); } +#[test] +fn test_dekaf_materialization_indirect_config() { + let fixture = r##" +test://example/dekaf.yaml: {} +test://example/catalog.yaml: + collections: + testing/schema_with_properties: + schema: + type: object + properties: + id: { type: string } + required: [id] + key: [/id] + materializations: + testing/test_dekaf: + endpoint: + dekaf: example/dekaf.yaml + bindings: + - source: testing/schema_with_properties + resource: {} +driver: + dataPlanes: + "1d:1d:1d:1d:1d:1d:1d:1d": + default: true +"##; + + let outcome = common::run(fixture, "{}"); + // Expect not to see any projections for the empty properties + insta::assert_debug_snapshot!(outcome); +} + +#[test] +fn test_dekaf_materialization_invalid() { + let fixture = r##" +test://example/dekaf.yaml: {} +test://example/catalog.yaml: + collections: + testing/schema_with_properties: + schema: + type: object + properties: + id: { type: string } + required: [id] + key: [/id] + materializations: + testing/test_dekaf: + endpoint: + dekaf: false + bindings: + - source: testing/schema_with_properties + resource: {} +driver: + dataPlanes: + "1d:1d:1d:1d:1d:1d:1d:1d": + default: true +"##; + + let outcome = common::run(fixture, "{}"); + // Expect not to see any projections for the empty properties + insta::assert_debug_snapshot!(outcome); +} + +#[test] +fn test_dekaf_materialization_nonexistent() { + let fixture = r##" +test://example/dekaf.yaml: {} +test://example/catalog.yaml: + collections: + testing/schema_with_properties: + schema: + type: object + properties: + id: { type: string } + required: [id] + key: [/id] + materializations: + testing/test_dekaf: + endpoint: + dekaf: foo/bar + bindings: + - source: testing/schema_with_properties + resource: {} +driver: + dataPlanes: + "1d:1d:1d:1d:1d:1d:1d:1d": + default: true +"##; + + let outcome = common::run(fixture, "{}"); + // Expect not to see any projections for the empty properties + insta::assert_debug_snapshot!(outcome); +} + #[test] fn test_projection_not_created_for_empty_properties() { let fixture = r##" diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap new file mode 100644 index 0000000000..961ba8d6fa --- /dev/null +++ b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap @@ -0,0 +1,661 @@ +--- +source: crates/validation/tests/scenario_tests.rs +assertion_line: 69 +expression: outcome +--- +Outcome { + built_captures: [], + built_collections: [ + BuiltCollection { + collection: testing/schema_with_properties, + scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, + control_id: "0000000000000000", + data_plane_id: "1d1d1d1d1d1d1d1d", + expect_pub_id: "0000000000000000", + expect_build_id: "0000000000000000", + model: { + "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, + "key": [ + "/id" + ] + }, + validated: NULL, + spec: CollectionSpec { + name: "testing/schema_with_properties", + write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", + read_schema_json: "", + key: [ + "/id", + ], + uuid_ptr: "/_meta/uuid", + partition_fields: [], + projections: [ + Projection { + ptr: "/_meta/flow_truncated", + field: "_meta/flow_truncated", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "boolean", + ], + string: None, + title: "Flow truncation indicator", + description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "", + field: "flow_document", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "object", + ], + string: None, + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/_meta/uuid", + field: "flow_published_at", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "date-time", + content_encoding: "uuid", + max_length: 0, + }, + ), + title: "Flow Publication Time", + description: "Flow publication date-time of this document", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/id", + field: "id", + explicit: false, + is_partition_key: false, + is_primary_key: true, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "", + content_encoding: "", + max_length: 0, + }, + ), + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + ], + ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", + partition_template: Some( + JournalSpec { + name: "testing/schema_with_properties/2020202020202020", + replication: 3, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "content-type", + value: "application/x-ndjson", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/collection", + value: "testing/schema_with_properties", + prefix: false, + }, + ], + }, + ), + fragment: Some( + Fragment { + length: 536870912, + compression_codec: Gzip, + stores: [ + "s3://a-bucket/", + ], + refresh_interval: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + retention: None, + flush_interval: None, + path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", + }, + ), + flags: 4, + max_append_rate: 4194304, + }, + ), + derivation: None, + }, + previous_spec: NULL, + is_touch: 0, + dependency_hash: NULL, + }, + ], + built_materializations: [ + BuiltMaterialization { + materialization: testing/test_dekaf, + scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, + control_id: "0000000000000000", + data_plane_id: "1d1d1d1d1d1d1d1d", + expect_pub_id: "0000000000000000", + expect_build_id: "0000000000000000", + model: { + "endpoint": { + "dekaf": "example/dekaf.yaml" + }, + "bindings": [ + { + "resource": {}, + "source": "testing/schema_with_properties", + "fields": { + "recommended": true + } + } + ] + }, + validated: Validated { + bindings: [ + Binding { + constraints: { + "_meta/flow_truncated": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + "flow_document": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + "flow_published_at": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + "id": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + }, + resource_path: [ + "binding-0", + ], + delta_updates: true, + }, + ], + }, + spec: MaterializationSpec { + name: "testing/test_dekaf", + connector_type: Dekaf, + config_json: "\"example/dekaf.yaml\"", + bindings: [ + Binding { + resource_config_json: "{}", + resource_path: [ + "binding-0", + ], + collection: Some( + CollectionSpec { + name: "testing/schema_with_properties", + write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", + read_schema_json: "", + key: [ + "/id", + ], + uuid_ptr: "/_meta/uuid", + partition_fields: [], + projections: [ + Projection { + ptr: "/_meta/flow_truncated", + field: "_meta/flow_truncated", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "boolean", + ], + string: None, + title: "Flow truncation indicator", + description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "", + field: "flow_document", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "object", + ], + string: None, + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/_meta/uuid", + field: "flow_published_at", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "date-time", + content_encoding: "uuid", + max_length: 0, + }, + ), + title: "Flow Publication Time", + description: "Flow publication date-time of this document", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/id", + field: "id", + explicit: false, + is_partition_key: false, + is_primary_key: true, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "", + content_encoding: "", + max_length: 0, + }, + ), + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + ], + ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", + partition_template: Some( + JournalSpec { + name: "testing/schema_with_properties/2020202020202020", + replication: 3, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "content-type", + value: "application/x-ndjson", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/collection", + value: "testing/schema_with_properties", + prefix: false, + }, + ], + }, + ), + fragment: Some( + Fragment { + length: 536870912, + compression_codec: Gzip, + stores: [ + "s3://a-bucket/", + ], + refresh_interval: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + retention: None, + flush_interval: None, + path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", + }, + ), + flags: 4, + max_append_rate: 4194304, + }, + ), + derivation: None, + }, + ), + partition_selector: Some( + LabelSelector { + include: Some( + LabelSet { + labels: [ + Label { + name: "estuary.dev/collection", + value: "testing/schema_with_properties", + prefix: false, + }, + ], + }, + ), + exclude: Some( + LabelSet { + labels: [], + }, + ), + }, + ), + priority: 0, + field_selection: Some( + FieldSelection { + keys: [], + values: [], + document: "", + field_config_json_map: {}, + }, + ), + delta_updates: true, + deprecated_shuffle: None, + journal_read_suffix: "materialize/testing/test_dekaf/binding-0", + not_before: None, + not_after: None, + backfill: 0, + state_key: "binding-0", + }, + ], + shard_template: Some( + ShardSpec { + id: "materialize/testing/test_dekaf/2020202020202020", + sources: [], + recovery_log_prefix: "recovery", + hint_prefix: "/estuary/flow/hints", + hint_backups: 2, + max_txn_duration: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + min_txn_duration: Some( + Duration { + seconds: 0, + nanos: 0, + }, + ), + disable: false, + hot_standbys: 0, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/log-level", + value: "info", + prefix: false, + }, + Label { + name: "estuary.dev/task-name", + value: "testing/test_dekaf", + prefix: false, + }, + Label { + name: "estuary.dev/task-type", + value: "materialization", + prefix: false, + }, + ], + }, + ), + disable_wait_for_ack: false, + ring_buffer_size: 65536, + read_channel_size: 4096, + }, + ), + recovery_log_template: Some( + JournalSpec { + name: "recovery/materialize/testing/test_dekaf/2020202020202020", + replication: 3, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "content-type", + value: "application/x-gazette-recoverylog", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/task-name", + value: "testing/test_dekaf", + prefix: false, + }, + Label { + name: "estuary.dev/task-type", + value: "materialization", + prefix: false, + }, + ], + }, + ), + fragment: Some( + Fragment { + length: 268435456, + compression_codec: Snappy, + stores: [ + "s3://a-bucket/", + ], + refresh_interval: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + retention: None, + flush_interval: None, + path_postfix_template: "", + }, + ), + flags: 4, + max_append_rate: 4194304, + }, + ), + network_ports: [], + }, + previous_spec: NULL, + is_touch: 0, + dependency_hash: 32182596aeb1e4a0, + }, + ], + built_tests: [], + captures: [], + collections: [ + DraftCollection { + collection: testing/schema_with_properties, + scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, + expect_pub_id: NULL, + model: { + "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, + "key": [ + "/id" + ] + }, + is_touch: 0, + }, + ], + errors: [], + errors_draft: [ + Error { + scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf/endpoint/dekaf/config, + error: failed to fetch resource test://example/example/dekaf.yaml: fixture not found, + }, + ], + fetches: [ + Fetch { + depth: 1, + resource: test://example/catalog.yaml, + }, + Fetch { + depth: 2, + resource: test://example/example/dekaf.yaml, + }, + ], + imports: [ + Import { + scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf/endpoint/dekaf/config, + to_resource: test://example/example/dekaf.yaml, + }, + ], + materializations: [ + DraftMaterialization { + materialization: testing/test_dekaf, + scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, + expect_pub_id: NULL, + model: { + "endpoint": { + "dekaf": "example/dekaf.yaml" + }, + "bindings": [ + { + "resource": {}, + "source": "testing/schema_with_properties", + "fields": { + "recommended": true + } + } + ] + }, + is_touch: 0, + }, + ], + resources: [ + Resource { + resource: test://example/catalog.yaml, + content_type: "CATALOG", + content: ".. binary ..", + content_dom: {"collections":{"testing/schema_with_properties":{"key":["/id"],"schema":{"properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}}},"materializations":{"testing/test_dekaf":{"bindings":[{"resource":{},"source":"testing/schema_with_properties"}],"endpoint":{"dekaf":"example/dekaf.yaml"}}}}, + }, + ], + storage_mappings: [ + StorageMapping { + catalog_prefix: , + control_id: "0000000000000000", + stores: [ + { + "provider": "S3", + "bucket": "a-bucket", + "prefix": null, + "region": null + } + ], + }, + ], + tests: [], +} diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_inline_config.snap similarity index 100% rename from crates/validation/tests/snapshots/scenario_tests__dekaf_materialization.snap rename to crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_inline_config.snap diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap new file mode 100644 index 0000000000..697478a621 --- /dev/null +++ b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap @@ -0,0 +1,51 @@ +--- +source: crates/validation/tests/scenario_tests.rs +assertion_line: 100 +expression: outcome +--- +Outcome { + built_captures: [], + built_collections: [], + built_materializations: [], + built_tests: [], + captures: [], + collections: [], + errors: [], + errors_draft: [ + Error { + scope: test://example/catalog.yaml, + error: failed to parse document (data did not match any variant of untagged enum DekafConfigContainer at line 1 column 288): data did not match any variant of untagged enum DekafConfigContainer at line 1 column 288, + }, + ], + fetches: [ + Fetch { + depth: 1, + resource: test://example/catalog.yaml, + }, + ], + imports: [], + materializations: [], + resources: [ + Resource { + resource: test://example/catalog.yaml, + content_type: "CATALOG", + content: ".. binary ..", + content_dom: {"collections":{"testing/schema_with_properties":{"key":["/id"],"schema":{"properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}}},"materializations":{"testing/test_dekaf":{"bindings":[{"resource":{},"source":"testing/schema_with_properties"}],"endpoint":{"dekaf":false}}}}, + }, + ], + storage_mappings: [ + StorageMapping { + catalog_prefix: , + control_id: "0000000000000000", + stores: [ + { + "provider": "S3", + "bucket": "a-bucket", + "prefix": null, + "region": null + } + ], + }, + ], + tests: [], +} diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap new file mode 100644 index 0000000000..7240236b95 --- /dev/null +++ b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap @@ -0,0 +1,661 @@ +--- +source: crates/validation/tests/scenario_tests.rs +assertion_line: 131 +expression: outcome +--- +Outcome { + built_captures: [], + built_collections: [ + BuiltCollection { + collection: testing/schema_with_properties, + scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, + control_id: "0000000000000000", + data_plane_id: "1d1d1d1d1d1d1d1d", + expect_pub_id: "0000000000000000", + expect_build_id: "0000000000000000", + model: { + "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, + "key": [ + "/id" + ] + }, + validated: NULL, + spec: CollectionSpec { + name: "testing/schema_with_properties", + write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", + read_schema_json: "", + key: [ + "/id", + ], + uuid_ptr: "/_meta/uuid", + partition_fields: [], + projections: [ + Projection { + ptr: "/_meta/flow_truncated", + field: "_meta/flow_truncated", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "boolean", + ], + string: None, + title: "Flow truncation indicator", + description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "", + field: "flow_document", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "object", + ], + string: None, + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/_meta/uuid", + field: "flow_published_at", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "date-time", + content_encoding: "uuid", + max_length: 0, + }, + ), + title: "Flow Publication Time", + description: "Flow publication date-time of this document", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/id", + field: "id", + explicit: false, + is_partition_key: false, + is_primary_key: true, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "", + content_encoding: "", + max_length: 0, + }, + ), + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + ], + ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", + partition_template: Some( + JournalSpec { + name: "testing/schema_with_properties/2020202020202020", + replication: 3, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "content-type", + value: "application/x-ndjson", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/collection", + value: "testing/schema_with_properties", + prefix: false, + }, + ], + }, + ), + fragment: Some( + Fragment { + length: 536870912, + compression_codec: Gzip, + stores: [ + "s3://a-bucket/", + ], + refresh_interval: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + retention: None, + flush_interval: None, + path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", + }, + ), + flags: 4, + max_append_rate: 4194304, + }, + ), + derivation: None, + }, + previous_spec: NULL, + is_touch: 0, + dependency_hash: NULL, + }, + ], + built_materializations: [ + BuiltMaterialization { + materialization: testing/test_dekaf, + scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, + control_id: "0000000000000000", + data_plane_id: "1d1d1d1d1d1d1d1d", + expect_pub_id: "0000000000000000", + expect_build_id: "0000000000000000", + model: { + "endpoint": { + "dekaf": "foo/bar" + }, + "bindings": [ + { + "resource": {}, + "source": "testing/schema_with_properties", + "fields": { + "recommended": true + } + } + ] + }, + validated: Validated { + bindings: [ + Binding { + constraints: { + "_meta/flow_truncated": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + "flow_document": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + "flow_published_at": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + "id": Constraint { + r#type: FieldOptional, + reason: "no-op validator allows everything", + }, + }, + resource_path: [ + "binding-0", + ], + delta_updates: true, + }, + ], + }, + spec: MaterializationSpec { + name: "testing/test_dekaf", + connector_type: Dekaf, + config_json: "\"foo/bar\"", + bindings: [ + Binding { + resource_config_json: "{}", + resource_path: [ + "binding-0", + ], + collection: Some( + CollectionSpec { + name: "testing/schema_with_properties", + write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", + read_schema_json: "", + key: [ + "/id", + ], + uuid_ptr: "/_meta/uuid", + partition_fields: [], + projections: [ + Projection { + ptr: "/_meta/flow_truncated", + field: "_meta/flow_truncated", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "boolean", + ], + string: None, + title: "Flow truncation indicator", + description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "", + field: "flow_document", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "object", + ], + string: None, + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/_meta/uuid", + field: "flow_published_at", + explicit: false, + is_partition_key: false, + is_primary_key: false, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "date-time", + content_encoding: "uuid", + max_length: 0, + }, + ), + title: "Flow Publication Time", + description: "Flow publication date-time of this document", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + Projection { + ptr: "/id", + field: "id", + explicit: false, + is_partition_key: false, + is_primary_key: true, + inference: Some( + Inference { + types: [ + "string", + ], + string: Some( + String { + content_type: "", + format: "", + content_encoding: "", + max_length: 0, + }, + ), + title: "", + description: "", + default_json: "", + secret: false, + exists: Must, + numeric: None, + }, + ), + }, + ], + ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", + partition_template: Some( + JournalSpec { + name: "testing/schema_with_properties/2020202020202020", + replication: 3, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "content-type", + value: "application/x-ndjson", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/collection", + value: "testing/schema_with_properties", + prefix: false, + }, + ], + }, + ), + fragment: Some( + Fragment { + length: 536870912, + compression_codec: Gzip, + stores: [ + "s3://a-bucket/", + ], + refresh_interval: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + retention: None, + flush_interval: None, + path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", + }, + ), + flags: 4, + max_append_rate: 4194304, + }, + ), + derivation: None, + }, + ), + partition_selector: Some( + LabelSelector { + include: Some( + LabelSet { + labels: [ + Label { + name: "estuary.dev/collection", + value: "testing/schema_with_properties", + prefix: false, + }, + ], + }, + ), + exclude: Some( + LabelSet { + labels: [], + }, + ), + }, + ), + priority: 0, + field_selection: Some( + FieldSelection { + keys: [], + values: [], + document: "", + field_config_json_map: {}, + }, + ), + delta_updates: true, + deprecated_shuffle: None, + journal_read_suffix: "materialize/testing/test_dekaf/binding-0", + not_before: None, + not_after: None, + backfill: 0, + state_key: "binding-0", + }, + ], + shard_template: Some( + ShardSpec { + id: "materialize/testing/test_dekaf/2020202020202020", + sources: [], + recovery_log_prefix: "recovery", + hint_prefix: "/estuary/flow/hints", + hint_backups: 2, + max_txn_duration: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + min_txn_duration: Some( + Duration { + seconds: 0, + nanos: 0, + }, + ), + disable: false, + hot_standbys: 0, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/log-level", + value: "info", + prefix: false, + }, + Label { + name: "estuary.dev/task-name", + value: "testing/test_dekaf", + prefix: false, + }, + Label { + name: "estuary.dev/task-type", + value: "materialization", + prefix: false, + }, + ], + }, + ), + disable_wait_for_ack: false, + ring_buffer_size: 65536, + read_channel_size: 4096, + }, + ), + recovery_log_template: Some( + JournalSpec { + name: "recovery/materialize/testing/test_dekaf/2020202020202020", + replication: 3, + labels: Some( + LabelSet { + labels: [ + Label { + name: "app.gazette.dev/managed-by", + value: "estuary.dev/flow", + prefix: false, + }, + Label { + name: "content-type", + value: "application/x-gazette-recoverylog", + prefix: false, + }, + Label { + name: "estuary.dev/build", + value: "2121212121212121", + prefix: false, + }, + Label { + name: "estuary.dev/task-name", + value: "testing/test_dekaf", + prefix: false, + }, + Label { + name: "estuary.dev/task-type", + value: "materialization", + prefix: false, + }, + ], + }, + ), + fragment: Some( + Fragment { + length: 268435456, + compression_codec: Snappy, + stores: [ + "s3://a-bucket/", + ], + refresh_interval: Some( + Duration { + seconds: 300, + nanos: 0, + }, + ), + retention: None, + flush_interval: None, + path_postfix_template: "", + }, + ), + flags: 4, + max_append_rate: 4194304, + }, + ), + network_ports: [], + }, + previous_spec: NULL, + is_touch: 0, + dependency_hash: 32182596aeb1e4a0, + }, + ], + built_tests: [], + captures: [], + collections: [ + DraftCollection { + collection: testing/schema_with_properties, + scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, + expect_pub_id: NULL, + model: { + "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, + "key": [ + "/id" + ] + }, + is_touch: 0, + }, + ], + errors: [], + errors_draft: [ + Error { + scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf/endpoint/dekaf/config, + error: failed to fetch resource test://example/foo/bar: fixture not found, + }, + ], + fetches: [ + Fetch { + depth: 1, + resource: test://example/catalog.yaml, + }, + Fetch { + depth: 2, + resource: test://example/foo/bar, + }, + ], + imports: [ + Import { + scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf/endpoint/dekaf/config, + to_resource: test://example/foo/bar, + }, + ], + materializations: [ + DraftMaterialization { + materialization: testing/test_dekaf, + scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, + expect_pub_id: NULL, + model: { + "endpoint": { + "dekaf": "foo/bar" + }, + "bindings": [ + { + "resource": {}, + "source": "testing/schema_with_properties", + "fields": { + "recommended": true + } + } + ] + }, + is_touch: 0, + }, + ], + resources: [ + Resource { + resource: test://example/catalog.yaml, + content_type: "CATALOG", + content: ".. binary ..", + content_dom: {"collections":{"testing/schema_with_properties":{"key":["/id"],"schema":{"properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}}},"materializations":{"testing/test_dekaf":{"bindings":[{"resource":{},"source":"testing/schema_with_properties"}],"endpoint":{"dekaf":"foo/bar"}}}}, + }, + ], + storage_mappings: [ + StorageMapping { + catalog_prefix: , + control_id: "0000000000000000", + stores: [ + { + "provider": "S3", + "bucket": "a-bucket", + "prefix": null, + "region": null + } + ], + }, + ], + tests: [], +} diff --git a/flow.schema.json b/flow.schema.json index 1573eeef73..775d792483 100644 --- a/flow.schema.json +++ b/flow.schema.json @@ -460,6 +460,16 @@ "description": "Dekaf configuration. Currently empty, but present to enable easy addition of config options when they show up in the future.", "type": "object" }, + "DekafConfigContainer": { + "anyOf": [ + { + "$ref": "#/definitions/DekafConfig" + }, + { + "type": "string" + } + ] + }, "Derivation": { "description": "Derive specifies how a collection is derived from other collections.", "type": "object", @@ -926,7 +936,7 @@ ], "properties": { "dekaf": { - "$ref": "#/definitions/DekafConfig" + "$ref": "#/definitions/DekafConfigContainer" } }, "additionalProperties": false From c101864f702abf044f2bdb503602ff13bf439a82 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Thu, 26 Sep 2024 14:01:30 -0400 Subject: [PATCH 05/26] dekaf: Introduce and validate `DekafResourceConfig` to configure a particular dekaf binding --- crates/models/src/connector.rs | 23 ++++++++++++++++++----- crates/models/src/lib.rs | 4 +++- crates/validation/src/materialization.rs | 14 ++++++++++++++ 3 files changed, 35 insertions(+), 6 deletions(-) diff --git a/crates/models/src/connector.rs b/crates/models/src/connector.rs index d97bd98484..704096fd6e 100644 --- a/crates/models/src/connector.rs +++ b/crates/models/src/connector.rs @@ -17,7 +17,7 @@ pub fn split_image_tag(image_full: &str) -> (String, String) { } } -/// Connector image and configuration specification. +/// Dekaf service configuration #[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] pub struct ConnectorConfig { /// # Image of the connector. @@ -66,13 +66,26 @@ impl LocalConfig { } } -/// Dekaf configuration. Currently empty, but present to enable easy addition -/// of config options when they show up in the future. +/// Configures the behavior of a whole dekaf task #[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] -pub struct DekafConfig {} +pub struct DekafConfig { + /// Whether or not to expose topic names in a strictly Kafka-compliant format + /// for systems that require it. Off by default. + pub strict_topic_names: bool, +} impl DekafConfig { pub fn example() -> Self { - Self {} + Self { + strict_topic_names: false, + } } } + +/// Configures a particular binding in a Dekaf-type materialization +#[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] +pub struct DekafResourceConfig { + /// The exposed name of the topic that maps to this binding. This + /// will be exposed through the Kafka metadata/discovery APIs. + pub topic_name: String, +} diff --git a/crates/models/src/lib.rs b/crates/models/src/lib.rs index d9e75534fd..a62986830e 100644 --- a/crates/models/src/lib.rs +++ b/crates/models/src/lib.rs @@ -25,7 +25,9 @@ pub use crate::labels::{Label, LabelSelector, LabelSet}; pub use captures::{AutoDiscover, CaptureBinding, CaptureDef, CaptureEndpoint}; pub use catalogs::{Capability, Catalog, CatalogType}; pub use collections::{CollectionDef, Projection}; -pub use connector::{split_image_tag, ConnectorConfig, DekafConfig, LocalConfig}; +pub use connector::{ + split_image_tag, ConnectorConfig, DekafConfig, DekafResourceConfig, LocalConfig, +}; pub use derivation::{Derivation, DeriveUsing, Shuffle, ShuffleType, TransformDef}; pub use derive_sqlite::DeriveUsingSqlite; pub use derive_typescript::DeriveUsingTypescript; diff --git a/crates/validation/src/materialization.rs b/crates/validation/src/materialization.rs index ba4cf7175f..d404243d35 100644 --- a/crates/validation/src/materialization.rs +++ b/crates/validation/src/materialization.rs @@ -4,6 +4,7 @@ use super::{ }; use itertools::Itertools; use proto_flow::{flow, materialize, ops::log::Level as LogLevel}; +use serde::Deserialize; use std::collections::{BTreeMap, HashMap}; use tables::EitherOrBoth as EOB; @@ -144,6 +145,7 @@ async fn walk_materialization( materialization, binding, built_collections, + endpoint, errors, ) }) @@ -380,6 +382,7 @@ fn walk_materialization_binding<'a>( catalog_name: &models::Materialization, binding: &'a models::MaterializationBinding, built_collections: &'a tables::BuiltCollections, + endpoint: &'a models::MaterializationEndpoint, errors: &mut tables::Errors, ) -> Option { let models::MaterializationBinding { @@ -397,6 +400,17 @@ fn walk_materialization_binding<'a>( on_incompatible_schema_change: _, } = binding; + if matches!(endpoint, models::MaterializationEndpoint::Dekaf(_)) { + // Validate dekaf resource config + match models::DekafResourceConfig::deserialize(resource.to_value()) { + Ok(_) => {} + Err(err) => { + Error::from(err).push(scope.push_prop("resource"), errors); + return None; + } + } + } + let (collection, source_partitions) = match source { models::Source::Collection(collection) => (collection, None), models::Source::Source(models::FullSource { From c9ff4df41cccbce72d5f4d68780e517e93263b0b Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Sun, 29 Sep 2024 22:02:38 -0400 Subject: [PATCH 06/26] temporarily remove `dekaf` dependency on `flowctl` --- Cargo.lock | 2 +- crates/agent/Cargo.toml | 1 + crates/agent/src/proxy_connectors.rs | 23 +++++--- crates/dekaf/Cargo.toml | 2 +- crates/dekaf/src/connector.rs | 68 ++++++++++++++++++++++++ crates/dekaf/src/lib.rs | 3 +- crates/dekaf/src/registry.rs | 52 +++++++++--------- crates/dekaf/src/session.rs | 4 +- crates/dekaf/src/topology.rs | 2 +- crates/flowctl/src/generate/mod.rs | 5 +- crates/models/src/connector.rs | 35 ++++-------- crates/models/src/lib.rs | 8 ++- crates/models/src/materializations.rs | 9 +--- crates/sources/src/indirect.rs | 34 +++++------- crates/sources/src/loader.rs | 7 +-- crates/validation/src/materialization.rs | 19 +------ flow.schema.json | 23 ++++---- 17 files changed, 162 insertions(+), 135 deletions(-) create mode 100644 crates/dekaf/src/connector.rs diff --git a/Cargo.lock b/Cargo.lock index aa5709896f..fa65c9c73d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -112,6 +112,7 @@ dependencies = [ "chrono", "clap 4.5.17", "colored_json", + "dekaf", "derivative", "doc", "futures", @@ -1825,7 +1826,6 @@ dependencies = [ "crypto-common", "deadpool", "doc", - "flowctl", "futures", "gazette", "hex", diff --git a/crates/agent/Cargo.toml b/crates/agent/Cargo.toml index beced2121d..3776cd6ec8 100644 --- a/crates/agent/Cargo.toml +++ b/crates/agent/Cargo.toml @@ -15,6 +15,7 @@ activate = { path = "../activate" } agent-sql = { path = "../agent-sql" } allocator = { path = "../allocator" } async-process = { path = "../async-process" } +dekaf = { path = "../dekaf" } build = { path = "../build" } doc = { path = "../doc" } gazette = { path = "../gazette" } diff --git a/crates/agent/src/proxy_connectors.rs b/crates/agent/src/proxy_connectors.rs index b47a16d6d2..ea963f8845 100644 --- a/crates/agent/src/proxy_connectors.rs +++ b/crates/agent/src/proxy_connectors.rs @@ -1,6 +1,6 @@ use anyhow::Context; use futures::{FutureExt, TryFutureExt, TryStreamExt}; -use proto_flow::{capture, derive, materialize}; +use proto_flow::{capture, derive, flow::materialization_spec, materialize}; use std::future::Future; pub struct ProxyConnectors { @@ -40,12 +40,21 @@ impl validation::Connectors for ProxyConnectors { request: materialize::Request, data_plane: &'a tables::DataPlane, ) -> futures::future::BoxFuture<'a, anyhow::Result> { - let task = ops::ShardRef { - name: request.validate.as_ref().unwrap().name.clone(), - kind: ops::TaskType::Materialization as i32, - ..Default::default() - }; - self.unary_materialize(data_plane, task, request).boxed() + match materialization_spec::ConnectorType::try_from( + request.validate.as_ref().unwrap().connector_type, + ) { + Ok(materialization_spec::ConnectorType::Dekaf) => { + dekaf::connector::unary_materialize(request).boxed() + } + _ => { + let task = ops::ShardRef { + name: request.validate.as_ref().unwrap().name.clone(), + kind: ops::TaskType::Materialization as i32, + ..Default::default() + }; + self.unary_materialize(data_plane, task, request).boxed() + } + } } } diff --git a/crates/dekaf/Cargo.toml b/crates/dekaf/Cargo.toml index c94317aa81..30325fc1f0 100644 --- a/crates/dekaf/Cargo.toml +++ b/crates/dekaf/Cargo.toml @@ -14,7 +14,7 @@ aes-siv = { workspace = true } allocator = { path = "../allocator" } avro = { path = "../avro" } doc = { path = "../doc" } -flowctl = { path = "../flowctl" } +# flowctl = { path = "../flowctl" } gazette = { path = "../gazette" } labels = { path = "../labels" } ops = { path = "../ops" } diff --git a/crates/dekaf/src/connector.rs b/crates/dekaf/src/connector.rs new file mode 100644 index 0000000000..44d435aedb --- /dev/null +++ b/crates/dekaf/src/connector.rs @@ -0,0 +1,68 @@ +use anyhow::{bail, Context}; +use proto_flow::materialize; +use serde::{Deserialize, Serialize}; +use std::collections::BTreeMap; + +/// Configures the behavior of a whole dekaf task +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] +pub struct DekafConfig { + /// Whether or not to expose topic names in a strictly Kafka-compliant format + /// for systems that require it. Off by default. + pub strict_topic_names: bool, +} + +/// Configures a particular binding in a Dekaf-type materialization +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] +pub struct DekafResourceConfig { + /// The exposed name of the topic that maps to this binding. This + /// will be exposed through the Kafka metadata/discovery APIs. + pub topic_name: String, +} + +pub async fn unary_materialize( + request: materialize::Request, +) -> anyhow::Result { + use proto_flow::materialize::response::validated; + if let Some(mut validate) = request.validate { + serde_json::de::from_str::(&validate.config_json) + .context("validating endpoint config")?; + + // Largely copied from crates/validation/src/noop.rs + let validated_bindings = std::mem::take(&mut validate.bindings) + .into_iter() + .enumerate() + .map(|(i, b)| { + let resource_path = vec![format!("binding-{}", i)]; + let constraints = b + .collection + .expect("collection must exist") + .projections + .into_iter() + .map(|proj| { + ( + proj.field, + validated::Constraint { + r#type: validated::constraint::Type::FieldOptional as i32, + reason: "Dekaf allows everything for now".to_string(), + }, + ) + }) + .collect::>(); + validated::Binding { + constraints, + resource_path, + delta_updates: false, + } + }) + .collect::>(); + + return Ok(materialize::Response { + validated: Some(materialize::response::Validated { + bindings: validated_bindings, + }), + ..Default::default() + }); + } else { + bail!("Unhandled request type") + } +} diff --git a/crates/dekaf/src/lib.rs b/crates/dekaf/src/lib.rs index 538b6da59a..8c4a061e25 100644 --- a/crates/dekaf/src/lib.rs +++ b/crates/dekaf/src/lib.rs @@ -7,7 +7,7 @@ use kafka_protocol::{ use tracing::instrument; mod topology; -use topology::{fetch_all_collection_names, Collection, Partition}; +use topology::{Collection, Partition}; mod read; use read::Read; @@ -15,6 +15,7 @@ use read::Read; mod session; pub use session::Session; +pub mod connector; pub mod metrics_server; pub mod registry; diff --git a/crates/dekaf/src/registry.rs b/crates/dekaf/src/registry.rs index a649bc23a3..e7fc45ecc9 100644 --- a/crates/dekaf/src/registry.rs +++ b/crates/dekaf/src/registry.rs @@ -34,31 +34,33 @@ async fn all_subjects( >, ) -> Response { wrap(async move { - let Authenticated { - client, - user_config, - .. - } = app.authenticate(auth.username(), auth.password()).await?; - - super::fetch_all_collection_names(&client) - .await - .context("failed to list collections from the control plane") - .map(|collections| { - collections - .into_iter() - .map(|name| { - if user_config.strict_topic_names { - to_downstream_topic_name(TopicName::from(StrBytes::from_string(name))) - .to_string() - } else { - name - } - }) - .flat_map(|collection| { - vec![format!("{collection}-key"), format!("{collection}-value")] - }) - .collect_vec() - }) + // let Authenticated { + // client, + // user_config, + // .. + // } = app.authenticate(auth.username(), auth.password()).await?; + + let r: Vec = vec![]; + Ok(r) + // fetch_all_collection_names(&client) + // .await + // .context("failed to list collections from the control plane") + // .map(|collections| { + // collections + // .into_iter() + // .map(|name| { + // if user_config.strict_topic_names { + // to_downstream_topic_name(TopicName::from(StrBytes::from_string(name))) + // .to_string() + // } else { + // name + // } + // }) + // .flat_map(|collection| { + // vec![format!("{collection}-key"), format!("{collection}-value")] + // }) + // .collect_vec() + // }) }) .await } diff --git a/crates/dekaf/src/session.rs b/crates/dekaf/src/session.rs index 18513a9a19..0fad7ed299 100644 --- a/crates/dekaf/src/session.rs +++ b/crates/dekaf/src/session.rs @@ -1,4 +1,4 @@ -use super::{fetch_all_collection_names, App, Collection, Read}; +use super::{App, Collection, Read}; use crate::{ from_downstream_topic_name, from_upstream_topic_name, to_downstream_topic_name, to_upstream_topic_name, Authenticated, ConfigOptions, @@ -144,7 +144,7 @@ impl Session { async fn metadata_all_topics( &mut self, ) -> anyhow::Result> { - let collections = fetch_all_collection_names(&self.client).await?; + let collections = vec![]; //fetch_all_collection_names(&self.client).await?; tracing::debug!(collections=?ops::DebugJson(&collections), "fetched all collections"); diff --git a/crates/dekaf/src/topology.rs b/crates/dekaf/src/topology.rs index 66911314f4..f1c8127e7d 100644 --- a/crates/dekaf/src/topology.rs +++ b/crates/dekaf/src/topology.rs @@ -17,7 +17,7 @@ pub async fn fetch_all_collection_names( .eq("spec_type", "collection") .select("catalog_name"); - let items = flowctl::pagination::into_items::(rows_builder) + let items = flow_client::pagination::into_items::(rows_builder) .map(|res| res.map(|Row { catalog_name }| catalog_name)) .try_collect() .await diff --git a/crates/flowctl/src/generate/mod.rs b/crates/flowctl/src/generate/mod.rs index b77946b265..61b90fd9f8 100644 --- a/crates/flowctl/src/generate/mod.rs +++ b/crates/flowctl/src/generate/mod.rs @@ -294,10 +294,7 @@ async fn generate_missing_materialization_configs( connector_type: flow::materialization_spec::ConnectorType::Dekaf as i32, config_json: serde_json::to_string(config).unwrap(), }, - match &config { - models::DekafConfigContainer::Indirect(s) => Url::from_str(s.as_str()).ok(), - _ => None, - }, + serde_json::from_str::(config.config.get()).ok(), ), }; let missing_resource_urls: Vec<(url::Url, models::Collection)> = bindings diff --git a/crates/models/src/connector.rs b/crates/models/src/connector.rs index 704096fd6e..788879cd99 100644 --- a/crates/models/src/connector.rs +++ b/crates/models/src/connector.rs @@ -1,6 +1,6 @@ use super::RawValue; use schemars::JsonSchema; -use serde::{Deserialize, Serialize}; +use serde::{Deserialize, Deserializer, Serialize}; use std::collections::BTreeMap; /// Splits a full connector image name into separate image and tag components. @@ -19,6 +19,15 @@ pub fn split_image_tag(image_full: &str) -> (String, String) { /// Dekaf service configuration #[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] +pub struct DekafConfig { + /// # Dekaf variant type + pub variant: String, + /// # Dekaf endpoint config. + pub config: RawValue, +} + +/// Connector image and configuration specification. +#[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] pub struct ConnectorConfig { /// # Image of the connector. pub image: String, @@ -65,27 +74,3 @@ impl LocalConfig { } } } - -/// Configures the behavior of a whole dekaf task -#[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] -pub struct DekafConfig { - /// Whether or not to expose topic names in a strictly Kafka-compliant format - /// for systems that require it. Off by default. - pub strict_topic_names: bool, -} - -impl DekafConfig { - pub fn example() -> Self { - Self { - strict_topic_names: false, - } - } -} - -/// Configures a particular binding in a Dekaf-type materialization -#[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] -pub struct DekafResourceConfig { - /// The exposed name of the topic that maps to this binding. This - /// will be exposed through the Kafka metadata/discovery APIs. - pub topic_name: String, -} diff --git a/crates/models/src/lib.rs b/crates/models/src/lib.rs index a62986830e..9cb965508b 100644 --- a/crates/models/src/lib.rs +++ b/crates/models/src/lib.rs @@ -25,9 +25,7 @@ pub use crate::labels::{Label, LabelSelector, LabelSet}; pub use captures::{AutoDiscover, CaptureBinding, CaptureDef, CaptureEndpoint}; pub use catalogs::{Capability, Catalog, CatalogType}; pub use collections::{CollectionDef, Projection}; -pub use connector::{ - split_image_tag, ConnectorConfig, DekafConfig, DekafResourceConfig, LocalConfig, -}; +pub use connector::{split_image_tag, ConnectorConfig, DekafConfig, LocalConfig}; pub use derivation::{Derivation, DeriveUsing, Shuffle, ShuffleType, TransformDef}; pub use derive_sqlite::DeriveUsingSqlite; pub use derive_typescript::DeriveUsingTypescript; @@ -38,8 +36,8 @@ pub use journals::{ AZURE_STORAGE_ACCOUNT_RE, GCS_BUCKET_RE, S3_BUCKET_RE, }; pub use materializations::{ - DekafConfigContainer, MaterializationBinding, MaterializationDef, MaterializationEndpoint, - MaterializationFields, SqliteConfig, + MaterializationBinding, MaterializationDef, MaterializationEndpoint, MaterializationFields, + SqliteConfig, }; pub use raw_value::RawValue; pub use references::{ diff --git a/crates/models/src/materializations.rs b/crates/models/src/materializations.rs index 31b86b9b0f..2c338fe60f 100644 --- a/crates/models/src/materializations.rs +++ b/crates/models/src/materializations.rs @@ -42,13 +42,6 @@ pub struct MaterializationDef { pub delete: bool, } -#[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] -#[serde(untagged)] -pub enum DekafConfigContainer { - Direct(DekafConfig), - Indirect(String), -} - /// An Endpoint connector used for Flow materializations. #[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] #[serde(deny_unknown_fields, rename_all = "camelCase")] @@ -59,7 +52,7 @@ pub enum MaterializationEndpoint { /// # A local command (development only). Local(LocalConfig), /// # A Dekaf connection - Dekaf(DekafConfigContainer), + Dekaf(DekafConfig), } #[derive(Serialize, Deserialize, Clone, Debug, JsonSchema, PartialEq)] diff --git a/crates/sources/src/indirect.rs b/crates/sources/src/indirect.rs index c7a36069a1..17ba75fd19 100644 --- a/crates/sources/src/indirect.rs +++ b/crates/sources/src/indirect.rs @@ -499,28 +499,18 @@ fn indirect_materialization( resources, threshold, ), - // I don't think this case can ever get hit as `indirect_materialization` is only called by - // `do_discover`, `do_pull_specs`, and `do_develop`, all of which are working with fully - // inlined specs. - models::MaterializationEndpoint::Dekaf(models::DekafConfigContainer::Indirect(_)) => { - tracing::warn!("Unexpectedly tried to indirect an already indirected location (dekaf)"); - } - models::MaterializationEndpoint::Dekaf(models::DekafConfigContainer::Direct(config)) => { - indirect_dom( - Scope::new(scope) - .push_prop("endpoint") - .push_prop("local") - .push_prop("config"), - &mut RawValue::from_value( - &serde_json::to_value(config).expect("Serializing DekafConfig should not fail"), - ), - ContentType::Config, - format!("{base}.config"), - imports, - resources, - threshold, - ) - } + models::MaterializationEndpoint::Dekaf(models::DekafConfig { config, .. }) => indirect_dom( + Scope::new(scope) + .push_prop("endpoint") + .push_prop("local") + .push_prop("config"), + config, + ContentType::Config, + format!("{base}.config"), + imports, + resources, + threshold, + ), } for (index, models::MaterializationBinding { resource, .. }) in bindings.iter_mut().enumerate() diff --git a/crates/sources/src/loader.rs b/crates/sources/src/loader.rs index 0a0e93b1da..8ab360517c 100644 --- a/crates/sources/src/loader.rs +++ b/crates/sources/src/loader.rs @@ -687,9 +687,7 @@ impl Loader { .boxed(), ); } - models::MaterializationEndpoint::Dekaf(models::DekafConfigContainer::Indirect( - location, - )) => { + models::MaterializationEndpoint::Dekaf(models::DekafConfig { config, .. }) => { tasks.push( async move { self.load_config( @@ -697,14 +695,13 @@ impl Loader { .push_prop("endpoint") .push_prop("dekaf") .push_prop("config"), - location.as_str(), + config, ) .await } .boxed(), ); } - models::MaterializationEndpoint::Dekaf(models::DekafConfigContainer::Direct(_)) => {} }; for (index, binding) in spec.bindings.iter().enumerate() { diff --git a/crates/validation/src/materialization.rs b/crates/validation/src/materialization.rs index d404243d35..8151cd9d05 100644 --- a/crates/validation/src/materialization.rs +++ b/crates/validation/src/materialization.rs @@ -145,7 +145,6 @@ async fn walk_materialization( materialization, binding, built_collections, - endpoint, errors, ) }) @@ -193,11 +192,7 @@ async fn walk_materialization( }); // If shards are disabled, then don't ask the connector to validate. - let response = if shard_template.disable - // TODO(jshearer): Are we sure that Dekaf is okay with _any_ projection? - // Currently, `NoOpConnectors` report every field as `FieldOptional` - || connector_type == flow::materialization_spec::ConnectorType::Dekaf as i32 - { + let response = if shard_template.disable { NoOpConnectors.validate_materialization(wrapped_request, data_plane) } else { connectors.validate_materialization(wrapped_request, data_plane) @@ -382,7 +377,6 @@ fn walk_materialization_binding<'a>( catalog_name: &models::Materialization, binding: &'a models::MaterializationBinding, built_collections: &'a tables::BuiltCollections, - endpoint: &'a models::MaterializationEndpoint, errors: &mut tables::Errors, ) -> Option { let models::MaterializationBinding { @@ -400,17 +394,6 @@ fn walk_materialization_binding<'a>( on_incompatible_schema_change: _, } = binding; - if matches!(endpoint, models::MaterializationEndpoint::Dekaf(_)) { - // Validate dekaf resource config - match models::DekafResourceConfig::deserialize(resource.to_value()) { - Ok(_) => {} - Err(err) => { - Error::from(err).push(scope.push_prop("resource"), errors); - return None; - } - } - } - let (collection, source_partitions) = match source { models::Source::Collection(collection) => (collection, None), models::Source::Source(models::FullSource { diff --git a/flow.schema.json b/flow.schema.json index 775d792483..168f85d74b 100644 --- a/flow.schema.json +++ b/flow.schema.json @@ -457,18 +457,21 @@ } }, "DekafConfig": { - "description": "Dekaf configuration. Currently empty, but present to enable easy addition of config options when they show up in the future.", - "type": "object" - }, - "DekafConfigContainer": { - "anyOf": [ - { - "$ref": "#/definitions/DekafConfig" + "description": "Dekaf service configuration", + "type": "object", + "required": [ + "config", + "variant" + ], + "properties": { + "config": { + "title": "Dekaf endpoint config." }, - { + "variant": { + "title": "Dekaf variant type", "type": "string" } - ] + } }, "Derivation": { "description": "Derive specifies how a collection is derived from other collections.", @@ -936,7 +939,7 @@ ], "properties": { "dekaf": { - "$ref": "#/definitions/DekafConfigContainer" + "$ref": "#/definitions/DekafConfig" } }, "additionalProperties": false From b8b33d63a38031500b356444a297dadf07936333 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Sun, 29 Sep 2024 22:04:23 -0400 Subject: [PATCH 07/26] runtime: Delegate to Dekaf's `unary_materialize` when appropriate --- crates/models/src/connector.rs | 2 +- crates/runtime/Cargo.toml | 7 ++++--- crates/runtime/src/unary.rs | 15 ++++++++++++--- 3 files changed, 17 insertions(+), 7 deletions(-) diff --git a/crates/models/src/connector.rs b/crates/models/src/connector.rs index 788879cd99..1bc7ad193a 100644 --- a/crates/models/src/connector.rs +++ b/crates/models/src/connector.rs @@ -1,6 +1,6 @@ use super::RawValue; use schemars::JsonSchema; -use serde::{Deserialize, Deserializer, Serialize}; +use serde::{Deserialize, Serialize}; use std::collections::BTreeMap; /// Splits a full connector image name into separate image and tag components. diff --git a/crates/runtime/Cargo.toml b/crates/runtime/Cargo.toml index a6cdf8de7d..c87a071a95 100644 --- a/crates/runtime/Cargo.toml +++ b/crates/runtime/Cargo.toml @@ -11,8 +11,9 @@ license.workspace = true [dependencies] assemble = { path = "../assemble" } async-process = { path = "../async-process" } -connector-init = { path = "../connector-init" } -coroutines = { path = "../coroutines" } +connector-init = { path = "../connector-init" } +coroutines = { path = "../coroutines" } +dekaf = { path = "../dekaf" } derive-sqlite = { path = "../derive-sqlite" } doc = { path = "../doc" } extractors = { path = "../extractors" } @@ -32,7 +33,7 @@ proto-grpc = { path = "../proto-grpc", features = [ "materialize_client", "materialize_server", "runtime_server", - ]} +] } simd-doc = { path = "../simd-doc" } tuple = { path = "../tuple" } diff --git a/crates/runtime/src/unary.rs b/crates/runtime/src/unary.rs index 7ad7e28b82..2cd3b502d8 100644 --- a/crates/runtime/src/unary.rs +++ b/crates/runtime/src/unary.rs @@ -1,6 +1,6 @@ use super::{LogHandler, Runtime}; use futures::{stream::BoxStream, StreamExt, TryStreamExt}; -use proto_flow::{capture, derive, materialize}; +use proto_flow::{capture, derive, flow::materialization_spec, materialize}; impl Runtime { pub async fn unary_capture( @@ -20,8 +20,17 @@ impl Runtime { self, request: materialize::Request, ) -> anyhow::Result { - let response = self.serve_materialize(unary_in(request)).boxed(); - unary_out(response).await + match materialization_spec::ConnectorType::try_from( + request.validate.as_ref().unwrap().connector_type, + ) { + Ok(materialization_spec::ConnectorType::Dekaf) => { + dekaf::connector::unary_materialize(request).await + } + _ => { + let response = self.serve_materialize(unary_in(request)).boxed(); + unary_out(response).await + } + } } } From 280195d833d43fb407931bb4c598c8a7bf8db38d Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Sun, 29 Sep 2024 22:59:58 -0400 Subject: [PATCH 08/26] Factor out `flow-client` from `flowctl` Largely retain the same functionality as `flowctl::Client`, just in a new home so that it can be shared without introducing inconvenient dependencies --- Cargo.lock | 27 ++++ crates/dekaf/Cargo.toml | 3 +- crates/dekaf/src/lib.rs | 66 ++------ crates/dekaf/src/registry.rs | 54 ++++--- crates/dekaf/src/session.rs | 45 ++++-- crates/dekaf/src/topology.rs | 43 +----- crates/flow-client/Cargo.toml | 29 ++++ crates/{flowctl => flow-client}/src/client.rs | 142 +++++++++++++++--- crates/flow-client/src/lib.rs | 65 ++++++++ .../src/pagination.rs | 0 crates/flowctl/Cargo.toml | 1 + crates/flowctl/src/collection/mod.rs | 4 +- crates/flowctl/src/collection/read/mod.rs | 2 +- crates/flowctl/src/config.rs | 34 ++--- crates/flowctl/src/draft/develop.rs | 3 +- crates/flowctl/src/generate/mod.rs | 2 - crates/flowctl/src/lib.rs | 94 +----------- crates/flowctl/src/ops.rs | 2 +- crates/flowctl/src/preview/journal_reader.rs | 2 +- crates/sources/src/indirect.rs | 1 - crates/validation/src/materialization.rs | 1 - 21 files changed, 353 insertions(+), 267 deletions(-) create mode 100644 crates/flow-client/Cargo.toml rename crates/{flowctl => flow-client}/src/client.rs (59%) create mode 100644 crates/flow-client/src/lib.rs rename crates/{flowctl => flow-client}/src/pagination.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index fa65c9c73d..2823e283a9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1002,6 +1002,7 @@ dependencies = [ "anyhow", "assemble", "bytes", + "dekaf", "futures", "models", "ops", @@ -1826,6 +1827,7 @@ dependencies = [ "crypto-common", "deadpool", "doc", + "flow-client", "futures", "gazette", "hex", @@ -1837,6 +1839,7 @@ dependencies = [ "md5", "metrics", "metrics-prometheus", + "models", "ops", "percent-encoding", "postgrest", @@ -2228,6 +2231,29 @@ dependencies = [ "miniz_oxide", ] +[[package]] +name = "flow-client" +version = "0.0.0" +dependencies = [ + "anyhow", + "base64 0.13.1", + "futures", + "gazette", + "lazy_static", + "models", + "ops", + "page-turner", + "postgrest", + "reqwest", + "serde", + "serde_json", + "time 0.3.36", + "tokio", + "tonic", + "tracing", + "url", +] + [[package]] name = "flow-web" version = "0.4.0" @@ -2279,6 +2305,7 @@ dependencies = [ "dirs", "doc", "extractors", + "flow-client", "futures", "gazette", "humantime", diff --git a/crates/dekaf/Cargo.toml b/crates/dekaf/Cargo.toml index 30325fc1f0..046c6abe4a 100644 --- a/crates/dekaf/Cargo.toml +++ b/crates/dekaf/Cargo.toml @@ -14,9 +14,10 @@ aes-siv = { workspace = true } allocator = { path = "../allocator" } avro = { path = "../avro" } doc = { path = "../doc" } -# flowctl = { path = "../flowctl" } +flow-client = { path = "../flow-client" } gazette = { path = "../gazette" } labels = { path = "../labels" } +models = { path = "../models" } ops = { path = "../ops" } proto-flow = { path = "../proto-flow" } proto-gazette = { path = "../proto-gazette" } diff --git a/crates/dekaf/src/lib.rs b/crates/dekaf/src/lib.rs index 8c4a061e25..8d6e40afe2 100644 --- a/crates/dekaf/src/lib.rs +++ b/crates/dekaf/src/lib.rs @@ -23,7 +23,7 @@ mod api_client; pub use api_client::KafkaApiClient; use aes_siv::{aead::Aead, Aes256SivAead, KeyInit, KeySizeUser}; -use itertools::Itertools; +use flow_client::{DEFAULT_AGENT_URL, DEFAULT_PG_PUBLIC_TOKEN, DEFAULT_PG_URL}; use percent_encoding::{percent_decode_str, utf8_percent_encode}; use serde::{Deserialize, Serialize}; use serde_json::de; @@ -49,17 +49,9 @@ pub struct ConfigOptions { } pub struct Authenticated { - client: postgrest::Postgrest, + client: flow_client::Client, user_config: ConfigOptions, - claims: JwtClaims, -} - -#[derive(Deserialize)] -struct JwtClaims { - /// Unix timestamp in seconds when this token will expire - exp: u64, - /// ID of the user that owns this token - sub: String, + claims: models::authorizations::ControlClaims, } impl App { @@ -73,51 +65,19 @@ impl App { let config: ConfigOptions = serde_json::from_str(&username_str) .context("failed to parse username as a JSON object")?; - #[derive(serde::Deserialize)] - struct RefreshToken { - id: String, - secret: String, - } - let RefreshToken { - id: refresh_token_id, - secret, - } = serde_json::from_slice(&base64::decode(password).context("password is not base64")?) - .context("failed to decode refresh token from password")?; - - tracing::info!(refresh_token_id, "authenticating refresh token"); + let mut client = flow_client::Client::new( + DEFAULT_AGENT_URL.to_owned(), + DEFAULT_PG_PUBLIC_TOKEN.to_string(), + DEFAULT_PG_URL.to_owned(), + None, + Some(String::from_utf8(base64::decode(password)?.to_vec())?.try_into()?), + ); - #[derive(serde::Deserialize)] - struct AccessToken { - access_token: String, - } - let AccessToken { access_token } = self - .anon_client - .rpc( - "generate_access_token", - serde_json::json!({"refresh_token_id": refresh_token_id, "secret": secret}) - .to_string(), - ) - .execute() - .await - .and_then(|r| r.error_for_status()) - .context("generating access token")? - .json() - .await?; - - let authenticated_client = self - .anon_client - .clone() - .insert_header("Authorization", format!("Bearer {access_token}")); - - let claims = base64::decode(access_token.split(".").collect_vec()[1]) - .map_err(anyhow::Error::from) - .and_then(|decoded| { - de::from_slice::(&decoded[..]).map_err(anyhow::Error::from) - }) - .context("Failed to parse access token claims")?; + client.refresh().await?; + let claims = client.claims()?; Ok(Authenticated { - client: authenticated_client, + client, user_config: config, claims, }) diff --git a/crates/dekaf/src/registry.rs b/crates/dekaf/src/registry.rs index e7fc45ecc9..62b5ae74da 100644 --- a/crates/dekaf/src/registry.rs +++ b/crates/dekaf/src/registry.rs @@ -34,33 +34,31 @@ async fn all_subjects( >, ) -> Response { wrap(async move { - // let Authenticated { - // client, - // user_config, - // .. - // } = app.authenticate(auth.username(), auth.password()).await?; - - let r: Vec = vec![]; - Ok(r) - // fetch_all_collection_names(&client) - // .await - // .context("failed to list collections from the control plane") - // .map(|collections| { - // collections - // .into_iter() - // .map(|name| { - // if user_config.strict_topic_names { - // to_downstream_topic_name(TopicName::from(StrBytes::from_string(name))) - // .to_string() - // } else { - // name - // } - // }) - // .flat_map(|collection| { - // vec![format!("{collection}-key"), format!("{collection}-value")] - // }) - // .collect_vec() - // }) + let Authenticated { + client, + user_config, + .. + } = app.authenticate(auth.username(), auth.password()).await?; + + topology::fetch_all_collection_names(&client.pg_client()) + .await + .context("failed to list collections from the control plane") + .map(|collections| { + collections + .into_iter() + .map(|name| { + if user_config.strict_topic_names { + to_downstream_topic_name(TopicName::from(StrBytes::from_string(name))) + .to_string() + } else { + name + } + }) + .flat_map(|collection| { + vec![format!("{collection}-key"), format!("{collection}-value")] + }) + .collect_vec() + }) }) .await } @@ -97,7 +95,7 @@ async fn get_subject_latest( .with_context(|| format!("collection {collection} does not exist"))?; let (key_id, value_id) = collection - .registered_schema_ids(&client) + .registered_schema_ids(&client.pg_client()) .await .context("failed to resolve registered Avro schemas")?; diff --git a/crates/dekaf/src/session.rs b/crates/dekaf/src/session.rs index 0fad7ed299..046621de39 100644 --- a/crates/dekaf/src/session.rs +++ b/crates/dekaf/src/session.rs @@ -1,7 +1,7 @@ use super::{App, Collection, Read}; use crate::{ - from_downstream_topic_name, from_upstream_topic_name, to_downstream_topic_name, - to_upstream_topic_name, Authenticated, ConfigOptions, + from_downstream_topic_name, from_upstream_topic_name, registry::fetch_all_collection_names, + to_downstream_topic_name, to_upstream_topic_name, Authenticated, ConfigOptions, }; use anyhow::Context; use bytes::{BufMut, BytesMut}; @@ -31,7 +31,7 @@ struct PendingRead { pub struct Session { app: Arc, - client: postgrest::Postgrest, + client: Option, reads: HashMap<(TopicName, i32), PendingRead>, /// ID of the authenticated user user_id: Option, @@ -41,10 +41,9 @@ pub struct Session { impl Session { pub fn new(app: Arc, secret: String) -> Self { - let client = app.anon_client.clone(); Self { app, - client, + client: None, reads: HashMap::new(), user_id: None, config: None, @@ -87,9 +86,9 @@ impl Session { user_config, claims, }) => { - self.client = client; + self.client.replace(client); self.config.replace(user_config); - self.user_id.replace(claims.sub); + self.user_id.replace(claims.sub.to_string()); let mut response = messages::SaslAuthenticateResponse::default(); response.session_lifetime_ms = (1000 @@ -144,7 +143,14 @@ impl Session { async fn metadata_all_topics( &mut self, ) -> anyhow::Result> { - let collections = vec![]; //fetch_all_collection_names(&self.client).await?; + let collections = fetch_all_collection_names( + &self + .client + .as_ref() + .ok_or(anyhow::anyhow!("Session not authenticated"))? + .pg_client(), + ) + .await?; tracing::debug!(collections=?ops::DebugJson(&collections), "fetched all collections"); @@ -170,7 +176,10 @@ impl Session { &mut self, requests: Vec, ) -> anyhow::Result> { - let client = &self.client; + let client = &self + .client + .as_ref() + .ok_or(anyhow::anyhow!("Session not authenticated"))?; // Concurrently fetch Collection instances for all requested topics. let collections: anyhow::Result)>> = @@ -247,7 +256,10 @@ impl Session { &mut self, request: messages::ListOffsetsRequest, ) -> anyhow::Result { - let client = &self.client; + let client = &self + .client + .as_ref() + .ok_or(anyhow::anyhow!("Session not authenticated"))?; // Concurrently fetch Collection instances and offsets for all requested topics and partitions. // Map each "topic" into Vec<(Partition Index, Option<(Journal Offset, Timestamp))>. @@ -342,7 +354,11 @@ impl Session { .. } = request; - let client = &self.client; + let client = &self + .client + .as_ref() + .ok_or(anyhow::anyhow!("Session not authenticated"))?; + let timeout = tokio::time::sleep(std::time::Duration::from_millis(max_wait_ms as u64)); let timeout = futures::future::maybe_done(timeout); tokio::pin!(timeout); @@ -370,10 +386,11 @@ impl Session { tracing::debug!(collection = ?&key.0, partition=partition_request.partition, "Partition doesn't exist!"); continue; // Partition doesn't exist. }; - let (key_schema_id, value_schema_id) = - collection.registered_schema_ids(&client).await?; + let (key_schema_id, value_schema_id) = collection + .registered_schema_ids(&client.pg_client()) + .await?; - let read = Read::new( + let read: Read = Read::new( collection.journal_client.clone(), &collection, partition, diff --git a/crates/dekaf/src/topology.rs b/crates/dekaf/src/topology.rs index f1c8127e7d..5f6a47530f 100644 --- a/crates/dekaf/src/topology.rs +++ b/crates/dekaf/src/topology.rs @@ -51,10 +51,11 @@ pub struct Partition { impl Collection { /// Build a Collection by fetching its spec, a authenticated data-plane access token, and its partitions. pub async fn new( - client: &postgrest::Postgrest, + client: &flow_client::Client, collection: &str, ) -> anyhow::Result> { let not_before = uuid::Clock::default(); + let pg_client = client.pg_client(); // Build a journal client and use it to fetch partitions while concurrently // fetching the collection's metadata from the control plane. @@ -64,7 +65,7 @@ impl Collection { Ok((journal_client, partitions)) }; let (spec, client_partitions): (anyhow::Result<_>, anyhow::Result<_>) = - futures::join!(Self::fetch_spec(&client, collection), client_partitions); + futures::join!(Self::fetch_spec(&pg_client, collection), client_partitions); let Some(spec) = spec? else { return Ok(None) }; let (journal_client, partitions) = client_partitions?; @@ -234,43 +235,13 @@ impl Collection { /// Build a journal client by resolving the collections data-plane gateway and an access token. async fn build_journal_client( - client: &postgrest::Postgrest, + client: &flow_client::Client, collection: &str, ) -> anyhow::Result { - let body = serde_json::json!({ - "prefixes": [collection], - }) - .to_string(); - - #[derive(serde::Deserialize)] - struct Auth { - token: String, - gateway_url: String, - } - - let [auth]: [Auth; 1] = client - .rpc("gateway_auth_token", body) - .build() - .send() - .await - .and_then(|r| r.error_for_status()) - .context("requesting data plane gateway auth token")? - .json() - .await?; - - tracing::debug!( - collection, - gateway = auth.gateway_url, - "fetched data-plane token" - ); - - let mut metadata = gazette::Metadata::default(); - metadata.bearer_token(&auth.token)?; - - let router = gazette::Router::new("dekaf"); - let client = journal::Client::new(auth.gateway_url, metadata, router); + let (_, journal_client) = + flow_client::fetch_collection_authorization(client, collection).await?; - Ok(client) + Ok(journal_client) } async fn registered_schema_id( diff --git a/crates/flow-client/Cargo.toml b/crates/flow-client/Cargo.toml new file mode 100644 index 0000000000..15c32c0cb3 --- /dev/null +++ b/crates/flow-client/Cargo.toml @@ -0,0 +1,29 @@ +[package] +name = "flow-client" +version.workspace = true +rust-version.workspace = true +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true + +[dependencies] +gazette = { path = "../gazette" } +models = { path = "../models" } +ops = { path = "../ops" } + +anyhow = { workspace = true } +base64 = { workspace = true } +futures = { workspace = true } +lazy_static = { workspace = true } +page-turner = { workspace = true } +postgrest = { workspace = true } +reqwest = { workspace = true } +serde = { workspace = true } +serde_json = { workspace = true } +time = { workspace = true } +tokio = { workspace = true } +tonic = { workspace = true } +tracing = { workspace = true } +url = { workspace = true } diff --git a/crates/flowctl/src/client.rs b/crates/flow-client/src/client.rs similarity index 59% rename from crates/flowctl/src/client.rs rename to crates/flow-client/src/client.rs index 54f193decf..2c4c431c5e 100644 --- a/crates/flowctl/src/client.rs +++ b/crates/flow-client/src/client.rs @@ -1,3 +1,8 @@ +use crate::{api_exec, parse_jwt_claims}; +use anyhow::Context; +use models::authorizations::ControlClaims; +use url::Url; + /// Client encapsulates sub-clients for various control-plane /// and data-plane services that `flowctl` interacts with. #[derive(Clone)] @@ -6,10 +11,14 @@ pub struct Client { agent_endpoint: url::Url, // HTTP client to use for REST requests. http_client: reqwest::Client, - // PostgREST client. - pg_client: postgrest::Postgrest, + // PostgREST URL. + pg_url: url::Url, + // PostgREST access token. + pg_token: String, // User's access token, if authenticated. user_access_token: Option, + // User's refresh token, if authenticated. + user_refresh_token: Option, // Base shard client which is cloned to build token-specific clients. shard_client: gazette::shard::Client, // Base journal client which is cloned to build token-specific clients. @@ -18,16 +27,13 @@ pub struct Client { impl Client { /// Build a new Client from the Config. - pub fn new(config: &crate::config::Config) -> Self { - let user_access_token = config.user_access_token.clone(); - - let mut pg_client = postgrest::Postgrest::new(config.get_pg_url().as_str()) - .insert_header("apikey", config.get_pg_public_token()); - - if let Some(token) = user_access_token.as_ref() { - pg_client = pg_client.insert_header("Authorization", &format!("Bearer {token}")); - } - + pub fn new( + agent_endpoint: Url, + pg_token: String, + pg_url: Url, + access_token: Option, + refresh_token: Option, + ) -> Self { // Build journal and shard clients with an empty default service address. // We'll use their with_endpoint_and_metadata() routines to cheaply clone // new clients using dynamic addresses and access tokens, while re-using @@ -46,21 +52,109 @@ impl Client { ); Self { - agent_endpoint: config.get_agent_url().clone(), + agent_endpoint, http_client: reqwest::Client::new(), + pg_token, + pg_url, journal_client, - pg_client, shard_client, - user_access_token, + user_access_token: access_token, + user_refresh_token: refresh_token, } } + pub async fn refresh(&mut self) -> anyhow::Result<()> { + // Clear expired or soon-to-expire access token + if let Some(_) = &self.user_access_token { + let claims = self.claims()?; + + let now = time::OffsetDateTime::now_utc(); + let exp = time::OffsetDateTime::from_unix_timestamp(claims.exp as i64).unwrap(); + + // Refresh access tokens with plenty of time to spare if we have a + // refresh token. If not, allow refreshing right until the token expires + match ((now - exp).whole_seconds(), &self.user_refresh_token) { + (exp_seconds, Some(_)) if exp_seconds < 60 => self.user_access_token = None, + (exp_seconds, None) if exp_seconds <= 0 => self.user_access_token = None, + _ => {} + } + } + + if self.user_access_token.is_some() && self.user_refresh_token.is_some() { + // Authorization is current: nothing to do. + Ok(()) + } else if self.user_access_token.is_some() { + // We have an access token but no refresh token. Create one. + let refresh_token = api_exec::( + self.rpc( + "create_refresh_token", + serde_json::json!({"multi_use": true, "valid_for": "90d", "detail": "Created by flowctl"}) + .to_string(), + ), + ) + .await?; + + self.user_refresh_token = Some(refresh_token); + + tracing::info!("created new refresh token"); + Ok(()) + } else if let Some(RefreshToken { id, secret }) = &self.user_refresh_token { + // We have a refresh token but no access token. Generate one. + + #[derive(serde::Deserialize)] + struct Response { + access_token: String, + refresh_token: Option, // Set iff the token was single-use. + } + let Response { + access_token, + refresh_token: next_refresh_token, + } = api_exec::(self.rpc( + "generate_access_token", + serde_json::json!({"refresh_token_id": id, "secret": secret}).to_string(), + )) + .await + .context("failed to obtain access token")?; + + if next_refresh_token.is_some() { + self.user_refresh_token = next_refresh_token; + } + + self.user_access_token = Some(access_token); + + tracing::info!("generated a new access token"); + Ok(()) + } else { + anyhow::bail!("Client not authenticated"); + } + } + + pub fn pg_client(&self) -> postgrest::Postgrest { + let pg_client = postgrest::Postgrest::new(self.pg_url.as_str()) + .insert_header("apikey", self.pg_token.as_str()); + + if let Some(token) = &self.user_access_token { + return pg_client.insert_header("Authorization", &format!("Bearer {token}")); + } + + pg_client + } + + pub fn claims(&self) -> anyhow::Result { + parse_jwt_claims( + self.user_access_token + .as_ref() + .ok_or(anyhow::anyhow!("Client is not authenticated"))? + .as_str(), + ) + } + pub fn from(&self, table: &str) -> postgrest::Builder { - self.pg_client.from(table) + self.pg_client().from(table) } pub fn rpc(&self, function: &str, params: String) -> postgrest::Builder { - self.pg_client.rpc(function, params) + self.pg_client().rpc(function, params) } pub fn is_authenticated(&self) -> bool { @@ -97,6 +191,20 @@ impl Client { } } +#[derive(Debug, Clone, serde::Deserialize, serde::Serialize)] +pub struct RefreshToken { + pub id: models::Id, + pub secret: String, +} + +impl TryFrom for RefreshToken { + type Error = anyhow::Error; + + fn try_from(value: String) -> Result { + parse_jwt_claims(value.as_str()) + } +} + #[tracing::instrument(skip(client), err)] pub async fn fetch_task_authorization( client: &Client, diff --git a/crates/flow-client/src/lib.rs b/crates/flow-client/src/lib.rs new file mode 100644 index 0000000000..90b492072f --- /dev/null +++ b/crates/flow-client/src/lib.rs @@ -0,0 +1,65 @@ +use anyhow::Context; + +pub mod client; +pub use client::{fetch_collection_authorization, fetch_task_authorization, Client}; + +pub mod pagination; + +// api_exec runs a PostgREST request, debug-logs its request, and turns non-success status into an anyhow::Error. +pub async fn api_exec(b: postgrest::Builder) -> anyhow::Result +where + for<'de> T: serde::Deserialize<'de>, +{ + let req = b.build(); + tracing::debug!(?req, "built request to execute"); + + let resp = req.send().await?; + let status = resp.status(); + + if status.is_success() { + let body: models::RawValue = resp.json().await?; + tracing::trace!(body = ?::ops::DebugJson(&body), status = %status, "got successful response"); + let t: T = serde_json::from_str(body.get()).context("deserializing response body")?; + Ok(t) + } else { + let body = resp.text().await?; + anyhow::bail!("{status}: {body}"); + } +} + +/// Execute a [`postgrest::Builder`] request returning multiple rows. Unlike [`api_exec`] +/// which is limited to however many rows Postgrest is configured to return in a single response, +/// this will issue as many paginated requests as necessary to fetch every row. +pub async fn api_exec_paginated(b: postgrest::Builder) -> anyhow::Result> +where + T: serde::de::DeserializeOwned + Send + Sync + 'static, +{ + use futures::TryStreamExt; + + let pages = pagination::into_items(b).try_collect().await?; + + Ok(pages) +} + +pub fn parse_jwt_claims(token: &str) -> anyhow::Result { + let claims = token + .split('.') + .nth(1) + .ok_or_else(|| anyhow::anyhow!("malformed token"))?; + let claims = base64::decode_config(claims, base64::URL_SAFE_NO_PAD)?; + anyhow::Result::Ok(serde_json::from_slice(&claims)?) +} + +lazy_static::lazy_static! { + pub static ref DEFAULT_AGENT_URL: url::Url = url::Url::parse("https://agent-api-1084703453822.us-central1.run.app").unwrap(); + pub static ref DEFAULT_DASHBOARD_URL: url::Url = url::Url::parse("https://dashboard.estuary.dev/").unwrap(); + pub static ref DEFAULT_PG_URL: url::Url = url::Url::parse("https://eyrcnmuzzyriypdajwdk.supabase.co/rest/v1").unwrap(); + + // Used only when profile is "local". + pub static ref LOCAL_AGENT_URL: url::Url = url::Url::parse("http://localhost:8675/").unwrap(); + pub static ref LOCAL_DASHBOARD_URL: url::Url = url::Url::parse("http://localhost:3000/").unwrap(); + pub static ref LOCAL_PG_URL: url::Url = url::Url::parse("http://localhost:5431/rest/v1").unwrap(); +} + +pub const DEFAULT_PG_PUBLIC_TOKEN: &str = "eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJzdXBhYmFzZSIsInJlZiI6ImV5cmNubXV6enlyaXlwZGFqd2RrIiwicm9sZSI6ImFub24iLCJpYXQiOjE2NDg3NTA1NzksImV4cCI6MTk2NDMyNjU3OX0.y1OyXD3-DYMz10eGxzo1eeamVMMUwIIeOoMryTRAoco"; +pub const LOCAL_PG_PUBLIC_TOKEN: &str = "eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJzdXBhYmFzZS1kZW1vIiwicm9sZSI6ImFub24iLCJleHAiOjE5ODM4MTI5OTZ9.CRXP1A7WOeoJeXxjNni43kdQwgnWNReilDMblYTn_I0"; diff --git a/crates/flowctl/src/pagination.rs b/crates/flow-client/src/pagination.rs similarity index 100% rename from crates/flowctl/src/pagination.rs rename to crates/flow-client/src/pagination.rs diff --git a/crates/flowctl/Cargo.toml b/crates/flowctl/Cargo.toml index 7e70e0cbf5..1976536e63 100644 --- a/crates/flowctl/Cargo.toml +++ b/crates/flowctl/Cargo.toml @@ -18,6 +18,7 @@ connector-init = { path = "../connector-init" } coroutines = { path = "../coroutines" } doc = { path = "../doc" } extractors = { path = "../extractors" } +flow-client = { path = "../flow-client" } gazette = { path = "../gazette" } json = { path = "../json" } labels = { path = "../labels" } diff --git a/crates/flowctl/src/collection/mod.rs b/crates/flowctl/src/collection/mod.rs index 6088a947f3..590621cc64 100644 --- a/crates/flowctl/src/collection/mod.rs +++ b/crates/flowctl/src/collection/mod.rs @@ -185,7 +185,7 @@ async fn do_list_fragments( }: &ListFragmentsArgs, ) -> Result<(), anyhow::Error> { let (journal_name_prefix, client) = - crate::client::fetch_collection_authorization(&ctx.client, &selector.collection).await?; + flow_client::fetch_collection_authorization(&ctx.client, &selector.collection).await?; let list_resp = client .list(broker::ListRequest { @@ -225,7 +225,7 @@ async fn do_list_journals( selector: &CollectionJournalSelector, ) -> Result<(), anyhow::Error> { let (journal_name_prefix, client) = - crate::client::fetch_collection_authorization(&ctx.client, &selector.collection).await?; + flow_client::fetch_collection_authorization(&ctx.client, &selector.collection).await?; let list_resp = client .list(broker::ListRequest { diff --git a/crates/flowctl/src/collection/read/mod.rs b/crates/flowctl/src/collection/read/mod.rs index 81171247bd..394121ae38 100644 --- a/crates/flowctl/src/collection/read/mod.rs +++ b/crates/flowctl/src/collection/read/mod.rs @@ -63,7 +63,7 @@ pub async fn read_collection( } let (journal_name_prefix, journal_client) = - crate::client::fetch_collection_authorization(&ctx.client, &selector.collection).await?; + flow_client::fetch_collection_authorization(&ctx.client, &selector.collection).await?; let list_resp = journal_client .list(broker::ListRequest { diff --git a/crates/flowctl/src/config.rs b/crates/flowctl/src/config.rs index c03d4a6c9f..9cb7660581 100644 --- a/crates/flowctl/src/config.rs +++ b/crates/flowctl/src/config.rs @@ -1,6 +1,11 @@ use anyhow::Context; use std::path::PathBuf; +use flow_client::{ + client::RefreshToken, DEFAULT_AGENT_URL, DEFAULT_DASHBOARD_URL, DEFAULT_PG_PUBLIC_TOKEN, + DEFAULT_PG_URL, LOCAL_AGENT_URL, LOCAL_DASHBOARD_URL, LOCAL_PG_PUBLIC_TOKEN, LOCAL_PG_URL, +}; + /// Configuration of `flowctl`. /// /// We generally keep this minimal and prefer to use built-in default @@ -40,12 +45,6 @@ pub struct Config { api: Option, } -#[derive(Debug, serde::Deserialize, serde::Serialize)] -pub struct RefreshToken { - pub id: models::Id, - pub secret: String, -} - #[derive(Debug, serde::Deserialize)] struct DeprecatedAPISection { #[allow(dead_code)] @@ -182,6 +181,15 @@ impl Config { Ok(()) } + pub fn build_client(&self) -> flow_client::Client { + flow_client::Client::new( + self.get_agent_url().clone(), + self.get_pg_public_token().to_string(), + self.get_pg_url().clone(), + self.user_access_token.clone(), + self.user_refresh_token.clone(), + ) + } fn config_dir() -> anyhow::Result { let path = dirs::config_dir() @@ -196,19 +204,5 @@ impl Config { } } -lazy_static::lazy_static! { - static ref DEFAULT_AGENT_URL: url::Url = url::Url::parse("https://agent-api-1084703453822.us-central1.run.app").unwrap(); - static ref DEFAULT_DASHBOARD_URL: url::Url = url::Url::parse("https://dashboard.estuary.dev/").unwrap(); - static ref DEFAULT_PG_URL: url::Url = url::Url::parse("https://eyrcnmuzzyriypdajwdk.supabase.co/rest/v1").unwrap(); - - // Used only when profile is "local". - static ref LOCAL_AGENT_URL: url::Url = url::Url::parse("http://localhost:8675/").unwrap(); - static ref LOCAL_DASHBOARD_URL: url::Url = url::Url::parse("http://localhost:3000/").unwrap(); - static ref LOCAL_PG_URL: url::Url = url::Url::parse("http://localhost:5431/rest/v1").unwrap(); -} - -const DEFAULT_PG_PUBLIC_TOKEN: &str = "eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJzdXBhYmFzZSIsInJlZiI6ImV5cmNubXV6enlyaXlwZGFqd2RrIiwicm9sZSI6ImFub24iLCJpYXQiOjE2NDg3NTA1NzksImV4cCI6MTk2NDMyNjU3OX0.y1OyXD3-DYMz10eGxzo1eeamVMMUwIIeOoMryTRAoco"; -const LOCAL_PG_PUBLIC_TOKEN: &str = "eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJzdXBhYmFzZS1kZW1vIiwicm9sZSI6ImFub24iLCJleHAiOjE5ODM4MTI5OTZ9.CRXP1A7WOeoJeXxjNni43kdQwgnWNReilDMblYTn_I0"; - // Environment variable which is inspected for a base64-encoded refresh token. const FLOW_AUTH_TOKEN: &str = "FLOW_AUTH_TOKEN"; diff --git a/crates/flowctl/src/draft/develop.rs b/crates/flowctl/src/draft/develop.rs index 03c46d2d47..6510ac9228 100644 --- a/crates/flowctl/src/draft/develop.rs +++ b/crates/flowctl/src/draft/develop.rs @@ -1,4 +1,5 @@ -use crate::{api_exec_paginated, catalog, local_specs}; +use crate::{catalog, local_specs}; +use flow_client::api_exec_paginated; use models::{CatalogType, RawValue}; use serde::{Deserialize, Serialize}; diff --git a/crates/flowctl/src/generate/mod.rs b/crates/flowctl/src/generate/mod.rs index 61b90fd9f8..d0485ad407 100644 --- a/crates/flowctl/src/generate/mod.rs +++ b/crates/flowctl/src/generate/mod.rs @@ -3,8 +3,6 @@ use anyhow::Context; use futures::{FutureExt, StreamExt}; use itertools::Itertools; use proto_flow::{capture, derive, flow, materialize}; -use std::str::FromStr; -use url::Url; #[derive(Debug, clap::Args)] #[clap(rename_all = "kebab-case")] diff --git a/crates/flowctl/src/lib.rs b/crates/flowctl/src/lib.rs index 59649aaf11..9ca8ee67e5 100644 --- a/crates/flowctl/src/lib.rs +++ b/crates/flowctl/src/lib.rs @@ -5,7 +5,6 @@ use clap::Parser; mod auth; mod catalog; -mod client; mod collection; mod config; mod draft; @@ -13,12 +12,12 @@ mod generate; mod local_specs; mod ops; mod output; -pub mod pagination; mod poll; mod preview; mod raw; -use client::Client; +pub(crate) use flow_client::client::Client; +pub(crate) use flow_client::{api_exec, api_exec_paginated, parse_jwt_claims}; use output::{Output, OutputType}; use poll::poll_while_queued; @@ -149,52 +148,16 @@ impl Cli { } } - if config.user_access_token.is_some() && config.user_refresh_token.is_some() { - // Authorization is current: nothing to do. - } else if config.user_access_token.is_some() { - // We have an access token but no refresh token. Create one. - let refresh_token = api_exec::( - Client::new(&config).rpc( - "create_refresh_token", - serde_json::json!({"multi_use": true, "valid_for": "90d", "detail": "Created by flowctl"}) - .to_string(), - ), - ) - .await?; + let mut client: flow_client::Client = config.build_client(); - config.user_refresh_token = Some(refresh_token); - - tracing::info!("created new refresh token"); - } else if let Some(config::RefreshToken { id, secret }) = &config.user_refresh_token { - // We have a refresh token but no access token. Generate one. - - #[derive(serde::Deserialize)] - struct Response { - access_token: String, - refresh_token: Option, // Set iff the token was single-use. - } - let Response { - access_token, - refresh_token: next_refresh_token, - } = api_exec::(Client::new(&config).rpc( - "generate_access_token", - serde_json::json!({"refresh_token_id": id, "secret": secret}).to_string(), - )) - .await - .context("failed to obtain access token")?; - - if next_refresh_token.is_some() { - config.user_refresh_token = next_refresh_token; - } - config.user_access_token = Some(access_token); - - tracing::info!("generated a new access token"); + if config.user_access_token.is_some() || config.user_refresh_token.is_some() { + client.refresh().await?; } else { tracing::warn!("You are not authenticated. Run `auth login` to login to Flow."); } let mut context = CliContext { - client: Client::new(&config), + client, config, output, }; @@ -216,42 +179,6 @@ impl Cli { } } -// api_exec runs a PostgREST request, debug-logs its request, and turns non-success status into an anyhow::Error. -async fn api_exec(b: postgrest::Builder) -> anyhow::Result -where - for<'de> T: serde::Deserialize<'de>, -{ - let req = b.build(); - tracing::debug!(?req, "built request to execute"); - - let resp = req.send().await?; - let status = resp.status(); - - if status.is_success() { - let body: models::RawValue = resp.json().await?; - tracing::trace!(body = ?::ops::DebugJson(&body), status = %status, "got successful response"); - let t: T = serde_json::from_str(body.get()).context("deserializing response body")?; - Ok(t) - } else { - let body = resp.text().await?; - anyhow::bail!("{status}: {body}"); - } -} - -/// Execute a [`postgrest::Builder`] request returning multiple rows. Unlike [`api_exec`] -/// which is limited to however many rows Postgrest is configured to return in a single response, -/// this will issue as many paginated requests as necessary to fetch every row. -async fn api_exec_paginated(b: postgrest::Builder) -> anyhow::Result> -where - T: serde::de::DeserializeOwned + Send + Sync + 'static, -{ - use futures::TryStreamExt; - - let pages = pagination::into_items(b).try_collect().await?; - - Ok(pages) -} - // new_table builds a comfy_table with UTF8 styling. fn new_table(headers: Vec<&str>) -> comfy_table::Table { let mut table = comfy_table::Table::new(); @@ -292,12 +219,3 @@ fn format_user(email: Option, full_name: Option, id: Option(token: &str) -> anyhow::Result { - let claims = token - .split('.') - .nth(1) - .ok_or_else(|| anyhow::anyhow!("malformed token"))?; - let claims = base64::decode_config(claims, base64::URL_SAFE_NO_PAD)?; - anyhow::Result::Ok(serde_json::from_slice(&claims)?) -} diff --git a/crates/flowctl/src/ops.rs b/crates/flowctl/src/ops.rs index 7649323d68..9b88a7dcff 100644 --- a/crates/flowctl/src/ops.rs +++ b/crates/flowctl/src/ops.rs @@ -42,7 +42,7 @@ pub async fn read_task_ops_journal( bounds: &ReadBounds, ) -> anyhow::Result<()> { let (_shard_id_prefix, ops_logs_journal, ops_stats_journal, _shard_client, journal_client) = - crate::client::fetch_task_authorization(client, task_name).await?; + flow_client::fetch_task_authorization(client, task_name).await?; let journal_name = match collection { OpsCollection::Logs => ops_logs_journal, diff --git a/crates/flowctl/src/preview/journal_reader.rs b/crates/flowctl/src/preview/journal_reader.rs index 2181bf2192..11fbd0190c 100644 --- a/crates/flowctl/src/preview/journal_reader.rs +++ b/crates/flowctl/src/preview/journal_reader.rs @@ -40,7 +40,7 @@ impl Reader { let reader = coroutines::try_coroutine(move |mut co| async move { // Concurrently fetch authorizations for all sourced collections. let sources = futures::future::try_join_all(sources.iter().map(|source| { - crate::client::fetch_collection_authorization(&self.client, &source.collection) + flow_client::fetch_collection_authorization(&self.client, &source.collection) .map_ok(move |(_journal_name_prefix, client)| (source, client)) })) .await?; diff --git a/crates/sources/src/indirect.rs b/crates/sources/src/indirect.rs index 17ba75fd19..47fd94e9a7 100644 --- a/crates/sources/src/indirect.rs +++ b/crates/sources/src/indirect.rs @@ -1,6 +1,5 @@ use super::Format; use crate::Scope; -use models::RawValue; use proto_flow::flow::ContentType; use std::collections::BTreeMap; diff --git a/crates/validation/src/materialization.rs b/crates/validation/src/materialization.rs index 8151cd9d05..60e44c7c68 100644 --- a/crates/validation/src/materialization.rs +++ b/crates/validation/src/materialization.rs @@ -4,7 +4,6 @@ use super::{ }; use itertools::Itertools; use proto_flow::{flow, materialize, ops::log::Level as LogLevel}; -use serde::Deserialize; use std::collections::{BTreeMap, HashMap}; use tables::EitherOrBoth as EOB; From 0e37c22740e634285aff944d023bdf82cf42e673 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Mon, 30 Sep 2024 00:05:31 -0400 Subject: [PATCH 09/26] dekaf: Clean up and stub out location where bearer token will be checked --- Cargo.lock | 1 + crates/dekaf/Cargo.toml | 3 +-- crates/dekaf/src/connector.rs | 9 ++++++-- crates/dekaf/src/lib.rs | 39 ++++++++++++++++++++++++----------- crates/dekaf/src/registry.rs | 6 +++--- crates/dekaf/src/session.rs | 15 +++++++------- 6 files changed, 47 insertions(+), 26 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2823e283a9..8278c6449d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1852,6 +1852,7 @@ dependencies = [ "rustls 0.23.10", "rustls-native-certs", "rustls-pemfile 2.1.3", + "schemars", "serde", "serde_json", "simd-doc", diff --git a/crates/dekaf/Cargo.toml b/crates/dekaf/Cargo.toml index 046c6abe4a..e9acc91a4c 100644 --- a/crates/dekaf/Cargo.toml +++ b/crates/dekaf/Cargo.toml @@ -29,8 +29,6 @@ axum-extra = { workspace = true } axum-server = { workspace = true } base64 = { workspace = true } bytes = { workspace = true } -# TODO(jshearer): Upgrade every other usage of clap to v4 and change this to: -# clap = { workspace = true } clap = { workspace = true } crypto-common = { workspace = true } deadpool = { workspace = true } @@ -52,6 +50,7 @@ rsasl = { workspace = true } rustls = { workspace = true } rustls-native-certs = { workspace = true } rustls-pemfile = { workspace = true } +schemars = { workspace = true } serde = { workspace = true } serde_json = { workspace = true } socket2 = { workspace = true } diff --git a/crates/dekaf/src/connector.rs b/crates/dekaf/src/connector.rs index 44d435aedb..dfce7911c6 100644 --- a/crates/dekaf/src/connector.rs +++ b/crates/dekaf/src/connector.rs @@ -1,18 +1,23 @@ use anyhow::{bail, Context}; use proto_flow::materialize; +use schemars::JsonSchema; use serde::{Deserialize, Serialize}; use std::collections::BTreeMap; /// Configures the behavior of a whole dekaf task -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, JsonSchema)] pub struct DekafConfig { /// Whether or not to expose topic names in a strictly Kafka-compliant format /// for systems that require it. Off by default. pub strict_topic_names: bool, + /// The password that will authenticate Kafka consumers to this task. + // TODO(jshearer): Uncomment when schemars 1.0 is out and we upgrade + // #[schemars(extend("secret" = true))] + pub token: String, } /// Configures a particular binding in a Dekaf-type materialization -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, JsonSchema)] pub struct DekafResourceConfig { /// The exposed name of the topic that maps to this binding. This /// will be exposed through the Kafka metadata/discovery APIs. diff --git a/crates/dekaf/src/lib.rs b/crates/dekaf/src/lib.rs index 8d6e40afe2..309fb11727 100644 --- a/crates/dekaf/src/lib.rs +++ b/crates/dekaf/src/lib.rs @@ -23,6 +23,7 @@ mod api_client; pub use api_client::KafkaApiClient; use aes_siv::{aead::Aead, Aes256SivAead, KeyInit, KeySizeUser}; +use connector::DekafConfig; use flow_client::{DEFAULT_AGENT_URL, DEFAULT_PG_PUBLIC_TOKEN, DEFAULT_PG_URL}; use percent_encoding::{percent_decode_str, utf8_percent_encode}; use serde::{Deserialize, Serialize}; @@ -43,27 +44,25 @@ pub struct App { } #[derive(Debug, Clone, Serialize, Deserialize)] -pub struct ConfigOptions { +pub struct DeprecatedConfigOptions { #[serde(default = "bool::")] pub strict_topic_names: bool, } pub struct Authenticated { client: flow_client::Client, - user_config: ConfigOptions, + task_config: DekafConfig, claims: models::authorizations::ControlClaims, } impl App { #[tracing::instrument(level = "info", err(Debug, level = "warn"), skip(self, password))] async fn authenticate(&self, username: &str, password: &str) -> anyhow::Result { - let username_str = if username.contains("{") { - username.to_string() + let username = if let Ok(decoded) = decode_safe_name(username.to_string()) { + decoded } else { - decode_safe_name(username.to_string()).context("failed to decode username")? + username.to_string() }; - let config: ConfigOptions = serde_json::from_str(&username_str) - .context("failed to parse username as a JSON object")?; let mut client = flow_client::Client::new( DEFAULT_AGENT_URL.to_owned(), @@ -76,11 +75,27 @@ impl App { client.refresh().await?; let claims = client.claims()?; - Ok(Authenticated { - client, - user_config: config, - claims, - }) + if models::Materialization::regex().is_match(username.as_ref()) { + Ok(Authenticated { + client, + task_config: todo!("Fetch and unseal task config"), + claims, + }) + } else if username.contains("{") { + let config: DeprecatedConfigOptions = serde_json::from_str(&username) + .context("failed to parse username as a JSON object")?; + + Ok(Authenticated { + client, + task_config: DekafConfig { + strict_topic_names: config.strict_topic_names, + token: "".to_string(), + }, + claims, + }) + } else { + anyhow::bail!("Invalid username or password") + } } } diff --git a/crates/dekaf/src/registry.rs b/crates/dekaf/src/registry.rs index 62b5ae74da..45840e0897 100644 --- a/crates/dekaf/src/registry.rs +++ b/crates/dekaf/src/registry.rs @@ -1,5 +1,5 @@ use super::App; -use crate::{from_downstream_topic_name, to_downstream_topic_name, Authenticated}; +use crate::{from_downstream_topic_name, to_downstream_topic_name, topology, Authenticated}; use anyhow::Context; use axum::response::{IntoResponse, Response}; use axum_extra::headers; @@ -36,7 +36,7 @@ async fn all_subjects( wrap(async move { let Authenticated { client, - user_config, + task_config, .. } = app.authenticate(auth.username(), auth.password()).await?; @@ -47,7 +47,7 @@ async fn all_subjects( collections .into_iter() .map(|name| { - if user_config.strict_topic_names { + if task_config.strict_topic_names { to_downstream_topic_name(TopicName::from(StrBytes::from_string(name))) .to_string() } else { diff --git a/crates/dekaf/src/session.rs b/crates/dekaf/src/session.rs index 046621de39..3032d4474d 100644 --- a/crates/dekaf/src/session.rs +++ b/crates/dekaf/src/session.rs @@ -1,7 +1,8 @@ use super::{App, Collection, Read}; use crate::{ - from_downstream_topic_name, from_upstream_topic_name, registry::fetch_all_collection_names, - to_downstream_topic_name, to_upstream_topic_name, Authenticated, ConfigOptions, + connector::DekafConfig, from_downstream_topic_name, from_upstream_topic_name, + to_downstream_topic_name, to_upstream_topic_name, topology::fetch_all_collection_names, + Authenticated, }; use anyhow::Context; use bytes::{BufMut, BytesMut}; @@ -35,7 +36,7 @@ pub struct Session { reads: HashMap<(TopicName, i32), PendingRead>, /// ID of the authenticated user user_id: Option, - config: Option, + task_config: Option, secret: String, } @@ -46,7 +47,7 @@ impl Session { client: None, reads: HashMap::new(), user_id: None, - config: None, + task_config: None, secret, } } @@ -83,11 +84,11 @@ impl Session { let response = match self.app.authenticate(authcid, password).await { Ok(Authenticated { client, - user_config, + task_config, claims, }) => { self.client.replace(client); - self.config.replace(user_config); + self.task_config.replace(task_config); self.user_id.replace(claims.sub.to_string()); let mut response = messages::SaslAuthenticateResponse::default(); @@ -1043,7 +1044,7 @@ impl Session { fn encode_topic_name(&self, name: String) -> TopicName { if self - .config + .task_config .as_ref() .expect("should have config already") .strict_topic_names From ad46121547ef7fee94c716eb7c615b57e4c47e89 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Mon, 30 Sep 2024 00:37:39 -0400 Subject: [PATCH 10/26] test: Update Dekaf tests --- ...ources__scenarios__test__dekaf_good-2.snap | 55 ++ ...ources__scenarios__test__dekaf_good-3.snap | 75 +++ ...ces__scenarios__test__dekaf_invalid-2.snap | 6 + ...ces__scenarios__test__dekaf_invalid-3.snap | 6 + crates/validation/tests/scenario_tests.rs | 91 ++- ...dekaf_materialization_indirect_config.snap | 516 +-------------- ...__dekaf_materialization_inline_config.snap | 494 +------------- ..._tests__dekaf_materialization_invalid.snap | 6 +- ...ts__dekaf_materialization_nonexistent.snap | 616 +----------------- 9 files changed, 270 insertions(+), 1595 deletions(-) create mode 100644 crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good-2.snap create mode 100644 crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good-3.snap create mode 100644 crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid-2.snap create mode 100644 crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid-3.snap diff --git a/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good-2.snap b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good-2.snap new file mode 100644 index 0000000000..3a9503ed78 --- /dev/null +++ b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good-2.snap @@ -0,0 +1,55 @@ +--- +source: crates/sources/src/scenarios/mod.rs +assertion_line: 63 +expression: tables +--- +DraftCatalog { + materializations: [ + DraftMaterialization { + materialization: materialization/dekaf/indirect, + scope: test://example/catalog.yaml#/materializations/materialization~1dekaf~1indirect, + expect_pub_id: NULL, + model: { + "endpoint": { + "dekaf": { + "variant": "foo", + "config": {"strict_topic_names":false,"token":"foo_token"} + } + }, + "bindings": [ + { + "resource": {"topic":"pizza"}, + "source": "some/source/materialization", + "fields": { + "recommended": true + } + } + ] + }, + is_touch: 0, + }, + DraftMaterialization { + materialization: materialization/dekaf/inline, + scope: test://example/catalog.yaml#/materializations/materialization~1dekaf~1inline, + expect_pub_id: NULL, + model: { + "endpoint": { + "dekaf": { + "variant": "foo", + "config": {"strict_topic_names":false} + } + }, + "bindings": [ + { + "resource": {"topic_name":"foo"}, + "source": "some/source/collection", + "fields": { + "recommended": true + } + } + ] + }, + is_touch: 0, + }, + ], +} diff --git a/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good-3.snap b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good-3.snap new file mode 100644 index 0000000000..9bd9c833a2 --- /dev/null +++ b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good-3.snap @@ -0,0 +1,75 @@ +--- +source: crates/sources/src/scenarios/mod.rs +assertion_line: 63 +expression: tables +--- +DraftCatalog { + materializations: [ + DraftMaterialization { + materialization: materialization/dekaf/indirect, + scope: test://example/catalog.yaml#/materializations/materialization~1dekaf~1indirect, + expect_pub_id: NULL, + model: { + "endpoint": { + "dekaf": { + "variant": "foo", + "config": "indirect.config.yaml" + } + }, + "bindings": [ + { + "resource": {"topic":"pizza"}, + "source": "some/source/materialization", + "fields": { + "recommended": true + } + } + ] + }, + is_touch: 0, + }, + DraftMaterialization { + materialization: materialization/dekaf/inline, + scope: test://example/catalog.yaml#/materializations/materialization~1dekaf~1inline, + expect_pub_id: NULL, + model: { + "endpoint": { + "dekaf": { + "variant": "foo", + "config": {"strict_topic_names":false} + } + }, + "bindings": [ + { + "resource": {"topic_name":"foo"}, + "source": "some/source/collection", + "fields": { + "recommended": true + } + } + ] + }, + is_touch: 0, + }, + ], + imports: [ + Import { + scope: test://example/catalog.yaml#/materializations/materialization~1dekaf~1indirect/endpoint/dekaf/config, + to_resource: test://example/indirect.config.yaml, + }, + ], + resources: [ + Resource { + resource: test://example/catalog.yaml, + content_type: "CATALOG", + content: ".. binary ..", + content_dom: {"materializations":{"materialization/dekaf/indirect":{"endpoint":{"dekaf":{"variant":"foo","config":"indirect.config.yaml"}},"bindings":[{"resource":{"topic":"pizza"},"source":"some/source/materialization","fields":{"recommended":true}}]},"materialization/dekaf/inline":{"endpoint":{"dekaf":{"variant":"foo","config":{"strict_topic_names":false}}},"bindings":[{"resource":{"topic_name":"foo"},"source":"some/source/collection","fields":{"recommended":true}}]}}}, + }, + Resource { + resource: test://example/indirect.config.yaml, + content_type: "CONFIG", + content: ".. binary ..", + content_dom: {"strict_topic_names":false,"token":"foo_token"}, + }, + ], +} diff --git a/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid-2.snap b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid-2.snap new file mode 100644 index 0000000000..e59b1b7491 --- /dev/null +++ b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid-2.snap @@ -0,0 +1,6 @@ +--- +source: crates/sources/src/scenarios/mod.rs +assertion_line: 63 +expression: tables +--- +DraftCatalog diff --git a/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid-3.snap b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid-3.snap new file mode 100644 index 0000000000..e59b1b7491 --- /dev/null +++ b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid-3.snap @@ -0,0 +1,6 @@ +--- +source: crates/sources/src/scenarios/mod.rs +assertion_line: 63 +expression: tables +--- +DraftCatalog diff --git a/crates/validation/tests/scenario_tests.rs b/crates/validation/tests/scenario_tests.rs index 5836fd7d6a..7689508e81 100644 --- a/crates/validation/tests/scenario_tests.rs +++ b/crates/validation/tests/scenario_tests.rs @@ -12,22 +12,20 @@ fn test_golden_all_visits() { fn test_dekaf_materialization_inline_config() { let fixture = r##" test://example/catalog.yaml: - collections: - testing/schema_with_properties: - schema: - type: object - properties: - id: { type: string } - required: [id] - key: [/id] materializations: - testing/test_dekaf: + good: endpoint: - dekaf: {} - bindings: - - source: testing/schema_with_properties - resource: {} + dekaf: &config + variant: foo + config: + strict_topic_names: false + bindings: [] driver: + materializations: + good: &connector + connectorType: DEKAF + config: *config + bindings: [] dataPlanes: "1d:1d:1d:1d:1d:1d:1d:1d": default: true @@ -41,24 +39,25 @@ driver: #[test] fn test_dekaf_materialization_indirect_config() { let fixture = r##" -test://example/dekaf.yaml: {} test://example/catalog.yaml: - collections: - testing/schema_with_properties: - schema: - type: object - properties: - id: { type: string } - required: [id] - key: [/id] materializations: - testing/test_dekaf: + good: endpoint: - dekaf: example/dekaf.yaml - bindings: - - source: testing/schema_with_properties - resource: {} + dekaf: &config + variant: foo + config: test://example/dekaf.yaml + bindings: [] +test://example/dekaf.yaml: + strict_topic_names: false driver: + materializations: + good: &connector + connectorType: DEKAF + config: + variant: foo + config: + strict_topic_names: false + bindings: [] dataPlanes: "1d:1d:1d:1d:1d:1d:1d:1d": default: true @@ -72,7 +71,6 @@ driver: #[test] fn test_dekaf_materialization_invalid() { let fixture = r##" -test://example/dekaf.yaml: {} test://example/catalog.yaml: collections: testing/schema_with_properties: @@ -83,13 +81,16 @@ test://example/catalog.yaml: required: [id] key: [/id] materializations: - testing/test_dekaf: - endpoint: + bad: + endpoint: &config dekaf: false - bindings: - - source: testing/schema_with_properties - resource: {} + bindings: [] driver: + materializations: + bad: &connector + connectorType: DEKAF + config: *config + bindings: [] dataPlanes: "1d:1d:1d:1d:1d:1d:1d:1d": default: true @@ -103,24 +104,20 @@ driver: #[test] fn test_dekaf_materialization_nonexistent() { let fixture = r##" -test://example/dekaf.yaml: {} test://example/catalog.yaml: - collections: - testing/schema_with_properties: - schema: - type: object - properties: - id: { type: string } - required: [id] - key: [/id] materializations: - testing/test_dekaf: + bad: endpoint: - dekaf: foo/bar - bindings: - - source: testing/schema_with_properties - resource: {} + dekaf: &config + variant: bar + config: foo/bar + bindings: [] driver: + materializations: + good: &connector + connectorType: DEKAF + config: *config + bindings: [] dataPlanes: "1d:1d:1d:1d:1d:1d:1d:1d": default: true diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap index 961ba8d6fa..b0c2bb9dfd 100644 --- a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap +++ b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap @@ -1,461 +1,39 @@ --- source: crates/validation/tests/scenario_tests.rs -assertion_line: 69 +assertion_line: 68 expression: outcome --- Outcome { built_captures: [], - built_collections: [ - BuiltCollection { - collection: testing/schema_with_properties, - scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, - control_id: "0000000000000000", - data_plane_id: "1d1d1d1d1d1d1d1d", - expect_pub_id: "0000000000000000", - expect_build_id: "0000000000000000", - model: { - "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, - "key": [ - "/id" - ] - }, - validated: NULL, - spec: CollectionSpec { - name: "testing/schema_with_properties", - write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", - read_schema_json: "", - key: [ - "/id", - ], - uuid_ptr: "/_meta/uuid", - partition_fields: [], - projections: [ - Projection { - ptr: "/_meta/flow_truncated", - field: "_meta/flow_truncated", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "boolean", - ], - string: None, - title: "Flow truncation indicator", - description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "", - field: "flow_document", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "object", - ], - string: None, - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/_meta/uuid", - field: "flow_published_at", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "date-time", - content_encoding: "uuid", - max_length: 0, - }, - ), - title: "Flow Publication Time", - description: "Flow publication date-time of this document", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/id", - field: "id", - explicit: false, - is_partition_key: false, - is_primary_key: true, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "", - content_encoding: "", - max_length: 0, - }, - ), - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - ], - ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", - partition_template: Some( - JournalSpec { - name: "testing/schema_with_properties/2020202020202020", - replication: 3, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "content-type", - value: "application/x-ndjson", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/collection", - value: "testing/schema_with_properties", - prefix: false, - }, - ], - }, - ), - fragment: Some( - Fragment { - length: 536870912, - compression_codec: Gzip, - stores: [ - "s3://a-bucket/", - ], - refresh_interval: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - retention: None, - flush_interval: None, - path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", - }, - ), - flags: 4, - max_append_rate: 4194304, - }, - ), - derivation: None, - }, - previous_spec: NULL, - is_touch: 0, - dependency_hash: NULL, - }, - ], + built_collections: [], built_materializations: [ BuiltMaterialization { - materialization: testing/test_dekaf, - scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, + materialization: good, + scope: test://example/catalog.yaml#/materializations/good, control_id: "0000000000000000", data_plane_id: "1d1d1d1d1d1d1d1d", expect_pub_id: "0000000000000000", expect_build_id: "0000000000000000", model: { "endpoint": { - "dekaf": "example/dekaf.yaml" - }, - "bindings": [ - { - "resource": {}, - "source": "testing/schema_with_properties", - "fields": { - "recommended": true - } + "dekaf": { + "variant": "foo", + "config": {"strict_topic_names":false} } - ] + }, + "bindings": [] }, validated: Validated { - bindings: [ - Binding { - constraints: { - "_meta/flow_truncated": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - "flow_document": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - "flow_published_at": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - "id": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - }, - resource_path: [ - "binding-0", - ], - delta_updates: true, - }, - ], + bindings: [], }, spec: MaterializationSpec { - name: "testing/test_dekaf", + name: "good", connector_type: Dekaf, - config_json: "\"example/dekaf.yaml\"", - bindings: [ - Binding { - resource_config_json: "{}", - resource_path: [ - "binding-0", - ], - collection: Some( - CollectionSpec { - name: "testing/schema_with_properties", - write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", - read_schema_json: "", - key: [ - "/id", - ], - uuid_ptr: "/_meta/uuid", - partition_fields: [], - projections: [ - Projection { - ptr: "/_meta/flow_truncated", - field: "_meta/flow_truncated", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "boolean", - ], - string: None, - title: "Flow truncation indicator", - description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "", - field: "flow_document", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "object", - ], - string: None, - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/_meta/uuid", - field: "flow_published_at", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "date-time", - content_encoding: "uuid", - max_length: 0, - }, - ), - title: "Flow Publication Time", - description: "Flow publication date-time of this document", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/id", - field: "id", - explicit: false, - is_partition_key: false, - is_primary_key: true, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "", - content_encoding: "", - max_length: 0, - }, - ), - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - ], - ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", - partition_template: Some( - JournalSpec { - name: "testing/schema_with_properties/2020202020202020", - replication: 3, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "content-type", - value: "application/x-ndjson", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/collection", - value: "testing/schema_with_properties", - prefix: false, - }, - ], - }, - ), - fragment: Some( - Fragment { - length: 536870912, - compression_codec: Gzip, - stores: [ - "s3://a-bucket/", - ], - refresh_interval: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - retention: None, - flush_interval: None, - path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", - }, - ), - flags: 4, - max_append_rate: 4194304, - }, - ), - derivation: None, - }, - ), - partition_selector: Some( - LabelSelector { - include: Some( - LabelSet { - labels: [ - Label { - name: "estuary.dev/collection", - value: "testing/schema_with_properties", - prefix: false, - }, - ], - }, - ), - exclude: Some( - LabelSet { - labels: [], - }, - ), - }, - ), - priority: 0, - field_selection: Some( - FieldSelection { - keys: [], - values: [], - document: "", - field_config_json_map: {}, - }, - ), - delta_updates: true, - deprecated_shuffle: None, - journal_read_suffix: "materialize/testing/test_dekaf/binding-0", - not_before: None, - not_after: None, - backfill: 0, - state_key: "binding-0", - }, - ], + config_json: "{\"variant\":\"foo\",\"config\":{\"strict_topic_names\":false}}", + bindings: [], shard_template: Some( ShardSpec { - id: "materialize/testing/test_dekaf/2020202020202020", + id: "materialize/good/2020202020202020", sources: [], recovery_log_prefix: "recovery", hint_prefix: "/estuary/flow/hints", @@ -494,7 +72,7 @@ Outcome { }, Label { name: "estuary.dev/task-name", - value: "testing/test_dekaf", + value: "good", prefix: false, }, Label { @@ -512,7 +90,7 @@ Outcome { ), recovery_log_template: Some( JournalSpec { - name: "recovery/materialize/testing/test_dekaf/2020202020202020", + name: "recovery/materialize/good/2020202020202020", replication: 3, labels: Some( LabelSet { @@ -534,7 +112,7 @@ Outcome { }, Label { name: "estuary.dev/task-name", - value: "testing/test_dekaf", + value: "good", prefix: false, }, Label { @@ -571,32 +149,14 @@ Outcome { }, previous_spec: NULL, is_touch: 0, - dependency_hash: 32182596aeb1e4a0, + dependency_hash: NULL, }, ], built_tests: [], captures: [], - collections: [ - DraftCollection { - collection: testing/schema_with_properties, - scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, - expect_pub_id: NULL, - model: { - "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, - "key": [ - "/id" - ] - }, - is_touch: 0, - }, - ], + collections: [], errors: [], - errors_draft: [ - Error { - scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf/endpoint/dekaf/config, - error: failed to fetch resource test://example/example/dekaf.yaml: fixture not found, - }, - ], + errors_draft: [], fetches: [ Fetch { depth: 1, @@ -604,33 +164,23 @@ Outcome { }, Fetch { depth: 2, - resource: test://example/example/dekaf.yaml, - }, - ], - imports: [ - Import { - scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf/endpoint/dekaf/config, - to_resource: test://example/example/dekaf.yaml, + resource: test://example/dekaf.yaml, }, ], + imports: [], materializations: [ DraftMaterialization { - materialization: testing/test_dekaf, - scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, + materialization: good, + scope: test://example/catalog.yaml#/materializations/good, expect_pub_id: NULL, model: { "endpoint": { - "dekaf": "example/dekaf.yaml" - }, - "bindings": [ - { - "resource": {}, - "source": "testing/schema_with_properties", - "fields": { - "recommended": true - } + "dekaf": { + "variant": "foo", + "config": {"strict_topic_names":false} } - ] + }, + "bindings": [] }, is_touch: 0, }, @@ -640,7 +190,13 @@ Outcome { resource: test://example/catalog.yaml, content_type: "CATALOG", content: ".. binary ..", - content_dom: {"collections":{"testing/schema_with_properties":{"key":["/id"],"schema":{"properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}}},"materializations":{"testing/test_dekaf":{"bindings":[{"resource":{},"source":"testing/schema_with_properties"}],"endpoint":{"dekaf":"example/dekaf.yaml"}}}}, + content_dom: {"materializations":{"good":{"bindings":[],"endpoint":{"dekaf":{"config":"test://example/dekaf.yaml","variant":"foo"}}}}}, + }, + Resource { + resource: test://example/dekaf.yaml, + content_type: "CONFIG", + content: ".. binary ..", + content_dom: {"strict_topic_names":false}, }, ], storage_mappings: [ diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_inline_config.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_inline_config.snap index 61615d4d03..6cfa7d5c48 100644 --- a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_inline_config.snap +++ b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_inline_config.snap @@ -1,461 +1,39 @@ --- source: crates/validation/tests/scenario_tests.rs -assertion_line: 38 +assertion_line: 36 expression: outcome --- Outcome { built_captures: [], - built_collections: [ - BuiltCollection { - collection: testing/schema_with_properties, - scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, - control_id: "0000000000000000", - data_plane_id: "1d1d1d1d1d1d1d1d", - expect_pub_id: "0000000000000000", - expect_build_id: "0000000000000000", - model: { - "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, - "key": [ - "/id" - ] - }, - validated: NULL, - spec: CollectionSpec { - name: "testing/schema_with_properties", - write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", - read_schema_json: "", - key: [ - "/id", - ], - uuid_ptr: "/_meta/uuid", - partition_fields: [], - projections: [ - Projection { - ptr: "/_meta/flow_truncated", - field: "_meta/flow_truncated", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "boolean", - ], - string: None, - title: "Flow truncation indicator", - description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "", - field: "flow_document", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "object", - ], - string: None, - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/_meta/uuid", - field: "flow_published_at", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "date-time", - content_encoding: "uuid", - max_length: 0, - }, - ), - title: "Flow Publication Time", - description: "Flow publication date-time of this document", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/id", - field: "id", - explicit: false, - is_partition_key: false, - is_primary_key: true, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "", - content_encoding: "", - max_length: 0, - }, - ), - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - ], - ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", - partition_template: Some( - JournalSpec { - name: "testing/schema_with_properties/2020202020202020", - replication: 3, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "content-type", - value: "application/x-ndjson", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/collection", - value: "testing/schema_with_properties", - prefix: false, - }, - ], - }, - ), - fragment: Some( - Fragment { - length: 536870912, - compression_codec: Gzip, - stores: [ - "s3://a-bucket/", - ], - refresh_interval: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - retention: None, - flush_interval: None, - path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", - }, - ), - flags: 4, - max_append_rate: 4194304, - }, - ), - derivation: None, - }, - previous_spec: NULL, - is_touch: 0, - dependency_hash: NULL, - }, - ], + built_collections: [], built_materializations: [ BuiltMaterialization { - materialization: testing/test_dekaf, - scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, + materialization: good, + scope: test://example/catalog.yaml#/materializations/good, control_id: "0000000000000000", data_plane_id: "1d1d1d1d1d1d1d1d", expect_pub_id: "0000000000000000", expect_build_id: "0000000000000000", model: { "endpoint": { - "dekaf": {} - }, - "bindings": [ - { - "resource": {}, - "source": "testing/schema_with_properties", - "fields": { - "recommended": true - } + "dekaf": { + "variant": "foo", + "config": {"strict_topic_names":false} } - ] + }, + "bindings": [] }, validated: Validated { - bindings: [ - Binding { - constraints: { - "_meta/flow_truncated": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - "flow_document": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - "flow_published_at": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - "id": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - }, - resource_path: [ - "binding-0", - ], - delta_updates: true, - }, - ], + bindings: [], }, spec: MaterializationSpec { - name: "testing/test_dekaf", + name: "good", connector_type: Dekaf, - config_json: "{}", - bindings: [ - Binding { - resource_config_json: "{}", - resource_path: [ - "binding-0", - ], - collection: Some( - CollectionSpec { - name: "testing/schema_with_properties", - write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", - read_schema_json: "", - key: [ - "/id", - ], - uuid_ptr: "/_meta/uuid", - partition_fields: [], - projections: [ - Projection { - ptr: "/_meta/flow_truncated", - field: "_meta/flow_truncated", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "boolean", - ], - string: None, - title: "Flow truncation indicator", - description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "", - field: "flow_document", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "object", - ], - string: None, - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/_meta/uuid", - field: "flow_published_at", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "date-time", - content_encoding: "uuid", - max_length: 0, - }, - ), - title: "Flow Publication Time", - description: "Flow publication date-time of this document", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/id", - field: "id", - explicit: false, - is_partition_key: false, - is_primary_key: true, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "", - content_encoding: "", - max_length: 0, - }, - ), - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - ], - ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", - partition_template: Some( - JournalSpec { - name: "testing/schema_with_properties/2020202020202020", - replication: 3, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "content-type", - value: "application/x-ndjson", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/collection", - value: "testing/schema_with_properties", - prefix: false, - }, - ], - }, - ), - fragment: Some( - Fragment { - length: 536870912, - compression_codec: Gzip, - stores: [ - "s3://a-bucket/", - ], - refresh_interval: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - retention: None, - flush_interval: None, - path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", - }, - ), - flags: 4, - max_append_rate: 4194304, - }, - ), - derivation: None, - }, - ), - partition_selector: Some( - LabelSelector { - include: Some( - LabelSet { - labels: [ - Label { - name: "estuary.dev/collection", - value: "testing/schema_with_properties", - prefix: false, - }, - ], - }, - ), - exclude: Some( - LabelSet { - labels: [], - }, - ), - }, - ), - priority: 0, - field_selection: Some( - FieldSelection { - keys: [], - values: [], - document: "", - field_config_json_map: {}, - }, - ), - delta_updates: true, - deprecated_shuffle: None, - journal_read_suffix: "materialize/testing/test_dekaf/binding-0", - not_before: None, - not_after: None, - backfill: 0, - state_key: "binding-0", - }, - ], + config_json: "{\"variant\":\"foo\",\"config\":{\"strict_topic_names\":false}}", + bindings: [], shard_template: Some( ShardSpec { - id: "materialize/testing/test_dekaf/2020202020202020", + id: "materialize/good/2020202020202020", sources: [], recovery_log_prefix: "recovery", hint_prefix: "/estuary/flow/hints", @@ -494,7 +72,7 @@ Outcome { }, Label { name: "estuary.dev/task-name", - value: "testing/test_dekaf", + value: "good", prefix: false, }, Label { @@ -512,7 +90,7 @@ Outcome { ), recovery_log_template: Some( JournalSpec { - name: "recovery/materialize/testing/test_dekaf/2020202020202020", + name: "recovery/materialize/good/2020202020202020", replication: 3, labels: Some( LabelSet { @@ -534,7 +112,7 @@ Outcome { }, Label { name: "estuary.dev/task-name", - value: "testing/test_dekaf", + value: "good", prefix: false, }, Label { @@ -571,25 +149,12 @@ Outcome { }, previous_spec: NULL, is_touch: 0, - dependency_hash: 32182596aeb1e4a0, + dependency_hash: NULL, }, ], built_tests: [], captures: [], - collections: [ - DraftCollection { - collection: testing/schema_with_properties, - scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, - expect_pub_id: NULL, - model: { - "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, - "key": [ - "/id" - ] - }, - is_touch: 0, - }, - ], + collections: [], errors: [], errors_draft: [], fetches: [ @@ -601,22 +166,17 @@ Outcome { imports: [], materializations: [ DraftMaterialization { - materialization: testing/test_dekaf, - scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, + materialization: good, + scope: test://example/catalog.yaml#/materializations/good, expect_pub_id: NULL, model: { "endpoint": { - "dekaf": {} - }, - "bindings": [ - { - "resource": {}, - "source": "testing/schema_with_properties", - "fields": { - "recommended": true - } + "dekaf": { + "variant": "foo", + "config": {"strict_topic_names":false} } - ] + }, + "bindings": [] }, is_touch: 0, }, @@ -626,7 +186,7 @@ Outcome { resource: test://example/catalog.yaml, content_type: "CATALOG", content: ".. binary ..", - content_dom: {"collections":{"testing/schema_with_properties":{"key":["/id"],"schema":{"properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}}},"materializations":{"testing/test_dekaf":{"bindings":[{"resource":{},"source":"testing/schema_with_properties"}],"endpoint":{"dekaf":{}}}}}, + content_dom: {"materializations":{"good":{"bindings":[],"endpoint":{"dekaf":{"config":{"strict_topic_names":false},"variant":"foo"}}}}}, }, ], storage_mappings: [ diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap index 697478a621..34fc8a0dec 100644 --- a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap +++ b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap @@ -1,6 +1,6 @@ --- source: crates/validation/tests/scenario_tests.rs -assertion_line: 100 +assertion_line: 98 expression: outcome --- Outcome { @@ -14,7 +14,7 @@ Outcome { errors_draft: [ Error { scope: test://example/catalog.yaml, - error: failed to parse document (data did not match any variant of untagged enum DekafConfigContainer at line 1 column 288): data did not match any variant of untagged enum DekafConfigContainer at line 1 column 288, + error: failed to parse document (invalid type: boolean `false`, expected struct DekafConfig at line 1 column 215): invalid type: boolean `false`, expected struct DekafConfig at line 1 column 215, }, ], fetches: [ @@ -30,7 +30,7 @@ Outcome { resource: test://example/catalog.yaml, content_type: "CATALOG", content: ".. binary ..", - content_dom: {"collections":{"testing/schema_with_properties":{"key":["/id"],"schema":{"properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}}},"materializations":{"testing/test_dekaf":{"bindings":[{"resource":{},"source":"testing/schema_with_properties"}],"endpoint":{"dekaf":false}}}}, + content_dom: {"collections":{"testing/schema_with_properties":{"key":["/id"],"schema":{"properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}}},"materializations":{"bad":{"bindings":[],"endpoint":{"dekaf":false}}}}, }, ], storage_mappings: [ diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap index 7240236b95..cdbe3a3744 100644 --- a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap +++ b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap @@ -1,599 +1,24 @@ --- source: crates/validation/tests/scenario_tests.rs -assertion_line: 131 +assertion_line: 125 expression: outcome --- Outcome { built_captures: [], - built_collections: [ - BuiltCollection { - collection: testing/schema_with_properties, - scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, - control_id: "0000000000000000", - data_plane_id: "1d1d1d1d1d1d1d1d", - expect_pub_id: "0000000000000000", - expect_build_id: "0000000000000000", - model: { - "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, - "key": [ - "/id" - ] - }, - validated: NULL, - spec: CollectionSpec { - name: "testing/schema_with_properties", - write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", - read_schema_json: "", - key: [ - "/id", - ], - uuid_ptr: "/_meta/uuid", - partition_fields: [], - projections: [ - Projection { - ptr: "/_meta/flow_truncated", - field: "_meta/flow_truncated", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "boolean", - ], - string: None, - title: "Flow truncation indicator", - description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "", - field: "flow_document", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "object", - ], - string: None, - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/_meta/uuid", - field: "flow_published_at", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "date-time", - content_encoding: "uuid", - max_length: 0, - }, - ), - title: "Flow Publication Time", - description: "Flow publication date-time of this document", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/id", - field: "id", - explicit: false, - is_partition_key: false, - is_primary_key: true, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "", - content_encoding: "", - max_length: 0, - }, - ), - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - ], - ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", - partition_template: Some( - JournalSpec { - name: "testing/schema_with_properties/2020202020202020", - replication: 3, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "content-type", - value: "application/x-ndjson", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/collection", - value: "testing/schema_with_properties", - prefix: false, - }, - ], - }, - ), - fragment: Some( - Fragment { - length: 536870912, - compression_codec: Gzip, - stores: [ - "s3://a-bucket/", - ], - refresh_interval: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - retention: None, - flush_interval: None, - path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", - }, - ), - flags: 4, - max_append_rate: 4194304, - }, - ), - derivation: None, - }, - previous_spec: NULL, - is_touch: 0, - dependency_hash: NULL, - }, - ], - built_materializations: [ - BuiltMaterialization { - materialization: testing/test_dekaf, - scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, - control_id: "0000000000000000", - data_plane_id: "1d1d1d1d1d1d1d1d", - expect_pub_id: "0000000000000000", - expect_build_id: "0000000000000000", - model: { - "endpoint": { - "dekaf": "foo/bar" - }, - "bindings": [ - { - "resource": {}, - "source": "testing/schema_with_properties", - "fields": { - "recommended": true - } - } - ] - }, - validated: Validated { - bindings: [ - Binding { - constraints: { - "_meta/flow_truncated": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - "flow_document": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - "flow_published_at": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - "id": Constraint { - r#type: FieldOptional, - reason: "no-op validator allows everything", - }, - }, - resource_path: [ - "binding-0", - ], - delta_updates: true, - }, - ], - }, - spec: MaterializationSpec { - name: "testing/test_dekaf", - connector_type: Dekaf, - config_json: "\"foo/bar\"", - bindings: [ - Binding { - resource_config_json: "{}", - resource_path: [ - "binding-0", - ], - collection: Some( - CollectionSpec { - name: "testing/schema_with_properties", - write_schema_json: "{\"$id\":\"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema\",\"properties\":{\"id\":{\"type\":\"string\"}},\"required\":[\"id\"],\"type\":\"object\"}", - read_schema_json: "", - key: [ - "/id", - ], - uuid_ptr: "/_meta/uuid", - partition_fields: [], - projections: [ - Projection { - ptr: "/_meta/flow_truncated", - field: "_meta/flow_truncated", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "boolean", - ], - string: None, - title: "Flow truncation indicator", - description: "Indicates whether any of the materialized values for this row have been truncated to make them fit inside the limitations of the destination system.", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "", - field: "flow_document", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "object", - ], - string: None, - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/_meta/uuid", - field: "flow_published_at", - explicit: false, - is_partition_key: false, - is_primary_key: false, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "date-time", - content_encoding: "uuid", - max_length: 0, - }, - ), - title: "Flow Publication Time", - description: "Flow publication date-time of this document", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - Projection { - ptr: "/id", - field: "id", - explicit: false, - is_partition_key: false, - is_primary_key: true, - inference: Some( - Inference { - types: [ - "string", - ], - string: Some( - String { - content_type: "", - format: "", - content_encoding: "", - max_length: 0, - }, - ), - title: "", - description: "", - default_json: "", - secret: false, - exists: Must, - numeric: None, - }, - ), - }, - ], - ack_template_json: "{\"_meta\":{\"ack\":true,\"uuid\":\"DocUUIDPlaceholder-329Bb50aa48EAa9ef\"}}", - partition_template: Some( - JournalSpec { - name: "testing/schema_with_properties/2020202020202020", - replication: 3, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "content-type", - value: "application/x-ndjson", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/collection", - value: "testing/schema_with_properties", - prefix: false, - }, - ], - }, - ), - fragment: Some( - Fragment { - length: 536870912, - compression_codec: Gzip, - stores: [ - "s3://a-bucket/", - ], - refresh_interval: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - retention: None, - flush_interval: None, - path_postfix_template: "utc_date={{.Spool.FirstAppendTime.Format \"2006-01-02\"}}/utc_hour={{.Spool.FirstAppendTime.Format \"15\"}}", - }, - ), - flags: 4, - max_append_rate: 4194304, - }, - ), - derivation: None, - }, - ), - partition_selector: Some( - LabelSelector { - include: Some( - LabelSet { - labels: [ - Label { - name: "estuary.dev/collection", - value: "testing/schema_with_properties", - prefix: false, - }, - ], - }, - ), - exclude: Some( - LabelSet { - labels: [], - }, - ), - }, - ), - priority: 0, - field_selection: Some( - FieldSelection { - keys: [], - values: [], - document: "", - field_config_json_map: {}, - }, - ), - delta_updates: true, - deprecated_shuffle: None, - journal_read_suffix: "materialize/testing/test_dekaf/binding-0", - not_before: None, - not_after: None, - backfill: 0, - state_key: "binding-0", - }, - ], - shard_template: Some( - ShardSpec { - id: "materialize/testing/test_dekaf/2020202020202020", - sources: [], - recovery_log_prefix: "recovery", - hint_prefix: "/estuary/flow/hints", - hint_backups: 2, - max_txn_duration: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - min_txn_duration: Some( - Duration { - seconds: 0, - nanos: 0, - }, - ), - disable: false, - hot_standbys: 0, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/log-level", - value: "info", - prefix: false, - }, - Label { - name: "estuary.dev/task-name", - value: "testing/test_dekaf", - prefix: false, - }, - Label { - name: "estuary.dev/task-type", - value: "materialization", - prefix: false, - }, - ], - }, - ), - disable_wait_for_ack: false, - ring_buffer_size: 65536, - read_channel_size: 4096, - }, - ), - recovery_log_template: Some( - JournalSpec { - name: "recovery/materialize/testing/test_dekaf/2020202020202020", - replication: 3, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "content-type", - value: "application/x-gazette-recoverylog", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/task-name", - value: "testing/test_dekaf", - prefix: false, - }, - Label { - name: "estuary.dev/task-type", - value: "materialization", - prefix: false, - }, - ], - }, - ), - fragment: Some( - Fragment { - length: 268435456, - compression_codec: Snappy, - stores: [ - "s3://a-bucket/", - ], - refresh_interval: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - retention: None, - flush_interval: None, - path_postfix_template: "", - }, - ), - flags: 4, - max_append_rate: 4194304, - }, - ), - network_ports: [], - }, - previous_spec: NULL, - is_touch: 0, - dependency_hash: 32182596aeb1e4a0, - }, - ], + built_collections: [], + built_materializations: [], built_tests: [], captures: [], - collections: [ - DraftCollection { - collection: testing/schema_with_properties, - scope: test://example/catalog.yaml#/collections/testing~1schema_with_properties, - expect_pub_id: NULL, - model: { - "schema": {"$id":"test://example/catalog.yaml?ptr=/collections/testing~1schema_with_properties/schema","properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}, - "key": [ - "/id" - ] - }, - is_touch: 0, + collections: [], + errors: [ + Error { + scope: test://example/catalog.yaml#/materializations/bad, + error: driver fixture not found: bad, }, ], - errors: [], errors_draft: [ Error { - scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf/endpoint/dekaf/config, + scope: test://example/catalog.yaml#/materializations/bad/endpoint/dekaf/config, error: failed to fetch resource test://example/foo/bar: fixture not found, }, ], @@ -609,28 +34,23 @@ Outcome { ], imports: [ Import { - scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf/endpoint/dekaf/config, + scope: test://example/catalog.yaml#/materializations/bad/endpoint/dekaf/config, to_resource: test://example/foo/bar, }, ], materializations: [ DraftMaterialization { - materialization: testing/test_dekaf, - scope: test://example/catalog.yaml#/materializations/testing~1test_dekaf, + materialization: bad, + scope: test://example/catalog.yaml#/materializations/bad, expect_pub_id: NULL, model: { "endpoint": { - "dekaf": "foo/bar" - }, - "bindings": [ - { - "resource": {}, - "source": "testing/schema_with_properties", - "fields": { - "recommended": true - } + "dekaf": { + "variant": "bar", + "config": "test://example/foo/bar" } - ] + }, + "bindings": [] }, is_touch: 0, }, @@ -640,7 +60,7 @@ Outcome { resource: test://example/catalog.yaml, content_type: "CATALOG", content: ".. binary ..", - content_dom: {"collections":{"testing/schema_with_properties":{"key":["/id"],"schema":{"properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}}},"materializations":{"testing/test_dekaf":{"bindings":[{"resource":{},"source":"testing/schema_with_properties"}],"endpoint":{"dekaf":"foo/bar"}}}}, + content_dom: {"materializations":{"bad":{"bindings":[],"endpoint":{"dekaf":{"config":"foo/bar","variant":"bar"}}}}}, }, ], storage_mappings: [ From c99ce10d6056a1a222d50337c5c3b921ec435490 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Mon, 30 Sep 2024 11:38:24 -0400 Subject: [PATCH 11/26] dekaf: Refresh token is not a JWT --- crates/dekaf/src/lib.rs | 9 +++++++-- crates/flow-client/src/client.rs | 8 -------- 2 files changed, 7 insertions(+), 10 deletions(-) diff --git a/crates/dekaf/src/lib.rs b/crates/dekaf/src/lib.rs index 309fb11727..0be0ec58f2 100644 --- a/crates/dekaf/src/lib.rs +++ b/crates/dekaf/src/lib.rs @@ -24,7 +24,9 @@ pub use api_client::KafkaApiClient; use aes_siv::{aead::Aead, Aes256SivAead, KeyInit, KeySizeUser}; use connector::DekafConfig; -use flow_client::{DEFAULT_AGENT_URL, DEFAULT_PG_PUBLIC_TOKEN, DEFAULT_PG_URL}; +use flow_client::{ + client::RefreshToken, DEFAULT_AGENT_URL, DEFAULT_PG_PUBLIC_TOKEN, DEFAULT_PG_URL, +}; use percent_encoding::{percent_decode_str, utf8_percent_encode}; use serde::{Deserialize, Serialize}; use serde_json::de; @@ -64,12 +66,15 @@ impl App { username.to_string() }; + let raw_token = String::from_utf8(base64::decode(password)?.to_vec())?; + let refresh: RefreshToken = serde_json::from_str(raw_token.as_str())?; + let mut client = flow_client::Client::new( DEFAULT_AGENT_URL.to_owned(), DEFAULT_PG_PUBLIC_TOKEN.to_string(), DEFAULT_PG_URL.to_owned(), None, - Some(String::from_utf8(base64::decode(password)?.to_vec())?.try_into()?), + Some(refresh), ); client.refresh().await?; diff --git a/crates/flow-client/src/client.rs b/crates/flow-client/src/client.rs index 2c4c431c5e..c513b87495 100644 --- a/crates/flow-client/src/client.rs +++ b/crates/flow-client/src/client.rs @@ -197,14 +197,6 @@ pub struct RefreshToken { pub secret: String, } -impl TryFrom for RefreshToken { - type Error = anyhow::Error; - - fn try_from(value: String) -> Result { - parse_jwt_claims(value.as_str()) - } -} - #[tracing::instrument(skip(client), err)] pub async fn fetch_task_authorization( client: &Client, From 705281d27aadb44e5ccafd38dbe104127dceebb1 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Tue, 1 Oct 2024 10:44:25 -0400 Subject: [PATCH 12/26] Some PR feedback --- crates/build/Cargo.toml | 1 + crates/build/src/lib.rs | 9 +- crates/flow-client/src/client.rs | 16 +- crates/runtime/Cargo.toml | 1 - crates/runtime/src/unary.rs | 13 +- crates/sources/src/indirect.rs | 2 +- crates/sources/src/scenarios/mod.rs | 2 + .../sources__scenarios__test__dekaf_good.snap | 99 ++++++++ ...urces__scenarios__test__dekaf_invalid.snap | 27 +++ .../src/scenarios/test_dekaf_good.yaml | 27 +++ .../src/scenarios/test_dekaf_invalid.yaml | 7 + crates/validation/tests/scenario_tests.rs | 120 ---------- ...dekaf_materialization_indirect_config.snap | 217 ------------------ ...__dekaf_materialization_inline_config.snap | 207 ----------------- ..._tests__dekaf_materialization_invalid.snap | 51 ---- ...ts__dekaf_materialization_nonexistent.snap | 81 ------- 16 files changed, 182 insertions(+), 698 deletions(-) create mode 100644 crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good.snap create mode 100644 crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid.snap create mode 100644 crates/sources/src/scenarios/test_dekaf_good.yaml create mode 100644 crates/sources/src/scenarios/test_dekaf_invalid.yaml delete mode 100644 crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap delete mode 100644 crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_inline_config.snap delete mode 100644 crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap delete mode 100644 crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap diff --git a/crates/build/Cargo.toml b/crates/build/Cargo.toml index 1a2caabb2e..63d9551dfd 100644 --- a/crates/build/Cargo.toml +++ b/crates/build/Cargo.toml @@ -10,6 +10,7 @@ license.workspace = true [dependencies] assemble = { path = "../assemble" } +dekaf = { path = "../dekaf" } models = { path = "../models" } ops = { path = "../ops" } proto-flow = { path = "../proto-flow" } diff --git a/crates/build/src/lib.rs b/crates/build/src/lib.rs index 292e6b9886..0ea4cdb116 100644 --- a/crates/build/src/lib.rs +++ b/crates/build/src/lib.rs @@ -376,7 +376,14 @@ impl validation::Connectors for RuntimeConnectors { request: materialize::Request, _data_plane: &'a tables::DataPlane, ) -> BoxFuture<'a, anyhow::Result> { - self.runtime.clone().unary_materialize(request).boxed() + match flow::materialization_spec::ConnectorType::try_from( + request.validate.as_ref().unwrap().connector_type, + ) { + Ok(flow::materialization_spec::ConnectorType::Dekaf) => { + dekaf::connector::unary_materialize(request).boxed() + } + _ => self.runtime.clone().unary_materialize(request).boxed(), + } } } diff --git a/crates/flow-client/src/client.rs b/crates/flow-client/src/client.rs index c513b87495..002a1b1bd2 100644 --- a/crates/flow-client/src/client.rs +++ b/crates/flow-client/src/client.rs @@ -14,7 +14,7 @@ pub struct Client { // PostgREST URL. pg_url: url::Url, // PostgREST access token. - pg_token: String, + pg_api_token: String, // User's access token, if authenticated. user_access_token: Option, // User's refresh token, if authenticated. @@ -29,10 +29,10 @@ impl Client { /// Build a new Client from the Config. pub fn new( agent_endpoint: Url, - pg_token: String, + pg_api_token: String, pg_url: Url, - access_token: Option, - refresh_token: Option, + user_access_token: Option, + userrefresh_token: Option, ) -> Self { // Build journal and shard clients with an empty default service address. // We'll use their with_endpoint_and_metadata() routines to cheaply clone @@ -54,12 +54,12 @@ impl Client { Self { agent_endpoint, http_client: reqwest::Client::new(), - pg_token, + pg_api_token, pg_url, journal_client, shard_client, - user_access_token: access_token, - user_refresh_token: refresh_token, + user_access_token, + user_refresh_token: userrefresh_token, } } @@ -131,7 +131,7 @@ impl Client { pub fn pg_client(&self) -> postgrest::Postgrest { let pg_client = postgrest::Postgrest::new(self.pg_url.as_str()) - .insert_header("apikey", self.pg_token.as_str()); + .insert_header("apikey", self.pg_api_token.as_str()); if let Some(token) = &self.user_access_token { return pg_client.insert_header("Authorization", &format!("Bearer {token}")); diff --git a/crates/runtime/Cargo.toml b/crates/runtime/Cargo.toml index c87a071a95..6116c3591a 100644 --- a/crates/runtime/Cargo.toml +++ b/crates/runtime/Cargo.toml @@ -13,7 +13,6 @@ assemble = { path = "../assemble" } async-process = { path = "../async-process" } connector-init = { path = "../connector-init" } coroutines = { path = "../coroutines" } -dekaf = { path = "../dekaf" } derive-sqlite = { path = "../derive-sqlite" } doc = { path = "../doc" } extractors = { path = "../extractors" } diff --git a/crates/runtime/src/unary.rs b/crates/runtime/src/unary.rs index 2cd3b502d8..f71652c803 100644 --- a/crates/runtime/src/unary.rs +++ b/crates/runtime/src/unary.rs @@ -20,17 +20,8 @@ impl Runtime { self, request: materialize::Request, ) -> anyhow::Result { - match materialization_spec::ConnectorType::try_from( - request.validate.as_ref().unwrap().connector_type, - ) { - Ok(materialization_spec::ConnectorType::Dekaf) => { - dekaf::connector::unary_materialize(request).await - } - _ => { - let response = self.serve_materialize(unary_in(request)).boxed(); - unary_out(response).await - } - } + let response = self.serve_materialize(unary_in(request)).boxed(); + unary_out(response).await } } diff --git a/crates/sources/src/indirect.rs b/crates/sources/src/indirect.rs index 47fd94e9a7..8cd7c4e73b 100644 --- a/crates/sources/src/indirect.rs +++ b/crates/sources/src/indirect.rs @@ -501,7 +501,7 @@ fn indirect_materialization( models::MaterializationEndpoint::Dekaf(models::DekafConfig { config, .. }) => indirect_dom( Scope::new(scope) .push_prop("endpoint") - .push_prop("local") + .push_prop("dekaf") .push_prop("config"), config, ContentType::Config, diff --git a/crates/sources/src/scenarios/mod.rs b/crates/sources/src/scenarios/mod.rs index 033767fd01..c6a613acb3 100644 --- a/crates/sources/src/scenarios/mod.rs +++ b/crates/sources/src/scenarios/mod.rs @@ -66,6 +66,8 @@ mod test { test_derivations, test_deletions, test_endpoints_captures_materializations, + test_dekaf_good, + test_dekaf_invalid, test_schema_with_anchors, test_schema_with_inline, test_schema_with_nested_ids, diff --git a/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good.snap b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good.snap new file mode 100644 index 0000000000..32e83ad9ec --- /dev/null +++ b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_good.snap @@ -0,0 +1,99 @@ +--- +source: crates/sources/src/scenarios/mod.rs +assertion_line: 63 +expression: tables +--- +DraftCatalog { + materializations: [ + DraftMaterialization { + materialization: materialization/dekaf/indirect, + scope: test://example/catalog.yaml#/materializations/materialization~1dekaf~1indirect, + expect_pub_id: NULL, + model: { + "endpoint": { + "dekaf": { + "variant": "foo", + "config": "test://example/referenced/dekaf_endpoint.yaml" + } + }, + "bindings": [ + { + "resource": "test://example/referenced/dekaf_resource.yaml", + "source": "some/source/materialization", + "fields": { + "recommended": true + } + } + ] + }, + is_touch: 0, + }, + DraftMaterialization { + materialization: materialization/dekaf/inline, + scope: test://example/catalog.yaml#/materializations/materialization~1dekaf~1inline, + expect_pub_id: NULL, + model: { + "endpoint": { + "dekaf": { + "variant": "foo", + "config": {"strict_topic_names":false} + } + }, + "bindings": [ + { + "resource": {"topic_name":"foo"}, + "source": "some/source/collection", + "fields": { + "recommended": true + } + } + ] + }, + is_touch: 0, + }, + ], + fetches: [ + Fetch { + depth: 1, + resource: test://example/catalog.yaml, + }, + Fetch { + depth: 2, + resource: test://example/referenced/dekaf_endpoint.yaml, + }, + Fetch { + depth: 2, + resource: test://example/referenced/dekaf_resource.yaml, + }, + ], + imports: [ + Import { + scope: test://example/catalog.yaml#/materializations/materialization~1dekaf~1indirect/bindings/0/resource, + to_resource: test://example/referenced/dekaf_resource.yaml, + }, + Import { + scope: test://example/catalog.yaml#/materializations/materialization~1dekaf~1indirect/endpoint/dekaf/config, + to_resource: test://example/referenced/dekaf_endpoint.yaml, + }, + ], + resources: [ + Resource { + resource: test://example/catalog.yaml, + content_type: "CATALOG", + content: ".. binary ..", + content_dom: {"materializations":{"materialization/dekaf/indirect":{"bindings":[{"resource":"test://example/referenced/dekaf_resource.yaml","source":"some/source/materialization"}],"endpoint":{"dekaf":{"config":"test://example/referenced/dekaf_endpoint.yaml","variant":"foo"}}},"materialization/dekaf/inline":{"bindings":[{"resource":{"topic_name":"foo"},"source":"some/source/collection"}],"endpoint":{"dekaf":{"config":{"strict_topic_names":false},"variant":"foo"}}}}}, + }, + Resource { + resource: test://example/referenced/dekaf_endpoint.yaml, + content_type: "CONFIG", + content: ".. binary ..", + content_dom: {"strict_topic_names":false,"token":"foo_token"}, + }, + Resource { + resource: test://example/referenced/dekaf_resource.yaml, + content_type: "CONFIG", + content: ".. binary ..", + content_dom: {"topic":"pizza"}, + }, + ], +} diff --git a/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid.snap b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid.snap new file mode 100644 index 0000000000..898e2ea934 --- /dev/null +++ b/crates/sources/src/scenarios/snapshots/sources__scenarios__test__dekaf_invalid.snap @@ -0,0 +1,27 @@ +--- +source: crates/sources/src/scenarios/mod.rs +assertion_line: 63 +expression: tables +--- +DraftCatalog { + errors: [ + Error { + scope: test://example/catalog.yaml, + error: failed to parse document (missing field `variant` at line 1 column 103): missing field `variant` at line 1 column 103, + }, + ], + fetches: [ + Fetch { + depth: 1, + resource: test://example/catalog.yaml, + }, + ], + resources: [ + Resource { + resource: test://example/catalog.yaml, + content_type: "CATALOG", + content: ".. binary ..", + content_dom: {"materializations":{"materialization/dekaf/invalid":{"bindings":[],"endpoint":{"dekaf":{"test":"fail"}}}}}, + }, + ], +} diff --git a/crates/sources/src/scenarios/test_dekaf_good.yaml b/crates/sources/src/scenarios/test_dekaf_good.yaml new file mode 100644 index 0000000000..5ca092c902 --- /dev/null +++ b/crates/sources/src/scenarios/test_dekaf_good.yaml @@ -0,0 +1,27 @@ +test://example/catalog.yaml: + materializations: + materialization/dekaf/inline: + endpoint: + dekaf: + variant: foo + config: + strict_topic_names: false + bindings: + - source: some/source/collection + resource: + topic_name: foo + materialization/dekaf/indirect: + endpoint: + dekaf: + variant: foo + config: test://example/referenced/dekaf_endpoint.yaml + bindings: + - source: some/source/materialization + resource: test://example/referenced/dekaf_resource.yaml + +test://example/referenced/dekaf_endpoint.yaml: + strict_topic_names: false + token: "foo_token" + +test://example/referenced/dekaf_resource.yaml: + topic: "pizza" \ No newline at end of file diff --git a/crates/sources/src/scenarios/test_dekaf_invalid.yaml b/crates/sources/src/scenarios/test_dekaf_invalid.yaml new file mode 100644 index 0000000000..ddd0443b15 --- /dev/null +++ b/crates/sources/src/scenarios/test_dekaf_invalid.yaml @@ -0,0 +1,7 @@ +test://example/catalog.yaml: + materializations: + materialization/dekaf/invalid: + endpoint: + dekaf: + test: "fail" + bindings: [] \ No newline at end of file diff --git a/crates/validation/tests/scenario_tests.rs b/crates/validation/tests/scenario_tests.rs index 7689508e81..4ea38f84a7 100644 --- a/crates/validation/tests/scenario_tests.rs +++ b/crates/validation/tests/scenario_tests.rs @@ -8,126 +8,6 @@ fn test_golden_all_visits() { insta::assert_debug_snapshot!(outcome); } -#[test] -fn test_dekaf_materialization_inline_config() { - let fixture = r##" -test://example/catalog.yaml: - materializations: - good: - endpoint: - dekaf: &config - variant: foo - config: - strict_topic_names: false - bindings: [] -driver: - materializations: - good: &connector - connectorType: DEKAF - config: *config - bindings: [] - dataPlanes: - "1d:1d:1d:1d:1d:1d:1d:1d": - default: true -"##; - - let outcome = common::run(fixture, "{}"); - // Expect not to see any projections for the empty properties - insta::assert_debug_snapshot!(outcome); -} - -#[test] -fn test_dekaf_materialization_indirect_config() { - let fixture = r##" -test://example/catalog.yaml: - materializations: - good: - endpoint: - dekaf: &config - variant: foo - config: test://example/dekaf.yaml - bindings: [] -test://example/dekaf.yaml: - strict_topic_names: false -driver: - materializations: - good: &connector - connectorType: DEKAF - config: - variant: foo - config: - strict_topic_names: false - bindings: [] - dataPlanes: - "1d:1d:1d:1d:1d:1d:1d:1d": - default: true -"##; - - let outcome = common::run(fixture, "{}"); - // Expect not to see any projections for the empty properties - insta::assert_debug_snapshot!(outcome); -} - -#[test] -fn test_dekaf_materialization_invalid() { - let fixture = r##" -test://example/catalog.yaml: - collections: - testing/schema_with_properties: - schema: - type: object - properties: - id: { type: string } - required: [id] - key: [/id] - materializations: - bad: - endpoint: &config - dekaf: false - bindings: [] -driver: - materializations: - bad: &connector - connectorType: DEKAF - config: *config - bindings: [] - dataPlanes: - "1d:1d:1d:1d:1d:1d:1d:1d": - default: true -"##; - - let outcome = common::run(fixture, "{}"); - // Expect not to see any projections for the empty properties - insta::assert_debug_snapshot!(outcome); -} - -#[test] -fn test_dekaf_materialization_nonexistent() { - let fixture = r##" -test://example/catalog.yaml: - materializations: - bad: - endpoint: - dekaf: &config - variant: bar - config: foo/bar - bindings: [] -driver: - materializations: - good: &connector - connectorType: DEKAF - config: *config - bindings: [] - dataPlanes: - "1d:1d:1d:1d:1d:1d:1d:1d": - default: true -"##; - - let outcome = common::run(fixture, "{}"); - // Expect not to see any projections for the empty properties - insta::assert_debug_snapshot!(outcome); -} - #[test] fn test_projection_not_created_for_empty_properties() { let fixture = r##" diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap deleted file mode 100644 index b0c2bb9dfd..0000000000 --- a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_indirect_config.snap +++ /dev/null @@ -1,217 +0,0 @@ ---- -source: crates/validation/tests/scenario_tests.rs -assertion_line: 68 -expression: outcome ---- -Outcome { - built_captures: [], - built_collections: [], - built_materializations: [ - BuiltMaterialization { - materialization: good, - scope: test://example/catalog.yaml#/materializations/good, - control_id: "0000000000000000", - data_plane_id: "1d1d1d1d1d1d1d1d", - expect_pub_id: "0000000000000000", - expect_build_id: "0000000000000000", - model: { - "endpoint": { - "dekaf": { - "variant": "foo", - "config": {"strict_topic_names":false} - } - }, - "bindings": [] - }, - validated: Validated { - bindings: [], - }, - spec: MaterializationSpec { - name: "good", - connector_type: Dekaf, - config_json: "{\"variant\":\"foo\",\"config\":{\"strict_topic_names\":false}}", - bindings: [], - shard_template: Some( - ShardSpec { - id: "materialize/good/2020202020202020", - sources: [], - recovery_log_prefix: "recovery", - hint_prefix: "/estuary/flow/hints", - hint_backups: 2, - max_txn_duration: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - min_txn_duration: Some( - Duration { - seconds: 0, - nanos: 0, - }, - ), - disable: false, - hot_standbys: 0, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/log-level", - value: "info", - prefix: false, - }, - Label { - name: "estuary.dev/task-name", - value: "good", - prefix: false, - }, - Label { - name: "estuary.dev/task-type", - value: "materialization", - prefix: false, - }, - ], - }, - ), - disable_wait_for_ack: false, - ring_buffer_size: 65536, - read_channel_size: 4096, - }, - ), - recovery_log_template: Some( - JournalSpec { - name: "recovery/materialize/good/2020202020202020", - replication: 3, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "content-type", - value: "application/x-gazette-recoverylog", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/task-name", - value: "good", - prefix: false, - }, - Label { - name: "estuary.dev/task-type", - value: "materialization", - prefix: false, - }, - ], - }, - ), - fragment: Some( - Fragment { - length: 268435456, - compression_codec: Snappy, - stores: [ - "s3://a-bucket/", - ], - refresh_interval: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - retention: None, - flush_interval: None, - path_postfix_template: "", - }, - ), - flags: 4, - max_append_rate: 4194304, - }, - ), - network_ports: [], - }, - previous_spec: NULL, - is_touch: 0, - dependency_hash: NULL, - }, - ], - built_tests: [], - captures: [], - collections: [], - errors: [], - errors_draft: [], - fetches: [ - Fetch { - depth: 1, - resource: test://example/catalog.yaml, - }, - Fetch { - depth: 2, - resource: test://example/dekaf.yaml, - }, - ], - imports: [], - materializations: [ - DraftMaterialization { - materialization: good, - scope: test://example/catalog.yaml#/materializations/good, - expect_pub_id: NULL, - model: { - "endpoint": { - "dekaf": { - "variant": "foo", - "config": {"strict_topic_names":false} - } - }, - "bindings": [] - }, - is_touch: 0, - }, - ], - resources: [ - Resource { - resource: test://example/catalog.yaml, - content_type: "CATALOG", - content: ".. binary ..", - content_dom: {"materializations":{"good":{"bindings":[],"endpoint":{"dekaf":{"config":"test://example/dekaf.yaml","variant":"foo"}}}}}, - }, - Resource { - resource: test://example/dekaf.yaml, - content_type: "CONFIG", - content: ".. binary ..", - content_dom: {"strict_topic_names":false}, - }, - ], - storage_mappings: [ - StorageMapping { - catalog_prefix: , - control_id: "0000000000000000", - stores: [ - { - "provider": "S3", - "bucket": "a-bucket", - "prefix": null, - "region": null - } - ], - }, - ], - tests: [], -} diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_inline_config.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_inline_config.snap deleted file mode 100644 index 6cfa7d5c48..0000000000 --- a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_inline_config.snap +++ /dev/null @@ -1,207 +0,0 @@ ---- -source: crates/validation/tests/scenario_tests.rs -assertion_line: 36 -expression: outcome ---- -Outcome { - built_captures: [], - built_collections: [], - built_materializations: [ - BuiltMaterialization { - materialization: good, - scope: test://example/catalog.yaml#/materializations/good, - control_id: "0000000000000000", - data_plane_id: "1d1d1d1d1d1d1d1d", - expect_pub_id: "0000000000000000", - expect_build_id: "0000000000000000", - model: { - "endpoint": { - "dekaf": { - "variant": "foo", - "config": {"strict_topic_names":false} - } - }, - "bindings": [] - }, - validated: Validated { - bindings: [], - }, - spec: MaterializationSpec { - name: "good", - connector_type: Dekaf, - config_json: "{\"variant\":\"foo\",\"config\":{\"strict_topic_names\":false}}", - bindings: [], - shard_template: Some( - ShardSpec { - id: "materialize/good/2020202020202020", - sources: [], - recovery_log_prefix: "recovery", - hint_prefix: "/estuary/flow/hints", - hint_backups: 2, - max_txn_duration: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - min_txn_duration: Some( - Duration { - seconds: 0, - nanos: 0, - }, - ), - disable: false, - hot_standbys: 0, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/log-level", - value: "info", - prefix: false, - }, - Label { - name: "estuary.dev/task-name", - value: "good", - prefix: false, - }, - Label { - name: "estuary.dev/task-type", - value: "materialization", - prefix: false, - }, - ], - }, - ), - disable_wait_for_ack: false, - ring_buffer_size: 65536, - read_channel_size: 4096, - }, - ), - recovery_log_template: Some( - JournalSpec { - name: "recovery/materialize/good/2020202020202020", - replication: 3, - labels: Some( - LabelSet { - labels: [ - Label { - name: "app.gazette.dev/managed-by", - value: "estuary.dev/flow", - prefix: false, - }, - Label { - name: "content-type", - value: "application/x-gazette-recoverylog", - prefix: false, - }, - Label { - name: "estuary.dev/build", - value: "2121212121212121", - prefix: false, - }, - Label { - name: "estuary.dev/task-name", - value: "good", - prefix: false, - }, - Label { - name: "estuary.dev/task-type", - value: "materialization", - prefix: false, - }, - ], - }, - ), - fragment: Some( - Fragment { - length: 268435456, - compression_codec: Snappy, - stores: [ - "s3://a-bucket/", - ], - refresh_interval: Some( - Duration { - seconds: 300, - nanos: 0, - }, - ), - retention: None, - flush_interval: None, - path_postfix_template: "", - }, - ), - flags: 4, - max_append_rate: 4194304, - }, - ), - network_ports: [], - }, - previous_spec: NULL, - is_touch: 0, - dependency_hash: NULL, - }, - ], - built_tests: [], - captures: [], - collections: [], - errors: [], - errors_draft: [], - fetches: [ - Fetch { - depth: 1, - resource: test://example/catalog.yaml, - }, - ], - imports: [], - materializations: [ - DraftMaterialization { - materialization: good, - scope: test://example/catalog.yaml#/materializations/good, - expect_pub_id: NULL, - model: { - "endpoint": { - "dekaf": { - "variant": "foo", - "config": {"strict_topic_names":false} - } - }, - "bindings": [] - }, - is_touch: 0, - }, - ], - resources: [ - Resource { - resource: test://example/catalog.yaml, - content_type: "CATALOG", - content: ".. binary ..", - content_dom: {"materializations":{"good":{"bindings":[],"endpoint":{"dekaf":{"config":{"strict_topic_names":false},"variant":"foo"}}}}}, - }, - ], - storage_mappings: [ - StorageMapping { - catalog_prefix: , - control_id: "0000000000000000", - stores: [ - { - "provider": "S3", - "bucket": "a-bucket", - "prefix": null, - "region": null - } - ], - }, - ], - tests: [], -} diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap deleted file mode 100644 index 34fc8a0dec..0000000000 --- a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_invalid.snap +++ /dev/null @@ -1,51 +0,0 @@ ---- -source: crates/validation/tests/scenario_tests.rs -assertion_line: 98 -expression: outcome ---- -Outcome { - built_captures: [], - built_collections: [], - built_materializations: [], - built_tests: [], - captures: [], - collections: [], - errors: [], - errors_draft: [ - Error { - scope: test://example/catalog.yaml, - error: failed to parse document (invalid type: boolean `false`, expected struct DekafConfig at line 1 column 215): invalid type: boolean `false`, expected struct DekafConfig at line 1 column 215, - }, - ], - fetches: [ - Fetch { - depth: 1, - resource: test://example/catalog.yaml, - }, - ], - imports: [], - materializations: [], - resources: [ - Resource { - resource: test://example/catalog.yaml, - content_type: "CATALOG", - content: ".. binary ..", - content_dom: {"collections":{"testing/schema_with_properties":{"key":["/id"],"schema":{"properties":{"id":{"type":"string"}},"required":["id"],"type":"object"}}},"materializations":{"bad":{"bindings":[],"endpoint":{"dekaf":false}}}}, - }, - ], - storage_mappings: [ - StorageMapping { - catalog_prefix: , - control_id: "0000000000000000", - stores: [ - { - "provider": "S3", - "bucket": "a-bucket", - "prefix": null, - "region": null - } - ], - }, - ], - tests: [], -} diff --git a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap b/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap deleted file mode 100644 index cdbe3a3744..0000000000 --- a/crates/validation/tests/snapshots/scenario_tests__dekaf_materialization_nonexistent.snap +++ /dev/null @@ -1,81 +0,0 @@ ---- -source: crates/validation/tests/scenario_tests.rs -assertion_line: 125 -expression: outcome ---- -Outcome { - built_captures: [], - built_collections: [], - built_materializations: [], - built_tests: [], - captures: [], - collections: [], - errors: [ - Error { - scope: test://example/catalog.yaml#/materializations/bad, - error: driver fixture not found: bad, - }, - ], - errors_draft: [ - Error { - scope: test://example/catalog.yaml#/materializations/bad/endpoint/dekaf/config, - error: failed to fetch resource test://example/foo/bar: fixture not found, - }, - ], - fetches: [ - Fetch { - depth: 1, - resource: test://example/catalog.yaml, - }, - Fetch { - depth: 2, - resource: test://example/foo/bar, - }, - ], - imports: [ - Import { - scope: test://example/catalog.yaml#/materializations/bad/endpoint/dekaf/config, - to_resource: test://example/foo/bar, - }, - ], - materializations: [ - DraftMaterialization { - materialization: bad, - scope: test://example/catalog.yaml#/materializations/bad, - expect_pub_id: NULL, - model: { - "endpoint": { - "dekaf": { - "variant": "bar", - "config": "test://example/foo/bar" - } - }, - "bindings": [] - }, - is_touch: 0, - }, - ], - resources: [ - Resource { - resource: test://example/catalog.yaml, - content_type: "CATALOG", - content: ".. binary ..", - content_dom: {"materializations":{"bad":{"bindings":[],"endpoint":{"dekaf":{"config":"foo/bar","variant":"bar"}}}}}, - }, - ], - storage_mappings: [ - StorageMapping { - catalog_prefix: , - control_id: "0000000000000000", - stores: [ - { - "provider": "S3", - "bucket": "a-bucket", - "prefix": null, - "region": null - } - ], - }, - ], - tests: [], -} From 0a951141fa3bd36d575bcab36f2d0db212f7d23b Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Wed, 2 Oct 2024 12:14:15 -0400 Subject: [PATCH 13/26] dekaf: Implement a couple memory optimizations --- crates/dekaf/src/api_client.rs | 27 +++++++++++++++++++-------- crates/dekaf/src/lib.rs | 16 ++++++++-------- crates/dekaf/src/main.rs | 25 +++++++++++-------------- crates/dekaf/src/session.rs | 10 +++++----- 4 files changed, 43 insertions(+), 35 deletions(-) diff --git a/crates/dekaf/src/api_client.rs b/crates/dekaf/src/api_client.rs index bab79307a6..17ae6133c6 100644 --- a/crates/dekaf/src/api_client.rs +++ b/crates/dekaf/src/api_client.rs @@ -7,8 +7,10 @@ use kafka_protocol::{ protocol::{self, Decodable, Encodable, Request}, }; use rsasl::{config::SASLConfig, mechname::Mechname, prelude::SASLClient}; +use rustls::RootCertStore; use std::{boxed::Box, cell::Cell, collections::HashMap, fmt::Debug, io, time::Duration}; use std::{io::BufWriter, pin::Pin, sync::Arc}; +use tokio::sync::OnceCell; use tokio::sync::RwLock; use tokio_rustls::rustls; use tokio_util::{codec, task::AbortOnDropHandle}; @@ -24,22 +26,31 @@ type BoxedKafkaConnection = Pin< >, >; +static ROOT_CERT_STORE: OnceCell> = OnceCell::const_new(); + #[tracing::instrument(skip_all)] async fn async_connect(broker_url: &str) -> anyhow::Result { // Establish a TCP connection to the Kafka broker let parsed_url = Url::parse(broker_url)?; - // This returns an Err indicating that the default provider is already set - // but without this call rustls crashes with the following error: - // `no process-level CryptoProvider available -- call CryptoProvider::install_default() before this point` - let _ = rustls::crypto::aws_lc_rs::default_provider().install_default(); - - let mut root_cert_store = rustls::RootCertStore::empty(); - root_cert_store.add_parsable_certificates(rustls_native_certs::load_native_certs()?); + let root_certs = ROOT_CERT_STORE + .get_or_try_init(|| async { + // This returns an Err indicating that the default provider is already set + // but without this call rustls crashes with the following error: + // `no process-level CryptoProvider available -- call CryptoProvider::install_default() before this point` + let _ = rustls::crypto::aws_lc_rs::default_provider().install_default(); + + let mut certs = rustls::RootCertStore::empty(); + certs.add_parsable_certificates( + rustls_native_certs::load_native_certs().expect("failed to load native certs"), + ); + Ok::, anyhow::Error>(Arc::new(certs)) + }) + .await?; let tls_config = rustls::ClientConfig::builder() - .with_root_certificates(root_cert_store) + .with_root_certificates(root_certs.to_owned()) .with_no_client_auth(); let tls_connector = tokio_rustls::TlsConnector::from(Arc::new(tls_config)); diff --git a/crates/dekaf/src/lib.rs b/crates/dekaf/src/lib.rs index 0be0ec58f2..8c1016e607 100644 --- a/crates/dekaf/src/lib.rs +++ b/crates/dekaf/src/lib.rs @@ -24,17 +24,13 @@ pub use api_client::KafkaApiClient; use aes_siv::{aead::Aead, Aes256SivAead, KeyInit, KeySizeUser}; use connector::DekafConfig; -use flow_client::{ - client::RefreshToken, DEFAULT_AGENT_URL, DEFAULT_PG_PUBLIC_TOKEN, DEFAULT_PG_URL, -}; +use flow_client::{client::RefreshToken, DEFAULT_AGENT_URL}; use percent_encoding::{percent_decode_str, utf8_percent_encode}; use serde::{Deserialize, Serialize}; -use serde_json::de; use std::time::SystemTime; +use url::Url; pub struct App { - /// Anonymous API client for the Estuary control plane. - pub anon_client: postgrest::Postgrest, /// Hostname which is advertised for Kafka access. pub advertise_host: String, /// Port which is advertised for Kafka access. @@ -43,6 +39,10 @@ pub struct App { pub kafka_client: KafkaApiClient, /// Secret used to secure Prometheus endpoint pub secret: String, + /// Supabase endpoint + pub api_endpoint: Url, + /// Supabase api key + pub api_key: String, } #[derive(Debug, Clone, Serialize, Deserialize)] @@ -71,8 +71,8 @@ impl App { let mut client = flow_client::Client::new( DEFAULT_AGENT_URL.to_owned(), - DEFAULT_PG_PUBLIC_TOKEN.to_string(), - DEFAULT_PG_URL.to_owned(), + self.api_key.to_owned(), + self.api_endpoint.to_owned(), None, Some(refresh), ); diff --git a/crates/dekaf/src/main.rs b/crates/dekaf/src/main.rs index 19286dd661..6fcfd2e3c7 100644 --- a/crates/dekaf/src/main.rs +++ b/crates/dekaf/src/main.rs @@ -5,6 +5,7 @@ use anyhow::{bail, Context}; use axum_server::tls_rustls::RustlsConfig; use clap::{Args, Parser}; use dekaf::{KafkaApiClient, Session}; +use flow_client::{DEFAULT_PG_PUBLIC_TOKEN, DEFAULT_PG_URL, LOCAL_PG_PUBLIC_TOKEN, LOCAL_PG_URL}; use futures::{FutureExt, TryStreamExt}; use rsasl::config::SASLConfig; use rustls::pki_types::CertificateDer; @@ -16,6 +17,7 @@ use std::{ }; use tokio::io::{split, AsyncRead, AsyncWrite, AsyncWriteExt}; use tracing_subscriber::{filter::LevelFilter, EnvFilter}; +use url::Url; /// A Kafka-compatible proxy for reading Estuary Flow collections. #[derive(Debug, Parser, serde::Serialize)] @@ -24,14 +26,14 @@ pub struct Cli { /// Endpoint of the Estuary API to use. #[arg( long, - default_value = MANAGED_API_ENDPOINT, + default_value = DEFAULT_PG_URL.as_str(), env = "API_ENDPOINT" )] - api_endpoint: String, + api_endpoint: Url, /// Public (anon) API key to use during authentication to the Estuary API. #[arg( long, - default_value = MANAGED_API_KEY, + default_value = DEFAULT_PG_PUBLIC_TOKEN, env = "API_KEY" )] api_key: String, @@ -106,10 +108,10 @@ async fn main() -> anyhow::Result<()> { let cli = Cli::parse(); tracing::info!("Starting dekaf"); - let (api_endpoint, api_token) = if cli.local { - (LOCAL_API_ENDPOINT, LOCAL_API_KEY) + let (api_endpoint, api_key) = if cli.local { + (LOCAL_PG_URL.to_owned(), LOCAL_PG_PUBLIC_TOKEN.to_string()) } else { - (cli.api_endpoint.as_str(), cli.api_key.as_str()) + (cli.api_endpoint, cli.api_key) }; let upstream_kafka_host = format!( @@ -118,7 +120,6 @@ async fn main() -> anyhow::Result<()> { ); let app = Arc::new(dekaf::App { - anon_client: postgrest::Postgrest::new(api_endpoint).insert_header("apikey", api_token), advertise_host: cli.advertise_host.to_owned(), advertise_kafka_port: cli.kafka_port, kafka_client: KafkaApiClient::connect( @@ -131,7 +132,9 @@ async fn main() -> anyhow::Result<()> { ).await.context( "failed to connect or authenticate to upstream Kafka broker used for serving group management APIs", )?, - secret: cli.encryption_secret.to_owned() + secret: cli.encryption_secret.to_owned(), + api_endpoint, + api_key }); tracing::info!( @@ -320,9 +323,3 @@ fn validate_certificate_name( } return Ok(false); } - -const MANAGED_API_KEY: &str = "eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJzdXBhYmFzZSIsInJlZiI6ImV5cmNubXV6enlyaXlwZGFqd2RrIiwicm9sZSI6ImFub24iLCJpYXQiOjE2NDg3NTA1NzksImV4cCI6MTk2NDMyNjU3OX0.y1OyXD3-DYMz10eGxzo1eeamVMMUwIIeOoMryTRAoco"; -const MANAGED_API_ENDPOINT: &str = "https://eyrcnmuzzyriypdajwdk.supabase.co/rest/v1"; - -const LOCAL_API_KEY: &str = "eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJzdXBhYmFzZS1kZW1vIiwicm9sZSI6ImFub24iLCJleHAiOjE5ODM4MTI5OTZ9.CRXP1A7WOeoJeXxjNni43kdQwgnWNReilDMblYTn_I0"; -const LOCAL_API_ENDPOINT: &str = "http://127.0.0.1:5431/rest/v1"; diff --git a/crates/dekaf/src/session.rs b/crates/dekaf/src/session.rs index 3032d4474d..5a1cdcf748 100644 --- a/crates/dekaf/src/session.rs +++ b/crates/dekaf/src/session.rs @@ -27,7 +27,7 @@ use tracing::instrument; struct PendingRead { offset: i64, // Journal offset to be completed by this PendingRead. last_write_head: i64, // Most-recent observed journal write head. - handle: tokio::task::JoinHandle>, + handle: tokio_util::task::AbortOnDropHandle>, } pub struct Session { @@ -402,9 +402,9 @@ impl Session { let pending = PendingRead { offset: fetch_offset, last_write_head: fetch_offset, - handle: tokio::spawn( + handle: tokio_util::task::AbortOnDropHandle::new(tokio::spawn( read.next_batch(partition_request.partition_max_bytes as usize), - ), + )), }; tracing::info!( @@ -453,9 +453,9 @@ impl Session { } { pending.offset = read.offset; pending.last_write_head = read.last_write_head; - pending.handle = tokio::spawn( + pending.handle = tokio_util::task::AbortOnDropHandle::new(tokio::spawn( read.next_batch(partition_request.partition_max_bytes as usize), - ); + )); batch } else { bytes::Bytes::new() From 6b52dbd663ab1637c41a192b21b10c24973662af Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Wed, 2 Oct 2024 14:52:22 -0400 Subject: [PATCH 14/26] dekaf: Add some important annotations to endpoint/resource schemas --- crates/dekaf/src/connector.rs | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/crates/dekaf/src/connector.rs b/crates/dekaf/src/connector.rs index dfce7911c6..253bdbb2f9 100644 --- a/crates/dekaf/src/connector.rs +++ b/crates/dekaf/src/connector.rs @@ -13,6 +13,7 @@ pub struct DekafConfig { /// The password that will authenticate Kafka consumers to this task. // TODO(jshearer): Uncomment when schemars 1.0 is out and we upgrade // #[schemars(extend("secret" = true))] + #[schemars(schema_with = "token_secret")] pub token: String, } @@ -21,9 +22,25 @@ pub struct DekafConfig { pub struct DekafResourceConfig { /// The exposed name of the topic that maps to this binding. This /// will be exposed through the Kafka metadata/discovery APIs. + #[schemars(schema_with = "collection_name")] pub topic_name: String, } +fn collection_name(_gen: &mut schemars::gen::SchemaGenerator) -> schemars::schema::Schema { + serde_json::from_value(serde_json::json!({ + "x-collection-name": true, + })) + .unwrap() +} + +fn token_secret(_gen: &mut schemars::gen::SchemaGenerator) -> schemars::schema::Schema { + serde_json::from_value(serde_json::json!({ + "title": "Dekaf Auth Token", + "secret": true, + })) + .unwrap() +} + pub async fn unary_materialize( request: materialize::Request, ) -> anyhow::Result { From 1ac9c900f988e8543ac0c691bc427c53c85d00f0 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Wed, 2 Oct 2024 14:53:14 -0400 Subject: [PATCH 15/26] flow-client: Refactor to extract out refresh logic into bare function, and re-use Postgrest connection pool instead of creating a new one on each call to `pg_client()` --- crates/dekaf/src/lib.rs | 7 +- crates/flow-client/src/client.rs | 154 +++++++++++++++---------------- crates/flowctl/src/lib.rs | 3 +- 3 files changed, 84 insertions(+), 80 deletions(-) diff --git a/crates/dekaf/src/lib.rs b/crates/dekaf/src/lib.rs index 8c1016e607..608a78e969 100644 --- a/crates/dekaf/src/lib.rs +++ b/crates/dekaf/src/lib.rs @@ -24,7 +24,10 @@ pub use api_client::KafkaApiClient; use aes_siv::{aead::Aead, Aes256SivAead, KeyInit, KeySizeUser}; use connector::DekafConfig; -use flow_client::{client::RefreshToken, DEFAULT_AGENT_URL}; +use flow_client::{ + client::{refresh_client, RefreshToken}, + DEFAULT_AGENT_URL, +}; use percent_encoding::{percent_decode_str, utf8_percent_encode}; use serde::{Deserialize, Serialize}; use std::time::SystemTime; @@ -77,7 +80,7 @@ impl App { Some(refresh), ); - client.refresh().await?; + refresh_client(&mut client).await?; let claims = client.claims()?; if models::Materialization::regex().is_match(username.as_ref()) { diff --git a/crates/flow-client/src/client.rs b/crates/flow-client/src/client.rs index 002a1b1bd2..f81522055d 100644 --- a/crates/flow-client/src/client.rs +++ b/crates/flow-client/src/client.rs @@ -11,10 +11,6 @@ pub struct Client { agent_endpoint: url::Url, // HTTP client to use for REST requests. http_client: reqwest::Client, - // PostgREST URL. - pg_url: url::Url, - // PostgREST access token. - pg_api_token: String, // User's access token, if authenticated. user_access_token: Option, // User's refresh token, if authenticated. @@ -23,6 +19,10 @@ pub struct Client { shard_client: gazette::shard::Client, // Base journal client which is cloned to build token-specific clients. journal_client: gazette::journal::Client, + // Keep a single Postgrest and hand out clones of it in order to maintain + // a single connection pool. The clones can have different headers while + // still re-using the same connection pool, so this will work across refreshes. + pg_parent: postgrest::Postgrest, } impl Client { @@ -54,8 +54,8 @@ impl Client { Self { agent_endpoint, http_client: reqwest::Client::new(), - pg_api_token, - pg_url, + pg_parent: postgrest::Postgrest::new(pg_url.as_str()) + .insert_header("apikey", pg_api_token.as_str()), journal_client, shard_client, user_access_token, @@ -63,81 +63,15 @@ impl Client { } } - pub async fn refresh(&mut self) -> anyhow::Result<()> { - // Clear expired or soon-to-expire access token - if let Some(_) = &self.user_access_token { - let claims = self.claims()?; - - let now = time::OffsetDateTime::now_utc(); - let exp = time::OffsetDateTime::from_unix_timestamp(claims.exp as i64).unwrap(); - - // Refresh access tokens with plenty of time to spare if we have a - // refresh token. If not, allow refreshing right until the token expires - match ((now - exp).whole_seconds(), &self.user_refresh_token) { - (exp_seconds, Some(_)) if exp_seconds < 60 => self.user_access_token = None, - (exp_seconds, None) if exp_seconds <= 0 => self.user_access_token = None, - _ => {} - } - } - - if self.user_access_token.is_some() && self.user_refresh_token.is_some() { - // Authorization is current: nothing to do. - Ok(()) - } else if self.user_access_token.is_some() { - // We have an access token but no refresh token. Create one. - let refresh_token = api_exec::( - self.rpc( - "create_refresh_token", - serde_json::json!({"multi_use": true, "valid_for": "90d", "detail": "Created by flowctl"}) - .to_string(), - ), - ) - .await?; - - self.user_refresh_token = Some(refresh_token); - - tracing::info!("created new refresh token"); - Ok(()) - } else if let Some(RefreshToken { id, secret }) = &self.user_refresh_token { - // We have a refresh token but no access token. Generate one. - - #[derive(serde::Deserialize)] - struct Response { - access_token: String, - refresh_token: Option, // Set iff the token was single-use. - } - let Response { - access_token, - refresh_token: next_refresh_token, - } = api_exec::(self.rpc( - "generate_access_token", - serde_json::json!({"refresh_token_id": id, "secret": secret}).to_string(), - )) - .await - .context("failed to obtain access token")?; - - if next_refresh_token.is_some() { - self.user_refresh_token = next_refresh_token; - } - - self.user_access_token = Some(access_token); - - tracing::info!("generated a new access token"); - Ok(()) - } else { - anyhow::bail!("Client not authenticated"); - } - } - pub fn pg_client(&self) -> postgrest::Postgrest { - let pg_client = postgrest::Postgrest::new(self.pg_url.as_str()) - .insert_header("apikey", self.pg_api_token.as_str()); - if let Some(token) = &self.user_access_token { - return pg_client.insert_header("Authorization", &format!("Bearer {token}")); + return self + .pg_parent + .clone() + .insert_header("Authorization", &format!("Bearer {token}")); } - pg_client + self.pg_parent.clone() } pub fn claims(&self) -> anyhow::Result { @@ -331,3 +265,69 @@ pub async fn fetch_collection_authorization( Ok((journal_name_prefix, journal_client)) } + +pub async fn refresh_client(client: &mut Client) -> anyhow::Result<()> { + // Clear expired or soon-to-expire access token + if let Some(_) = &client.user_access_token { + let claims = client.claims()?; + + let now = time::OffsetDateTime::now_utc(); + let exp = time::OffsetDateTime::from_unix_timestamp(claims.exp as i64).unwrap(); + + // Refresh access tokens with plenty of time to spare if we have a + // refresh token. If not, allow refreshing right until the token expires + match ((now - exp).whole_seconds(), &client.user_refresh_token) { + (exp_seconds, Some(_)) if exp_seconds < 60 => client.user_access_token = None, + (exp_seconds, None) if exp_seconds <= 0 => client.user_access_token = None, + _ => {} + } + } + + if client.user_access_token.is_some() && client.user_refresh_token.is_some() { + // Authorization is current: nothing to do. + Ok(()) + } else if client.user_access_token.is_some() { + // We have an access token but no refresh token. Create one. + let refresh_token = api_exec::( + client.rpc( + "create_refresh_token", + serde_json::json!({"multi_use": true, "valid_for": "90d", "detail": "Created by flowctl"}) + .to_string(), + ), + ) + .await?; + + client.user_refresh_token = Some(refresh_token); + + tracing::info!("created new refresh token"); + Ok(()) + } else if let Some(RefreshToken { id, secret }) = &client.user_refresh_token { + // We have a refresh token but no access token. Generate one. + + #[derive(serde::Deserialize)] + struct Response { + access_token: String, + refresh_token: Option, // Set iff the token was single-use. + } + let Response { + access_token, + refresh_token: next_refresh_token, + } = api_exec::(client.rpc( + "generate_access_token", + serde_json::json!({"refresh_token_id": id, "secret": secret}).to_string(), + )) + .await + .context("failed to obtain access token")?; + + if next_refresh_token.is_some() { + client.user_refresh_token = next_refresh_token; + } + + client.user_access_token = Some(access_token); + + tracing::info!("generated a new access token"); + Ok(()) + } else { + anyhow::bail!("Client not authenticated"); + } +} diff --git a/crates/flowctl/src/lib.rs b/crates/flowctl/src/lib.rs index 9ca8ee67e5..45976a668e 100644 --- a/crates/flowctl/src/lib.rs +++ b/crates/flowctl/src/lib.rs @@ -16,6 +16,7 @@ mod poll; mod preview; mod raw; +use flow_client::client::refresh_client; pub(crate) use flow_client::client::Client; pub(crate) use flow_client::{api_exec, api_exec_paginated, parse_jwt_claims}; use output::{Output, OutputType}; @@ -151,7 +152,7 @@ impl Cli { let mut client: flow_client::Client = config.build_client(); if config.user_access_token.is_some() || config.user_refresh_token.is_some() { - client.refresh().await?; + refresh_client(&mut client).await?; } else { tracing::warn!("You are not authenticated. Run `auth login` to login to Flow."); } From fdd7c1a6ab9c172047847783b9eed8a3fe726fcf Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Thu, 3 Oct 2024 11:50:15 -0400 Subject: [PATCH 16/26] dekaf: Always decrement `total_connections`, even if error --- crates/dekaf/src/main.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/crates/dekaf/src/main.rs b/crates/dekaf/src/main.rs index 6fcfd2e3c7..a089415128 100644 --- a/crates/dekaf/src/main.rs +++ b/crates/dekaf/src/main.rs @@ -266,6 +266,7 @@ where let mut out = bytes::BytesMut::new(); let mut raw_sasl_auth = false; + let mut res = Ok(()); while let Some(frame) = r.try_next().await? { if let err @ Err(_) = dekaf::dispatch_request_frame(&mut session, &mut raw_sasl_auth, frame, &mut out) @@ -273,13 +274,13 @@ where { // Close the connection on error w.shutdown().await?; - return err; + res = err; } () = w.write_all(&mut out).await?; out.clear(); } - Ok(()) + res } .await; From 14c50a8c91ce3558aafdf1ff371ba9627aa11647 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Thu, 3 Oct 2024 14:17:33 -0400 Subject: [PATCH 17/26] flow_client: Refactor to expose refreshed credentials `flowctl` needs these credentials exposed in order to store them in `flowctl::Config` + on disk for the next flowctl invocation --- crates/dekaf/src/lib.rs | 29 +++---- crates/dekaf/src/main.rs | 13 ++- crates/flow-client/src/client.rs | 130 +++++++++++++++------------- crates/flowctl/src/config.rs | 6 +- crates/flowctl/src/lib.rs | 31 +++---- crates/models/src/authorizations.rs | 10 +++ 6 files changed, 116 insertions(+), 103 deletions(-) diff --git a/crates/dekaf/src/lib.rs b/crates/dekaf/src/lib.rs index 608a78e969..bc1c54f73f 100644 --- a/crates/dekaf/src/lib.rs +++ b/crates/dekaf/src/lib.rs @@ -24,14 +24,10 @@ pub use api_client::KafkaApiClient; use aes_siv::{aead::Aead, Aes256SivAead, KeyInit, KeySizeUser}; use connector::DekafConfig; -use flow_client::{ - client::{refresh_client, RefreshToken}, - DEFAULT_AGENT_URL, -}; +use flow_client::client::{refresh_authorizations, RefreshToken}; use percent_encoding::{percent_decode_str, utf8_percent_encode}; use serde::{Deserialize, Serialize}; use std::time::SystemTime; -use url::Url; pub struct App { /// Hostname which is advertised for Kafka access. @@ -42,10 +38,8 @@ pub struct App { pub kafka_client: KafkaApiClient, /// Secret used to secure Prometheus endpoint pub secret: String, - /// Supabase endpoint - pub api_endpoint: Url, - /// Supabase api key - pub api_key: String, + /// Share a single base client in order to re-use connection pools + pub client_base: flow_client::Client, } #[derive(Debug, Clone, Serialize, Deserialize)] @@ -72,16 +66,15 @@ impl App { let raw_token = String::from_utf8(base64::decode(password)?.to_vec())?; let refresh: RefreshToken = serde_json::from_str(raw_token.as_str())?; - let mut client = flow_client::Client::new( - DEFAULT_AGENT_URL.to_owned(), - self.api_key.to_owned(), - self.api_endpoint.to_owned(), - None, - Some(refresh), - ); + let (access, refresh) = + refresh_authorizations(&self.client_base, None, Some(refresh)).await?; + + let client = self + .client_base + .clone() + .with_creds(Some(access), Some(refresh)); - refresh_client(&mut client).await?; - let claims = client.claims()?; + let claims = flow_client::client::client_claims(&client)?; if models::Materialization::regex().is_match(username.as_ref()) { Ok(Authenticated { diff --git a/crates/dekaf/src/main.rs b/crates/dekaf/src/main.rs index a089415128..5d8cb1b009 100644 --- a/crates/dekaf/src/main.rs +++ b/crates/dekaf/src/main.rs @@ -5,7 +5,9 @@ use anyhow::{bail, Context}; use axum_server::tls_rustls::RustlsConfig; use clap::{Args, Parser}; use dekaf::{KafkaApiClient, Session}; -use flow_client::{DEFAULT_PG_PUBLIC_TOKEN, DEFAULT_PG_URL, LOCAL_PG_PUBLIC_TOKEN, LOCAL_PG_URL}; +use flow_client::{ + DEFAULT_AGENT_URL, DEFAULT_PG_PUBLIC_TOKEN, DEFAULT_PG_URL, LOCAL_PG_PUBLIC_TOKEN, LOCAL_PG_URL, +}; use futures::{FutureExt, TryStreamExt}; use rsasl::config::SASLConfig; use rustls::pki_types::CertificateDer; @@ -133,8 +135,13 @@ async fn main() -> anyhow::Result<()> { "failed to connect or authenticate to upstream Kafka broker used for serving group management APIs", )?, secret: cli.encryption_secret.to_owned(), - api_endpoint, - api_key + client_base: flow_client::Client::new( + DEFAULT_AGENT_URL.to_owned(), + api_key, + api_endpoint, + None, + None + ) }); tracing::info!( diff --git a/crates/flow-client/src/client.rs b/crates/flow-client/src/client.rs index f81522055d..33c9d62b7a 100644 --- a/crates/flow-client/src/client.rs +++ b/crates/flow-client/src/client.rs @@ -32,7 +32,7 @@ impl Client { pg_api_token: String, pg_url: Url, user_access_token: Option, - userrefresh_token: Option, + user_refresh_token: Option, ) -> Self { // Build journal and shard clients with an empty default service address. // We'll use their with_endpoint_and_metadata() routines to cheaply clone @@ -59,7 +59,19 @@ impl Client { journal_client, shard_client, user_access_token, - user_refresh_token: userrefresh_token, + user_refresh_token, + } + } + + pub fn with_creds( + self, + user_access_token: Option, + user_refresh_token: Option, + ) -> Self { + Self { + user_access_token: user_access_token.or(self.user_access_token), + user_refresh_token: user_refresh_token.or(self.user_refresh_token), + ..self } } @@ -74,15 +86,6 @@ impl Client { self.pg_parent.clone() } - pub fn claims(&self) -> anyhow::Result { - parse_jwt_claims( - self.user_access_token - .as_ref() - .ok_or(anyhow::anyhow!("Client is not authenticated"))? - .as_str(), - ) - } - pub fn from(&self, table: &str) -> postgrest::Builder { self.pg_client().from(table) } @@ -266,29 +269,34 @@ pub async fn fetch_collection_authorization( Ok((journal_name_prefix, journal_client)) } -pub async fn refresh_client(client: &mut Client) -> anyhow::Result<()> { +pub async fn refresh_authorizations( + client: &Client, + access_token: Option, + refresh_token: Option, +) -> anyhow::Result<(String, RefreshToken)> { // Clear expired or soon-to-expire access token - if let Some(_) = &client.user_access_token { - let claims = client.claims()?; - - let now = time::OffsetDateTime::now_utc(); - let exp = time::OffsetDateTime::from_unix_timestamp(claims.exp as i64).unwrap(); + let access_token = if let Some(token) = &access_token { + let claims: ControlClaims = parse_jwt_claims(token.as_str())?; // Refresh access tokens with plenty of time to spare if we have a // refresh token. If not, allow refreshing right until the token expires - match ((now - exp).whole_seconds(), &client.user_refresh_token) { - (exp_seconds, Some(_)) if exp_seconds < 60 => client.user_access_token = None, - (exp_seconds, None) if exp_seconds <= 0 => client.user_access_token = None, - _ => {} + match (claims.time_remaining().whole_seconds(), &refresh_token) { + (exp_seconds, Some(_)) if exp_seconds < 60 => None, + (exp_seconds, None) if exp_seconds <= 0 => None, + _ => Some(token.to_owned()), } - } + } else { + None + }; - if client.user_access_token.is_some() && client.user_refresh_token.is_some() { - // Authorization is current: nothing to do. - Ok(()) - } else if client.user_access_token.is_some() { - // We have an access token but no refresh token. Create one. - let refresh_token = api_exec::( + match (access_token, refresh_token) { + (Some(access), Some(refresh)) => { + // Authorization is current: nothing to do. + Ok((access, refresh)) + } + (Some(access), None) => { + // We have an access token but no refresh token. Create one. + let refresh_token = api_exec::( client.rpc( "create_refresh_token", serde_json::json!({"multi_use": true, "valid_for": "90d", "detail": "Created by flowctl"}) @@ -297,37 +305,43 @@ pub async fn refresh_client(client: &mut Client) -> anyhow::Result<()> { ) .await?; - client.user_refresh_token = Some(refresh_token); - - tracing::info!("created new refresh token"); - Ok(()) - } else if let Some(RefreshToken { id, secret }) = &client.user_refresh_token { - // We have a refresh token but no access token. Generate one. - - #[derive(serde::Deserialize)] - struct Response { - access_token: String, - refresh_token: Option, // Set iff the token was single-use. + tracing::info!("created new refresh token"); + Ok((access, refresh_token)) } - let Response { - access_token, - refresh_token: next_refresh_token, - } = api_exec::(client.rpc( - "generate_access_token", - serde_json::json!({"refresh_token_id": id, "secret": secret}).to_string(), - )) - .await - .context("failed to obtain access token")?; - - if next_refresh_token.is_some() { - client.user_refresh_token = next_refresh_token; + (None, Some(RefreshToken { id, secret })) => { + // We have a refresh token but no access token. Generate one. + + #[derive(serde::Deserialize)] + struct Response { + access_token: String, + refresh_token: Option, // Set iff the token was single-use. + } + let Response { + access_token, + refresh_token: next_refresh_token, + } = api_exec::(client.rpc( + "generate_access_token", + serde_json::json!({"refresh_token_id": id, "secret": secret}).to_string(), + )) + .await + .context("failed to obtain access token")?; + + tracing::info!("generated a new access token"); + Ok(( + access_token, + next_refresh_token.unwrap_or(RefreshToken { id, secret }), + )) } - - client.user_access_token = Some(access_token); - - tracing::info!("generated a new access token"); - Ok(()) - } else { - anyhow::bail!("Client not authenticated"); + _ => anyhow::bail!("Client not authenticated"), } } + +pub fn client_claims(client: &Client) -> anyhow::Result { + parse_jwt_claims( + client + .user_access_token + .as_ref() + .ok_or(anyhow::anyhow!("Client is not authenticated"))? + .as_str(), + ) +} diff --git a/crates/flowctl/src/config.rs b/crates/flowctl/src/config.rs index 9cb7660581..afbaf77dd5 100644 --- a/crates/flowctl/src/config.rs +++ b/crates/flowctl/src/config.rs @@ -181,13 +181,13 @@ impl Config { Ok(()) } - pub fn build_client(&self) -> flow_client::Client { + pub fn build_anon_client(&self) -> flow_client::Client { flow_client::Client::new( self.get_agent_url().clone(), self.get_pg_public_token().to_string(), self.get_pg_url().clone(), - self.user_access_token.clone(), - self.user_refresh_token.clone(), + None, + None, ) } diff --git a/crates/flowctl/src/lib.rs b/crates/flowctl/src/lib.rs index 45976a668e..00565e0849 100644 --- a/crates/flowctl/src/lib.rs +++ b/crates/flowctl/src/lib.rs @@ -1,6 +1,5 @@ use std::fmt::Debug; -use anyhow::Context; use clap::Parser; mod auth; @@ -16,9 +15,9 @@ mod poll; mod preview; mod raw; -use flow_client::client::refresh_client; +use flow_client::client::refresh_authorizations; pub(crate) use flow_client::client::Client; -pub(crate) use flow_client::{api_exec, api_exec_paginated, parse_jwt_claims}; +pub(crate) use flow_client::{api_exec, api_exec_paginated}; use output::{Output, OutputType}; use poll::poll_while_queued; @@ -135,27 +134,17 @@ impl Cli { let mut config = config::Config::load(&self.profile)?; let output = self.output.clone(); - // If the configured access token has expired then remove it before continuing. - if let Some(token) = &config.user_access_token { - let claims: models::authorizations::ControlClaims = - parse_jwt_claims(token).context("failed to parse control-plane access token")?; + let client: flow_client::Client = config.build_anon_client(); - let now = time::OffsetDateTime::now_utc(); - let exp = time::OffsetDateTime::from_unix_timestamp(claims.exp as i64).unwrap(); + let (access, refresh) = + refresh_authorizations(&client, config.user_access_token, config.user_refresh_token) + .await?; - if now + std::time::Duration::from_secs(60) > exp { - tracing::info!(expired=%exp, "removing expired user access token from configuration"); - config.user_access_token = None; - } - } - - let mut client: flow_client::Client = config.build_client(); + // Make sure to store refreshed tokens back in Config so they get written back to disk + config.user_access_token = Some(access.to_owned()); + config.user_refresh_token = Some(refresh.to_owned()); - if config.user_access_token.is_some() || config.user_refresh_token.is_some() { - refresh_client(&mut client).await?; - } else { - tracing::warn!("You are not authenticated. Run `auth login` to login to Flow."); - } + let client = client.with_creds(Some(access), Some(refresh)); let mut context = CliContext { client, diff --git a/crates/models/src/authorizations.rs b/crates/models/src/authorizations.rs index 53bf2d66d5..87f5b28af6 100644 --- a/crates/models/src/authorizations.rs +++ b/crates/models/src/authorizations.rs @@ -1,3 +1,4 @@ +use std::cmp::max; use validator::Validate; /// ControlClaims are claims encoded within control-plane access tokens. @@ -11,6 +12,15 @@ pub struct ControlClaims { pub exp: u64, } +impl ControlClaims { + pub fn time_remaining(&self) -> time::Duration { + let now = time::OffsetDateTime::now_utc(); + let exp = time::OffsetDateTime::from_unix_timestamp(self.exp as i64).unwrap(); + + max(exp - now, time::Duration::ZERO) + } +} + // Data-plane claims are represented by proto_gazette::Claims, // which is not re-exported by this crate. From 063f2862f943ae9411725675dcfd24aecaaa09e1 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Thu, 3 Oct 2024 18:34:40 -0400 Subject: [PATCH 18/26] dekaf: Expose more useful api histogram buckets --- Cargo.lock | 109 ++++++++++++++++++++++------- Cargo.toml | 2 +- crates/dekaf/Cargo.toml | 2 +- crates/dekaf/src/lib.rs | 2 +- crates/dekaf/src/main.rs | 4 +- crates/dekaf/src/metrics_server.rs | 26 ++++--- 6 files changed, 103 insertions(+), 42 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8278c6449d..4c3ba7426e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1838,7 +1838,7 @@ dependencies = [ "lz4_flex", "md5", "metrics", - "metrics-prometheus", + "metrics-exporter-prometheus", "models", "ops", "percent-encoding", @@ -1850,7 +1850,7 @@ dependencies = [ "regex", "rsasl", "rustls 0.23.10", - "rustls-native-certs", + "rustls-native-certs 0.7.2", "rustls-pemfile 2.1.3", "schemars", "serde", @@ -2972,6 +2972,25 @@ dependencies = [ "tokio-rustls 0.24.1", ] +[[package]] +name = "hyper-rustls" +version = "0.27.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08afdbb5c31130e3034af566421053ab03787c640246a446327f550d11bcb333" +dependencies = [ + "futures-util", + "http 1.1.0", + "hyper 1.4.1", + "hyper-util", + "log", + "rustls 0.23.10", + "rustls-native-certs 0.8.0", + "rustls-pki-types", + "tokio", + "tokio-rustls 0.26.0", + "tower-service", +] + [[package]] name = "hyper-timeout" version = "0.5.1" @@ -3467,7 +3486,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4979f22fdb869068da03c9f7528f8297c6fd2606bc3a4affe42e6a823fdb8da4" dependencies = [ "cfg-if 1.0.0", - "windows-targets 0.48.5", + "windows-targets 0.52.6", ] [[package]] @@ -3683,19 +3702,24 @@ dependencies = [ ] [[package]] -name = "metrics-prometheus" -version = "0.7.0" +name = "metrics-exporter-prometheus" +version = "0.15.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "51557a875fdbd5b953b698ecd6cd06efef47618e02d95ad912e2392e5b5617ff" +checksum = "b4f0c8427b39666bf970460908b213ec09b3b350f20c0c2eabcbba51704a08e6" dependencies = [ - "arc-swap", + "base64 0.22.1", + "http-body-util", + "hyper 1.4.1", + "hyper-rustls 0.27.3", + "hyper-util", + "indexmap 2.3.0", + "ipnet", "metrics", "metrics-util", - "once_cell", - "prometheus", - "sealed", - "smallvec", + "quanta", "thiserror", + "tokio", + "tracing", ] [[package]] @@ -3709,6 +3733,8 @@ dependencies = [ "hashbrown 0.14.5", "metrics", "num_cpus", + "quanta", + "sketches-ddsketch", ] [[package]] @@ -4782,6 +4808,21 @@ version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "658fa1faf7a4cc5f057c9ee5ef560f717ad9d8dc66d975267f709624d6e1ab88" +[[package]] +name = "quanta" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e5167a477619228a0b284fac2674e3c388cba90631d7b7de620e6f1fcd08da5" +dependencies = [ + "crossbeam-utils", + "libc", + "once_cell", + "raw-cpuid", + "wasi 0.11.0+wasi-snapshot-preview1", + "web-sys", + "winapi", +] + [[package]] name = "quickcheck" version = "1.0.3" @@ -4944,6 +4985,15 @@ dependencies = [ "rand_core 0.5.1", ] +[[package]] +name = "raw-cpuid" +version = "11.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ab240315c661615f2ee9f0f2cd32d5a7343a84d5ebcccb99d46e6637565e7b0" +dependencies = [ + "bitflags 2.6.0", +] + [[package]] name = "rayon" version = "1.10.0" @@ -5094,7 +5144,7 @@ dependencies = [ "http 0.2.12", "http-body 0.4.6", "hyper 0.14.30", - "hyper-rustls", + "hyper-rustls 0.24.2", "ipnet", "js-sys", "log", @@ -5341,6 +5391,19 @@ dependencies = [ "security-framework", ] +[[package]] +name = "rustls-native-certs" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcaf18a4f2be7326cd874a5fa579fae794320a0f388d365dca7e480e55f83f8a" +dependencies = [ + "openssl-probe", + "rustls-pemfile 2.1.3", + "rustls-pki-types", + "schannel", + "security-framework", +] + [[package]] name = "rustls-pemfile" version = "1.0.4" @@ -5522,18 +5585,6 @@ version = "4.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1c107b6f4780854c8b126e228ea8869f4d7b71260f962fefb57b996b8959ba6b" -[[package]] -name = "sealed" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4a8caec23b7800fb97971a1c6ae365b6239aaeddfb934d6265f8505e795699d" -dependencies = [ - "heck 0.4.1", - "proc-macro2", - "quote", - "syn 2.0.74", -] - [[package]] name = "security-framework" version = "2.11.1" @@ -5829,6 +5880,12 @@ version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9fed904c7fb2856d868b92464fc8fa597fce366edea1a9cbfaa8cb5fe080bd6d" +[[package]] +name = "sketches-ddsketch" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85636c14b73d81f541e525f585c0a2109e6744e1565b5c1668e31c70c10ed65c" + [[package]] name = "slab" version = "0.4.9" @@ -6487,7 +6544,7 @@ dependencies = [ "percent-encoding", "pin-project", "prost", - "rustls-native-certs", + "rustls-native-certs 0.7.2", "rustls-pemfile 2.1.3", "socket2", "tokio", @@ -6699,7 +6756,7 @@ version = "1.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" dependencies = [ - "cfg-if 0.1.10", + "cfg-if 1.0.0", "static_assertions", ] diff --git a/Cargo.toml b/Cargo.toml index 61dcd1dfe7..f7aa5b2dc6 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -86,7 +86,7 @@ lz4_flex = "0.11.0" mime = "0.3" memchr = "2.5" metrics = "0.23.0" -metrics-prometheus = "0.7.0" +metrics-exporter-prometheus = "0.15.3" prometheus = "0.13.4" md5 = "0.7.0" num-bigint = "0.4" diff --git a/crates/dekaf/Cargo.toml b/crates/dekaf/Cargo.toml index e9acc91a4c..c33527eda7 100644 --- a/crates/dekaf/Cargo.toml +++ b/crates/dekaf/Cargo.toml @@ -40,7 +40,7 @@ kafka-protocol = { workspace = true } lz4_flex = { workspace = true } md5 = { workspace = true } metrics = { workspace = true } -metrics-prometheus = { workspace = true } +metrics-exporter-prometheus = { workspace = true } percent-encoding = { workspace = true } postgrest = { workspace = true } prometheus = { workspace = true } diff --git a/crates/dekaf/src/lib.rs b/crates/dekaf/src/lib.rs index bc1c54f73f..bc254a47e2 100644 --- a/crates/dekaf/src/lib.rs +++ b/crates/dekaf/src/lib.rs @@ -302,7 +302,7 @@ async fn handle_api( let handle_duration = SystemTime::now().duration_since(start_time)?; metrics::histogram!("api_call_time", "api_key" => format!("{:?}",api_key)) - .record(handle_duration.as_millis() as f64); + .record(handle_duration.as_secs_f32() as f64); ret } diff --git a/crates/dekaf/src/main.rs b/crates/dekaf/src/main.rs index 5d8cb1b009..d60f6b046e 100644 --- a/crates/dekaf/src/main.rs +++ b/crates/dekaf/src/main.rs @@ -105,8 +105,6 @@ async fn main() -> anyhow::Result<()> { .with_writer(std::io::stderr) .init(); - metrics_prometheus::install(); - let cli = Cli::parse(); tracing::info!("Starting dekaf"); @@ -163,7 +161,7 @@ async fn main() -> anyhow::Result<()> { .await .context("failed to bind server port")?; - let metrics_router = dekaf::metrics_server::build_router(app.clone()); + let metrics_router = dekaf::metrics_server::build_router(); let metrics_server_task = axum_server::bind(metrics_addr).serve(metrics_router.into_make_service()); tokio::spawn(async move { metrics_server_task.await.unwrap() }); diff --git a/crates/dekaf/src/metrics_server.rs b/crates/dekaf/src/metrics_server.rs index 7fec64eb6d..58e234b5da 100644 --- a/crates/dekaf/src/metrics_server.rs +++ b/crates/dekaf/src/metrics_server.rs @@ -1,13 +1,21 @@ -use super::App; -use std::sync::Arc; +use metrics_exporter_prometheus::{PrometheusBuilder, PrometheusHandle}; -pub fn build_router(app: Arc) -> axum::Router<()> { +pub fn build_router() -> axum::Router<()> { use axum::routing::get; + let prom = PrometheusBuilder::new() + .set_buckets( + &prometheus::exponential_buckets(0.00001, 2.5, 15) + .expect("calculating histogram buckets"), + ) + .expect("calculating histogram buckets") + .install_recorder() + .expect("failed to install prometheus recorder"); + let schema_router = axum::Router::new() .route("/metrics", get(prometheus_metrics)) .layer(tower_http::trace::TraceLayer::new_for_http()) - .with_state(app); + .with_state(prom); schema_router } @@ -23,12 +31,10 @@ fn record_jemalloc_stats() { } #[tracing::instrument(skip_all)] -async fn prometheus_metrics() -> (axum::http::StatusCode, String) { +async fn prometheus_metrics( + axum::extract::State(prom_handle): axum::extract::State, +) -> (axum::http::StatusCode, String) { record_jemalloc_stats(); - match prometheus::TextEncoder::new().encode_to_string(&prometheus::default_registry().gather()) - { - Err(e) => (axum::http::StatusCode::INTERNAL_SERVER_ERROR, e.to_string()), - Ok(result) => (axum::http::StatusCode::OK, result), - } + (axum::http::StatusCode::OK, prom_handle.render()) } From d55ecee5736abe91321339015bc49d67b46cb8d0 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Fri, 4 Oct 2024 10:34:21 -0400 Subject: [PATCH 19/26] flowctl: Fix running unauthenticated --- crates/flowctl/src/lib.rs | 29 ++++++++++++++++++----------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/crates/flowctl/src/lib.rs b/crates/flowctl/src/lib.rs index 00565e0849..9ed6b55b9b 100644 --- a/crates/flowctl/src/lib.rs +++ b/crates/flowctl/src/lib.rs @@ -134,17 +134,24 @@ impl Cli { let mut config = config::Config::load(&self.profile)?; let output = self.output.clone(); - let client: flow_client::Client = config.build_anon_client(); - - let (access, refresh) = - refresh_authorizations(&client, config.user_access_token, config.user_refresh_token) - .await?; - - // Make sure to store refreshed tokens back in Config so they get written back to disk - config.user_access_token = Some(access.to_owned()); - config.user_refresh_token = Some(refresh.to_owned()); - - let client = client.with_creds(Some(access), Some(refresh)); + let anon_client: flow_client::Client = config.build_anon_client(); + + let client = if let Ok((access, refresh)) = refresh_authorizations( + &anon_client, + config.user_access_token.to_owned(), + config.user_refresh_token.to_owned(), + ) + .await + { + // Make sure to store refreshed tokens back in Config so they get written back to disk + config.user_access_token = Some(access.to_owned()); + config.user_refresh_token = Some(refresh.to_owned()); + + anon_client.with_creds(Some(access), Some(refresh)) + } else { + tracing::warn!("You are not authenticated. Run `auth login` to login to Flow."); + anon_client + }; let mut context = CliContext { client, From 02db74694e2bd0d441a072f2ae12492a17a61840 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Fri, 4 Oct 2024 12:09:57 -0400 Subject: [PATCH 20/26] dekaf: Enable tuning connection pool size --- crates/dekaf/src/api_client.rs | 14 ++++++++++++-- crates/dekaf/src/main.rs | 6 ++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/crates/dekaf/src/api_client.rs b/crates/dekaf/src/api_client.rs index 17ae6133c6..263617cd3f 100644 --- a/crates/dekaf/src/api_client.rs +++ b/crates/dekaf/src/api_client.rs @@ -345,7 +345,12 @@ impl KafkaApiClient { return Ok(client.clone()); } - let new_client = Self::connect(broker_url, self.sasl_config.clone()).await?; + let new_client = Self::connect( + broker_url, + self.sasl_config.clone(), + self.pool.status().max_size, + ) + .await?; clients.insert(broker_url.to_owned(), new_client.clone()); @@ -353,11 +358,16 @@ impl KafkaApiClient { } #[instrument(name = "api_client_connect", skip(sasl_config))] - pub async fn connect(broker_url: &str, sasl_config: Arc) -> anyhow::Result { + pub async fn connect( + broker_url: &str, + sasl_config: Arc, + pool_size: usize, + ) -> anyhow::Result { let pool = Pool::builder(KafkaConnectionParams { broker_url: broker_url.to_owned(), sasl_config: sasl_config.clone(), }) + .max_size(pool_size) .build()?; // Close idle connections, and any free connection older than 30m. diff --git a/crates/dekaf/src/main.rs b/crates/dekaf/src/main.rs index d60f6b046e..338c9efeae 100644 --- a/crates/dekaf/src/main.rs +++ b/crates/dekaf/src/main.rs @@ -77,6 +77,11 @@ pub struct Cli { #[arg(long, env = "ENCRYPTION_SECRET")] encryption_secret: String, + /// The maximum number of connections to a particular upstream kafka broker that can be + /// open at any one time. These connections will be pooled and shared between all requests. + #[arg(long, env = "BROKER_CONNECTION_POOL_SIZE", default_value = "20")] + broker_connection_pool_size: usize, + #[command(flatten)] tls: Option, } @@ -129,6 +134,7 @@ async fn main() -> anyhow::Result<()> { cli.default_broker_username, cli.default_broker_password, )?, + cli.broker_connection_pool_size ).await.context( "failed to connect or authenticate to upstream Kafka broker used for serving group management APIs", )?, From 1ce6a586db7e56f359bde5065371333aa6ff70cf Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Fri, 4 Oct 2024 14:18:44 -0400 Subject: [PATCH 21/26] dekaf: add more metrics and break out `dekaf_request_time` by request API key --- crates/dekaf/src/api_client.rs | 34 +++++++++++++++++++++++------- crates/dekaf/src/lib.rs | 2 +- crates/dekaf/src/main.rs | 4 ++-- crates/dekaf/src/metrics_server.rs | 12 +++++------ crates/dekaf/src/read.rs | 4 ++-- 5 files changed, 37 insertions(+), 19 deletions(-) diff --git a/crates/dekaf/src/api_client.rs b/crates/dekaf/src/api_client.rs index 263617cd3f..ea647abbd9 100644 --- a/crates/dekaf/src/api_client.rs +++ b/crates/dekaf/src/api_client.rs @@ -3,12 +3,19 @@ use bytes::{Bytes, BytesMut}; use futures::{SinkExt, TryStreamExt}; use kafka_protocol::{ error::ParseResponseErrorCode, - messages, + messages::{self, ApiKey}, protocol::{self, Decodable, Encodable, Request}, }; use rsasl::{config::SASLConfig, mechname::Mechname, prelude::SASLClient}; use rustls::RootCertStore; -use std::{boxed::Box, cell::Cell, collections::HashMap, fmt::Debug, io, time::Duration}; +use std::{ + boxed::Box, + cell::Cell, + collections::HashMap, + fmt::Debug, + io, + time::{Duration, SystemTime}, +}; use std::{io::BufWriter, pin::Pin, sync::Arc}; use tokio::sync::OnceCell; use tokio::sync::RwLock; @@ -384,11 +391,11 @@ impl KafkaApiClient { loop { let pool_state = pool.status(); - metrics::gauge!("pool_size", "upstream_broker" => broker_url.to_owned()) + metrics::gauge!("dekaf_pool_size", "upstream_broker" => broker_url.to_owned()) .set(pool_state.size as f64); - metrics::gauge!("pool_available", "upstream_broker" => broker_url.to_owned()) + metrics::gauge!("dekaf_pool_available", "upstream_broker" => broker_url.to_owned()) .set(pool_state.available as f64); - metrics::gauge!("pool_waiting", "upstream_broker" => broker_url.to_owned()) + metrics::gauge!("dekaf_pool_waiting", "upstream_broker" => broker_url.to_owned()) .set(pool_state.waiting as f64); let age_sum = Cell::new(Duration::ZERO); @@ -402,8 +409,8 @@ impl KafkaApiClient { metrics.age() < max_age && metrics.last_used() < max_idle }); - metrics::gauge!("pool_connection_avg_age", "upstream_broker" => broker_url.to_owned()).set(if connections.get() > 0 { age_sum.get()/connections.get() } else { Duration::ZERO }); - metrics::gauge!("pool_connection_avg_idle", "upstream_broker" => broker_url.to_owned()).set(if connections.get() > 0 { idle_sum.get()/connections.get() } else { Duration::ZERO }); + metrics::gauge!("dekaf_pool_connection_avg_age", "upstream_broker" => broker_url.to_owned()).set(if connections.get() > 0 { age_sum.get()/connections.get() } else { Duration::ZERO }); + metrics::gauge!("dekaf_pool_connection_avg_idle", "upstream_broker" => broker_url.to_owned()).set(if connections.get() > 0 { idle_sum.get()/connections.get() } else { Duration::ZERO }); } } })); @@ -439,6 +446,7 @@ impl KafkaApiClient { req: Req, header: Option, ) -> anyhow::Result { + let start_time = SystemTime::now(); // TODO: This could be optimized by pipelining. let mut conn = match self.pool.get().await { Ok(c) => c, @@ -448,7 +456,17 @@ impl KafkaApiClient { } }; - send_request(conn.as_mut(), req, header).await + metrics::histogram!("dekaf_pool_wait_time", "upstream_broker" => self.url.to_owned()) + .record(SystemTime::now().duration_since(start_time)?); + + let api_key = ApiKey::try_from(Req::KEY).expect("should be valid api key"); + + let start_time = SystemTime::now(); + let resp = send_request(conn.as_mut(), req, header).await; + metrics::histogram!("dekaf_request_time", "api_key" => format!("{:?}",api_key), "upstream_broker" => self.url.to_owned()) + .record(SystemTime::now().duration_since(start_time)?); + + resp } #[instrument(skip(self))] diff --git a/crates/dekaf/src/lib.rs b/crates/dekaf/src/lib.rs index bc254a47e2..d7bfdc4d2c 100644 --- a/crates/dekaf/src/lib.rs +++ b/crates/dekaf/src/lib.rs @@ -301,7 +301,7 @@ async fn handle_api( }; let handle_duration = SystemTime::now().duration_since(start_time)?; - metrics::histogram!("api_call_time", "api_key" => format!("{:?}",api_key)) + metrics::histogram!("dekaf_api_call_time", "api_key" => format!("{:?}",api_key)) .record(handle_duration.as_secs_f32() as f64); ret diff --git a/crates/dekaf/src/main.rs b/crates/dekaf/src/main.rs index 338c9efeae..58221250c5 100644 --- a/crates/dekaf/src/main.rs +++ b/crates/dekaf/src/main.rs @@ -262,7 +262,7 @@ where S: AsyncRead + AsyncWrite + Unpin, { tracing::info!("accepted client connection"); - metrics::gauge!("total_connections").increment(1); + metrics::gauge!("dekaf_total_connections").increment(1); let result = async { let (r, mut w) = split(socket); @@ -295,7 +295,7 @@ where } .await; - metrics::gauge!("total_connections").decrement(1); + metrics::gauge!("dekaf_total_connections").decrement(1); result } diff --git a/crates/dekaf/src/metrics_server.rs b/crates/dekaf/src/metrics_server.rs index 58e234b5da..fffefa5354 100644 --- a/crates/dekaf/src/metrics_server.rs +++ b/crates/dekaf/src/metrics_server.rs @@ -22,12 +22,12 @@ pub fn build_router() -> axum::Router<()> { fn record_jemalloc_stats() { let stats = allocator::current_mem_stats(); - metrics::gauge!("mem_allocated").set(stats.allocated as f64); - metrics::gauge!("mem_mapped").set(stats.mapped as f64); - metrics::gauge!("mem_metadata").set(stats.metadata as f64); - metrics::gauge!("mem_resident").set(stats.resident as f64); - metrics::gauge!("mem_retained").set(stats.retained as f64); - metrics::gauge!("mem_active").set(stats.active as f64); + metrics::gauge!("dekaf_mem_allocated").set(stats.allocated as f64); + metrics::gauge!("dekaf_mem_mapped").set(stats.mapped as f64); + metrics::gauge!("dekaf_mem_metadata").set(stats.metadata as f64); + metrics::gauge!("dekaf_mem_resident").set(stats.resident as f64); + metrics::gauge!("dekaf_mem_retained").set(stats.retained as f64); + metrics::gauge!("dekaf_mem_active").set(stats.active as f64); } #[tracing::instrument(skip_all)] diff --git a/crates/dekaf/src/read.rs b/crates/dekaf/src/read.rs index 2fa7d2dff2..f88b8d2706 100644 --- a/crates/dekaf/src/read.rs +++ b/crates/dekaf/src/read.rs @@ -272,9 +272,9 @@ impl Read { "returning records" ); - metrics::counter!("documents_read", "journal_name" => self.journal_name.to_owned()) + metrics::counter!("dekaf_documents_read", "journal_name" => self.journal_name.to_owned()) .increment(records.len() as u64); - metrics::counter!("bytes_read", "journal_name" => self.journal_name.to_owned()) + metrics::counter!("dekaf_bytes_read", "journal_name" => self.journal_name.to_owned()) .increment(records_bytes as u64); Ok((self, buf.freeze())) From a98284aa26604c5f978585940f6a8380a7cc6d16 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Fri, 4 Oct 2024 14:19:47 -0400 Subject: [PATCH 22/26] dekaf: Frequently kill idle connections --- crates/dekaf/src/api_client.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/dekaf/src/api_client.rs b/crates/dekaf/src/api_client.rs index ea647abbd9..f3489ed526 100644 --- a/crates/dekaf/src/api_client.rs +++ b/crates/dekaf/src/api_client.rs @@ -381,7 +381,7 @@ impl KafkaApiClient { // It seems that after running for a while, connections can get into // a broken state where every response returns an error. This, plus // the healthcheck when recycling a connection solves that problem. - let reap_interval = Duration::from_secs(30); + let reap_interval = Duration::from_secs(5); let max_age = Duration::from_secs(60 * 30); let max_idle = Duration::from_secs(60); let reaper = tokio_util::task::AbortOnDropHandle::new(tokio::spawn({ From 52defc3ed26e22b66f727ce2adea15cd608291e4 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Fri, 4 Oct 2024 14:41:13 -0400 Subject: [PATCH 23/26] dekaf: Remove connection healthcheck as `FindCoordinator` takes ~3-5 seconds to respond --- crates/dekaf/src/api_client.rs | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/crates/dekaf/src/api_client.rs b/crates/dekaf/src/api_client.rs index f3489ed526..17914fd12d 100644 --- a/crates/dekaf/src/api_client.rs +++ b/crates/dekaf/src/api_client.rs @@ -297,16 +297,10 @@ impl deadpool::managed::Manager for KafkaConnectionParams { async fn recycle( &self, - conn: &mut BoxedKafkaConnection, + _conn: &mut BoxedKafkaConnection, _: &deadpool::managed::Metrics, ) -> deadpool::managed::RecycleResult { - // Other than auth, Kafka connections themselves are stateless - // so the only thing we need to do when recycling a connection - // is to confirm that it's still connected. - get_versions(conn).await.map(|_| ()).map_err(|e| { - tracing::warn!(err=?e, broker=self.broker_url, "Connection failed healthcheck"); - deadpool::managed::RecycleError::Backend(e) - }) + Ok(()) } } @@ -381,9 +375,9 @@ impl KafkaApiClient { // It seems that after running for a while, connections can get into // a broken state where every response returns an error. This, plus // the healthcheck when recycling a connection solves that problem. - let reap_interval = Duration::from_secs(5); + let reap_interval = Duration::from_secs(30); let max_age = Duration::from_secs(60 * 30); - let max_idle = Duration::from_secs(60); + let max_idle = Duration::from_secs(60 * 5); let reaper = tokio_util::task::AbortOnDropHandle::new(tokio::spawn({ let pool = pool.clone(); let broker_url = broker_url.to_string(); From 86745ef14fd637349d293529fa7265c0645550ea Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Fri, 4 Oct 2024 18:35:15 -0400 Subject: [PATCH 24/26] dekaf: Add a bunch of descriptive timeouts --- crates/dekaf/src/session.rs | 25 +++++++++++++++---------- crates/dekaf/src/topology.rs | 17 +++++++++++++---- 2 files changed, 28 insertions(+), 14 deletions(-) diff --git a/crates/dekaf/src/session.rs b/crates/dekaf/src/session.rs index 5a1cdcf748..0a36aa5542 100644 --- a/crates/dekaf/src/session.rs +++ b/crates/dekaf/src/session.rs @@ -17,11 +17,11 @@ use kafka_protocol::{ }, protocol::{buf::ByteBuf, Decodable, Encodable, Message, StrBytes}, }; -use std::sync::Arc; use std::{ collections::HashMap, time::{SystemTime, UNIX_EPOCH}, }; +use std::{sync::Arc, time::Duration}; use tracing::instrument; struct PendingRead { @@ -184,15 +184,20 @@ impl Session { // Concurrently fetch Collection instances for all requested topics. let collections: anyhow::Result)>> = - futures::future::try_join_all(requests.into_iter().map(|topic| async move { - let maybe_collection = Collection::new( - client, - from_downstream_topic_name(topic.name.to_owned().unwrap_or_default()).as_str(), - ) - .await?; - Ok((topic.name.unwrap_or_default(), maybe_collection)) - })) - .await; + tokio::time::timeout( + Duration::from_secs(10), + futures::future::try_join_all(requests.into_iter().map(|topic| async move { + let maybe_collection = Collection::new( + client, + from_downstream_topic_name(topic.name.to_owned().unwrap_or_default()) + .as_str(), + ) + .await?; + Ok((topic.name.unwrap_or_default(), maybe_collection)) + })), + ) + .await + .map_err(|e| anyhow::anyhow!("Timed out loading metadata {e}"))?; let mut topics = IndexMap::new(); diff --git a/crates/dekaf/src/topology.rs b/crates/dekaf/src/topology.rs index 5f6a47530f..34b885f52d 100644 --- a/crates/dekaf/src/topology.rs +++ b/crates/dekaf/src/topology.rs @@ -1,7 +1,8 @@ use anyhow::Context; -use futures::{StreamExt, TryStreamExt}; +use futures::{StreamExt, TryFutureExt, TryStreamExt}; use gazette::{broker, journal, uuid}; use proto_flow::flow; +use std::time::Duration; /// Fetch the names of all collections which the current user may read. /// Each is mapped into a kafka topic. @@ -154,7 +155,11 @@ impl Collection { }), ..Default::default() }; - let response = journal_client.list(request).await?; + let response = tokio::time::timeout(Duration::from_secs(5), journal_client.list(request)) + .await + .map_err(|e| { + anyhow::anyhow!("timed out fetching partitions for {collection}: {e}") + })??; let mut partitions = Vec::with_capacity(response.journals.len()); for journal in response.journals { @@ -238,8 +243,12 @@ impl Collection { client: &flow_client::Client, collection: &str, ) -> anyhow::Result { - let (_, journal_client) = - flow_client::fetch_collection_authorization(client, collection).await?; + let (_, journal_client) = tokio::time::timeout( + Duration::from_secs(5), + flow_client::fetch_collection_authorization(client, collection), + ) + .map_err(|e| anyhow::anyhow!("timed out building journal client for {collection}: {e}")) + .await??; Ok(journal_client) } From 4064be86d4506452bfb2a9efd791e991d00cf08f Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Mon, 7 Oct 2024 12:04:06 -0400 Subject: [PATCH 25/26] dekaf: Temporarily implement and use `flow_client::Client::with_fresh_gazette_client()` gRPC connections seem to lose stability after a while, this temporarily fixes that --- crates/dekaf/src/lib.rs | 3 ++- crates/flow-client/src/client.rs | 24 ++++++++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/crates/dekaf/src/lib.rs b/crates/dekaf/src/lib.rs index d7bfdc4d2c..6c144a71d0 100644 --- a/crates/dekaf/src/lib.rs +++ b/crates/dekaf/src/lib.rs @@ -72,7 +72,8 @@ impl App { let client = self .client_base .clone() - .with_creds(Some(access), Some(refresh)); + .with_creds(Some(access), Some(refresh)) + .with_fresh_gazette_client(); let claims = flow_client::client::client_claims(&client)?; diff --git a/crates/flow-client/src/client.rs b/crates/flow-client/src/client.rs index 33c9d62b7a..170b25bccd 100644 --- a/crates/flow-client/src/client.rs +++ b/crates/flow-client/src/client.rs @@ -75,6 +75,30 @@ impl Client { } } + /// Build a fresh `gazette::journal::Client` and `gazette::shard::Client` + /// There is a bug that causes these clients to hang under heavy/varied load, + /// so until that bug is found+fixed, this is the work-around. + #[deprecated] + pub fn with_fresh_gazette_client(self) -> Self { + let router = gazette::Router::new("local"); + + let journal_client = gazette::journal::Client::new( + String::new(), + gazette::Metadata::default(), + router.clone(), + ); + let shard_client = gazette::shard::Client::new( + String::new(), + gazette::Metadata::default(), + router.clone(), + ); + Self { + journal_client, + shard_client, + ..self + } + } + pub fn pg_client(&self) -> postgrest::Postgrest { if let Some(token) = &self.user_access_token { return self From e1362fd16bb6c4cea971ecb9c1f34e2d9c924f31 Mon Sep 17 00:00:00 2001 From: Joseph Shearer Date: Mon, 7 Oct 2024 12:45:58 -0400 Subject: [PATCH 26/26] dekaf: Request clients to throttle if we hit a timeout --- crates/dekaf/src/read.rs | 42 ++++++++++++++++++++++++++++----- crates/dekaf/src/session.rs | 46 ++++++++++++++++++++----------------- 2 files changed, 61 insertions(+), 27 deletions(-) diff --git a/crates/dekaf/src/read.rs b/crates/dekaf/src/read.rs index f88b8d2706..8fa4e2dff7 100644 --- a/crates/dekaf/src/read.rs +++ b/crates/dekaf/src/read.rs @@ -7,7 +7,7 @@ use gazette::journal::{ReadJsonLine, ReadJsonLines}; use gazette::{broker, journal, uuid}; use kafka_protocol::records::Compression; use lz4_flex::frame::BlockMode; -use std::time::Duration; +use std::time::{Duration, Instant}; pub struct Read { /// Journal offset to be served by this Read. @@ -30,6 +30,15 @@ pub struct Read { journal_name: String, } +pub enum BatchResult { + /// Read some docs, stopped reading because reached target bytes + TargetExceededBeforeTimeout(bytes::Bytes), + /// Read some docs, stopped reading because reached timeout + TimeoutExceededBeforeTarget(bytes::Bytes), + /// Read no docs, stopped reading because reached timeout + TimeoutNoData, +} + impl Read { pub fn new( client: journal::Client, @@ -74,7 +83,11 @@ impl Read { } #[tracing::instrument(skip_all,fields(journal_name=self.journal_name))] - pub async fn next_batch(mut self, target_bytes: usize) -> anyhow::Result<(Self, bytes::Bytes)> { + pub async fn next_batch( + mut self, + target_bytes: usize, + timeout: Instant, + ) -> anyhow::Result<(Self, BatchResult)> { use kafka_protocol::records::{ Compression, Record, RecordBatchEncoder, RecordEncodeOptions, TimestampType, }; @@ -90,15 +103,22 @@ impl Read { let mut has_had_parsing_error = false; let mut transient_errors = 0; + let timeout = tokio::time::sleep_until(timeout.into()); + let timeout = futures::future::maybe_done(timeout); + tokio::pin!(timeout); + + let mut did_timeout = false; + while records_bytes < target_bytes { let read = match tokio::select! { biased; // Attempt to read before yielding. read = self.stream.next() => read, - () = std::future::ready(()), if records_bytes != 0 => { - break; // Yield if we have records and the stream isn't ready. - } + _ = &mut timeout => { + did_timeout = true; + break; // Yield if we reach a timeout + }, } { None => bail!("blocking gazette client read never returns EOF"), Some(resp) => match resp { @@ -277,7 +297,17 @@ impl Read { metrics::counter!("dekaf_bytes_read", "journal_name" => self.journal_name.to_owned()) .increment(records_bytes as u64); - Ok((self, buf.freeze())) + Ok(( + self, + match (records.len() > 0, did_timeout) { + (false, true) => BatchResult::TimeoutNoData, + (true, true) => BatchResult::TimeoutExceededBeforeTarget(buf.freeze()), + (true, false) => BatchResult::TargetExceededBeforeTimeout(buf.freeze()), + (false, false) => { + unreachable!("shouldn't be able see no documents, and also not timeout") + } + }, + )) } } diff --git a/crates/dekaf/src/session.rs b/crates/dekaf/src/session.rs index 0a36aa5542..053903a3f2 100644 --- a/crates/dekaf/src/session.rs +++ b/crates/dekaf/src/session.rs @@ -1,8 +1,8 @@ use super::{App, Collection, Read}; use crate::{ connector::DekafConfig, from_downstream_topic_name, from_upstream_topic_name, - to_downstream_topic_name, to_upstream_topic_name, topology::fetch_all_collection_names, - Authenticated, + read::BatchResult, to_downstream_topic_name, to_upstream_topic_name, + topology::fetch_all_collection_names, Authenticated, }; use anyhow::Context; use bytes::{BufMut, BytesMut}; @@ -27,7 +27,7 @@ use tracing::instrument; struct PendingRead { offset: i64, // Journal offset to be completed by this PendingRead. last_write_head: i64, // Most-recent observed journal write head. - handle: tokio_util::task::AbortOnDropHandle>, + handle: tokio_util::task::AbortOnDropHandle>, } pub struct Session { @@ -365,9 +365,10 @@ impl Session { .as_ref() .ok_or(anyhow::anyhow!("Session not authenticated"))?; - let timeout = tokio::time::sleep(std::time::Duration::from_millis(max_wait_ms as u64)); - let timeout = futures::future::maybe_done(timeout); - tokio::pin!(timeout); + let timeout_at = + std::time::Instant::now() + std::time::Duration::from_millis(max_wait_ms as u64); + + let mut hit_timeout = false; // Start reads for all partitions which aren't already pending. for topic_request in &topic_requests { @@ -408,7 +409,7 @@ impl Session { offset: fetch_offset, last_write_head: fetch_offset, handle: tokio_util::task::AbortOnDropHandle::new(tokio::spawn( - read.next_batch(partition_request.partition_max_bytes as usize), + read.next_batch(partition_request.partition_max_bytes as usize, timeout_at), )), }; @@ -451,25 +452,27 @@ impl Session { continue; }; - let batch = if let Some((read, batch)) = tokio::select! { - biased; // Prefer to complete a pending read. - read = &mut pending.handle => Some(read??), - _ = &mut timeout => None, - } { - pending.offset = read.offset; - pending.last_write_head = read.last_write_head; - pending.handle = tokio_util::task::AbortOnDropHandle::new(tokio::spawn( - read.next_batch(partition_request.partition_max_bytes as usize), - )); - batch - } else { - bytes::Bytes::new() + let (read, batch) = (&mut pending.handle).await??; + pending.offset = read.offset; + pending.last_write_head = read.last_write_head; + pending.handle = tokio_util::task::AbortOnDropHandle::new(tokio::spawn( + read.next_batch(partition_request.partition_max_bytes as usize, timeout_at), + )); + + let (timeout, batch) = match batch { + BatchResult::TargetExceededBeforeTimeout(b) => (false, Some(b)), + BatchResult::TimeoutExceededBeforeTarget(b) => (true, Some(b)), + BatchResult::TimeoutNoData => (true, None), }; + if timeout { + hit_timeout = true + } + partition_responses.push( PartitionData::default() .with_partition_index(partition_request.partition) - .with_records(Some(batch)) + .with_records(batch.to_owned()) .with_high_watermark(pending.last_write_head) // Map to kafka cursor. .with_last_stable_offset(pending.last_write_head), ); @@ -484,6 +487,7 @@ impl Session { Ok(messages::FetchResponse::default() .with_session_id(session_id) + .with_throttle_time_ms(if hit_timeout { 10000 } else { 0 }) .with_responses(topic_responses)) }