diff --git a/Cargo.lock b/Cargo.lock index aa5709896f..4c3ba7426e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -112,6 +112,7 @@ dependencies = [ "chrono", "clap 4.5.17", "colored_json", + "dekaf", "derivative", "doc", "futures", @@ -1001,6 +1002,7 @@ dependencies = [ "anyhow", "assemble", "bytes", + "dekaf", "futures", "models", "ops", @@ -1825,7 +1827,7 @@ dependencies = [ "crypto-common", "deadpool", "doc", - "flowctl", + "flow-client", "futures", "gazette", "hex", @@ -1836,7 +1838,8 @@ dependencies = [ "lz4_flex", "md5", "metrics", - "metrics-prometheus", + "metrics-exporter-prometheus", + "models", "ops", "percent-encoding", "postgrest", @@ -1847,8 +1850,9 @@ dependencies = [ "regex", "rsasl", "rustls 0.23.10", - "rustls-native-certs", + "rustls-native-certs 0.7.2", "rustls-pemfile 2.1.3", + "schemars", "serde", "serde_json", "simd-doc", @@ -2228,6 +2232,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 +2306,7 @@ dependencies = [ "dirs", "doc", "extractors", + "flow-client", "futures", "gazette", "humantime", @@ -2944,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" @@ -3439,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]] @@ -3655,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]] @@ -3681,6 +3733,8 @@ dependencies = [ "hashbrown 0.14.5", "metrics", "num_cpus", + "quanta", + "sketches-ddsketch", ] [[package]] @@ -4754,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" @@ -4916,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" @@ -5066,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", @@ -5313,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" @@ -5494,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" @@ -5801,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" @@ -6459,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", @@ -6671,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/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, 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/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/dekaf/Cargo.toml b/crates/dekaf/Cargo.toml index c94317aa81..c33527eda7 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" } @@ -28,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 } @@ -41,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 } @@ -51,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/api_client.rs b/crates/dekaf/src/api_client.rs index bab79307a6..17914fd12d 100644 --- a/crates/dekaf/src/api_client.rs +++ b/crates/dekaf/src/api_client.rs @@ -3,12 +3,21 @@ 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 std::{boxed::Box, cell::Cell, collections::HashMap, fmt::Debug, io, time::Duration}; +use rustls::RootCertStore; +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; use tokio_rustls::rustls; use tokio_util::{codec, task::AbortOnDropHandle}; @@ -24,22 +33,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)); @@ -279,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(()) } } @@ -334,7 +346,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()); @@ -342,11 +359,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. @@ -355,7 +377,7 @@ impl KafkaApiClient { // the healthcheck when recycling a connection solves that problem. 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(); @@ -363,11 +385,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); @@ -381,8 +403,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 }); } } })); @@ -418,6 +440,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, @@ -427,7 +450,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/connector.rs b/crates/dekaf/src/connector.rs new file mode 100644 index 0000000000..253bdbb2f9 --- /dev/null +++ b/crates/dekaf/src/connector.rs @@ -0,0 +1,90 @@ +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, 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))] + #[schemars(schema_with = "token_secret")] + pub token: String, +} + +/// Configures a particular binding in a Dekaf-type materialization +#[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. + #[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 { + 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..6c144a71d0 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; @@ -22,15 +23,13 @@ mod api_client; pub use api_client::KafkaApiClient; use aes_siv::{aead::Aead, Aes256SivAead, KeyInit, KeySizeUser}; -use itertools::Itertools; +use connector::DekafConfig; +use flow_client::client::{refresh_authorizations, RefreshToken}; use percent_encoding::{percent_decode_str, utf8_percent_encode}; use serde::{Deserialize, Serialize}; -use serde_json::de; use std::time::SystemTime; 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. @@ -39,87 +38,66 @@ pub struct App { pub kafka_client: KafkaApiClient, /// Secret used to secure Prometheus endpoint pub secret: String, + /// Share a single base client in order to re-use connection pools + pub client_base: flow_client::Client, } #[derive(Debug, Clone, Serialize, Deserialize)] -pub struct ConfigOptions { +pub struct DeprecatedConfigOptions { #[serde(default = "bool::")] pub strict_topic_names: bool, } pub struct Authenticated { - client: postgrest::Postgrest, - 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, + client: flow_client::Client, + 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")?; - #[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")?; + let raw_token = String::from_utf8(base64::decode(password)?.to_vec())?; + let refresh: RefreshToken = serde_json::from_str(raw_token.as_str())?; - tracing::info!(refresh_token_id, "authenticating refresh token"); + let (access, refresh) = + refresh_authorizations(&self.client_base, None, Some(refresh)).await?; - #[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 + let client = self + .client_base .clone() - .insert_header("Authorization", format!("Bearer {access_token}")); + .with_creds(Some(access), Some(refresh)) + .with_fresh_gazette_client(); - 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")?; + let claims = flow_client::client::client_claims(&client)?; - Ok(Authenticated { - client: 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") + } } } @@ -324,8 +302,8 @@ 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); + 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 19286dd661..58221250c5 100644 --- a/crates/dekaf/src/main.rs +++ b/crates/dekaf/src/main.rs @@ -5,6 +5,9 @@ use anyhow::{bail, Context}; use axum_server::tls_rustls::RustlsConfig; use clap::{Args, Parser}; use dekaf::{KafkaApiClient, Session}; +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; @@ -16,6 +19,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 +28,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, @@ -73,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, } @@ -101,15 +110,13 @@ async fn main() -> anyhow::Result<()> { .with_writer(std::io::stderr) .init(); - metrics_prometheus::install(); - 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 +125,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( @@ -128,10 +134,18 @@ 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", )?, - secret: cli.encryption_secret.to_owned() + secret: cli.encryption_secret.to_owned(), + client_base: flow_client::Client::new( + DEFAULT_AGENT_URL.to_owned(), + api_key, + api_endpoint, + None, + None + ) }); tracing::info!( @@ -153,7 +167,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() }); @@ -248,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); @@ -263,6 +277,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) @@ -270,17 +285,17 @@ where { // Close the connection on error w.shutdown().await?; - return err; + res = err; } () = w.write_all(&mut out).await?; out.clear(); } - Ok(()) + res } .await; - metrics::gauge!("total_connections").decrement(1); + metrics::gauge!("dekaf_total_connections").decrement(1); result } @@ -320,9 +335,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/metrics_server.rs b/crates/dekaf/src/metrics_server.rs index 7fec64eb6d..fffefa5354 100644 --- a/crates/dekaf/src/metrics_server.rs +++ b/crates/dekaf/src/metrics_server.rs @@ -1,34 +1,40 @@ -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 } 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)] -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()) } diff --git a/crates/dekaf/src/read.rs b/crates/dekaf/src/read.rs index 2fa7d2dff2..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 { @@ -272,12 +292,22 @@ 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())) + 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/registry.rs b/crates/dekaf/src/registry.rs index a649bc23a3..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,18 +36,18 @@ async fn all_subjects( wrap(async move { let Authenticated { client, - user_config, + task_config, .. } = app.authenticate(auth.username(), auth.password()).await?; - super::fetch_all_collection_names(&client) + 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 { + if task_config.strict_topic_names { to_downstream_topic_name(TopicName::from(StrBytes::from_string(name))) .to_string() } else { @@ -95,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 18513a9a19..053903a3f2 100644 --- a/crates/dekaf/src/session.rs +++ b/crates/dekaf/src/session.rs @@ -1,7 +1,8 @@ -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, + connector::DekafConfig, from_downstream_topic_name, from_upstream_topic_name, + read::BatchResult, to_downstream_topic_name, to_upstream_topic_name, + topology::fetch_all_collection_names, Authenticated, }; use anyhow::Context; use bytes::{BufMut, BytesMut}; @@ -16,38 +17,37 @@ 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 { 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 { app: Arc, - client: postgrest::Postgrest, + client: Option, reads: HashMap<(TopicName, i32), PendingRead>, /// ID of the authenticated user user_id: Option, - config: Option, + task_config: Option, secret: String, } 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, + task_config: None, secret, } } @@ -84,12 +84,12 @@ impl Session { let response = match self.app.authenticate(authcid, password).await { Ok(Authenticated { client, - user_config, + task_config, claims, }) => { - self.client = client; - self.config.replace(user_config); - self.user_id.replace(claims.sub); + self.client.replace(client); + self.task_config.replace(task_config); + self.user_id.replace(claims.sub.to_string()); let mut response = messages::SaslAuthenticateResponse::default(); response.session_lifetime_ms = (1000 @@ -144,7 +144,14 @@ impl Session { async fn metadata_all_topics( &mut self, ) -> anyhow::Result> { - let collections = 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,19 +177,27 @@ 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)>> = - 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(); @@ -247,7 +262,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,10 +360,15 @@ impl Session { .. } = request; - let client = &self.client; - 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 client = &self + .client + .as_ref() + .ok_or(anyhow::anyhow!("Session not authenticated"))?; + + 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 { @@ -370,10 +393,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, @@ -384,9 +408,9 @@ impl Session { let pending = PendingRead { offset: fetch_offset, last_write_head: fetch_offset, - handle: tokio::spawn( - read.next_batch(partition_request.partition_max_bytes as usize), - ), + handle: tokio_util::task::AbortOnDropHandle::new(tokio::spawn( + read.next_batch(partition_request.partition_max_bytes as usize, timeout_at), + )), }; tracing::info!( @@ -428,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::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), ); @@ -461,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)) } @@ -1026,7 +1053,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 diff --git a/crates/dekaf/src/topology.rs b/crates/dekaf/src/topology.rs index 66911314f4..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. @@ -17,7 +18,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 @@ -51,10 +52,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 +66,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?; @@ -153,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 { @@ -234,43 +240,17 @@ 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); - - Ok(client) + 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) } 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 55% rename from crates/flowctl/src/client.rs rename to crates/flow-client/src/client.rs index 54f193decf..170b25bccd 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,28 +11,29 @@ pub struct Client { agent_endpoint: url::Url, // HTTP client to use for REST requests. http_client: reqwest::Client, - // PostgREST client. - pg_client: postgrest::Postgrest, // 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. 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 { /// 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_api_token: String, + pg_url: Url, + user_access_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 // new clients using dynamic addresses and access tokens, while re-using @@ -46,21 +52,70 @@ impl Client { ); Self { - agent_endpoint: config.get_agent_url().clone(), + agent_endpoint, http_client: reqwest::Client::new(), + pg_parent: postgrest::Postgrest::new(pg_url.as_str()) + .insert_header("apikey", pg_api_token.as_str()), journal_client, - pg_client, shard_client, user_access_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 + } + } + + /// 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 + .pg_parent + .clone() + .insert_header("Authorization", &format!("Bearer {token}")); + } + + self.pg_parent.clone() + } + 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 +152,12 @@ impl Client { } } +#[derive(Debug, Clone, serde::Deserialize, serde::Serialize)] +pub struct RefreshToken { + pub id: models::Id, + pub secret: String, +} + #[tracing::instrument(skip(client), err)] pub async fn fetch_task_authorization( client: &Client, @@ -231,3 +292,80 @@ pub async fn fetch_collection_authorization( Ok((journal_name_prefix, journal_client)) } + +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 + 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 (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 + }; + + 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"}) + .to_string(), + ), + ) + .await?; + + tracing::info!("created new refresh token"); + Ok((access, 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 }), + )) + } + _ => 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/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..afbaf77dd5 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_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(), + None, + None, + ) + } 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 1476ed4182..d0485ad407 100644 --- a/crates/flowctl/src/generate/mod.rs +++ b/crates/flowctl/src/generate/mod.rs @@ -287,6 +287,13 @@ 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(), + }, + serde_json::from_str::(config.config.get()).ok(), + ), }; let missing_resource_urls: Vec<(url::Url, models::Collection)> = bindings .iter() diff --git a/crates/flowctl/src/lib.rs b/crates/flowctl/src/lib.rs index 59649aaf11..9ed6b55b9b 100644 --- a/crates/flowctl/src/lib.rs +++ b/crates/flowctl/src/lib.rs @@ -1,11 +1,9 @@ use std::fmt::Debug; -use anyhow::Context; use clap::Parser; mod auth; mod catalog; -mod client; mod collection; mod config; mod draft; @@ -13,12 +11,13 @@ mod generate; mod local_specs; mod ops; mod output; -pub mod pagination; mod poll; mod preview; mod raw; -use client::Client; +use flow_client::client::refresh_authorizations; +pub(crate) use flow_client::client::Client; +pub(crate) use flow_client::{api_exec, api_exec_paginated}; use output::{Output, OutputType}; use poll::poll_while_queued; @@ -135,66 +134,27 @@ 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 anon_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 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()); - 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; - } - } - - 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?; - - 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"); + 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: Client::new(&config), + client, config, output, }; @@ -216,42 +176,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 +216,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/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. diff --git a/crates/models/src/connector.rs b/crates/models/src/connector.rs index 0491801c43..1bc7ad193a 100644 --- a/crates/models/src/connector.rs +++ b/crates/models/src/connector.rs @@ -17,6 +17,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 { 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/Cargo.toml b/crates/runtime/Cargo.toml index a6cdf8de7d..6116c3591a 100644 --- a/crates/runtime/Cargo.toml +++ b/crates/runtime/Cargo.toml @@ -11,8 +11,8 @@ 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" } derive-sqlite = { path = "../derive-sqlite" } doc = { path = "../doc" } extractors = { path = "../extractors" } @@ -32,7 +32,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/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/runtime/src/unary.rs b/crates/runtime/src/unary.rs index 7ad7e28b82..f71652c803 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( diff --git a/crates/sources/src/indirect.rs b/crates/sources/src/indirect.rs index 1015ca1fea..8cd7c4e73b 100644 --- a/crates/sources/src/indirect.rs +++ b/crates/sources/src/indirect.rs @@ -498,6 +498,18 @@ fn indirect_materialization( resources, threshold, ), + models::MaterializationEndpoint::Dekaf(models::DekafConfig { config, .. }) => indirect_dom( + Scope::new(scope) + .push_prop("endpoint") + .push_prop("dekaf") + .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/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..8ab360517c 100644 --- a/crates/sources/src/loader.rs +++ b/crates/sources/src/loader.rs @@ -687,6 +687,21 @@ impl Loader { .boxed(), ); } + models::MaterializationEndpoint::Dekaf(models::DekafConfig { config, .. }) => { + tasks.push( + async move { + self.load_config( + scope + .push_prop("endpoint") + .push_prop("dekaf") + .push_prop("config"), + config, + ) + .await + } + .boxed(), + ); + } }; for (index, binding) in spec.bindings.iter().enumerate() { 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-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_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-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/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/sources/tests/snapshots/schema_generation__catalog_schema_snapshot.snap b/crates/sources/tests/snapshots/schema_generation__catalog_schema_snapshot.snap index 5d201e7603..cee3f54fc0 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,23 @@ expression: "&schema" } } }, + "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.", "type": "object", @@ -917,6 +935,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..60e44c7c68 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. diff --git a/flow.schema.json b/flow.schema.json index 16c83fbe35..168f85d74b 100644 --- a/flow.schema.json +++ b/flow.schema.json @@ -456,6 +456,23 @@ } } }, + "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.", "type": "object", @@ -913,6 +930,19 @@ } }, "additionalProperties": false + }, + { + "title": "A Dekaf connection", + "type": "object", + "required": [ + "dekaf" + ], + "properties": { + "dekaf": { + "$ref": "#/definitions/DekafConfig" + } + }, + "additionalProperties": false } ] }, 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.