From 8ebbe323a9bd041063bba28b2cdcc2617fb0cf75 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Sat, 14 Sep 2024 12:13:00 -0500 Subject: [PATCH 1/8] agent: add additional `data_planes` columns Track managed data-plane metadata that users care about. Also add an `enable_l2` toggle to toggle whether data-planes are included in L2 roll-ups. --- crates/agent/src/api/create_data_plane.rs | 6 ++++-- crates/agent/src/api/update_l2_reporting.rs | 16 ++++++++++------ crates/agent/src/integration_tests/harness.rs | 6 ++++-- .../migrations/63_new_data_plane_columns.sql | 19 +++++++++++++++++++ 4 files changed, 37 insertions(+), 10 deletions(-) create mode 100644 supabase/migrations/63_new_data_plane_columns.sql diff --git a/crates/agent/src/api/create_data_plane.rs b/crates/agent/src/api/create_data_plane.rs index 592314e786..623470ba50 100644 --- a/crates/agent/src/api/create_data_plane.rs +++ b/crates/agent/src/api/create_data_plane.rs @@ -147,9 +147,10 @@ async fn do_create_data_plane( ops_l2_stats_transform, broker_address, reactor_address, - hmac_keys + hmac_keys, + enable_l2 ) values ( - $1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11 + $1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12 ) on conflict (data_plane_name) do update set broker_address = $9, @@ -171,6 +172,7 @@ async fn do_create_data_plane( broker_address, reactor_address, hmac_keys.as_slice(), + !hmac_keys.is_empty(), // Enable L2 if HMAC keys are defined at creation. ) .fetch_one(pg_pool) .await?; diff --git a/crates/agent/src/api/update_l2_reporting.rs b/crates/agent/src/api/update_l2_reporting.rs index bdd0ea791c..b2fe5e4993 100644 --- a/crates/agent/src/api/update_l2_reporting.rs +++ b/crates/agent/src/api/update_l2_reporting.rs @@ -108,11 +108,9 @@ export class Derivation extends Types.IDerivation {"# ops_l1_inferred_name as "ops_l1_inferred_name: models::Collection", ops_l2_inferred_transform, ops_l1_stats_name as "ops_l1_stats_name: models::Collection", - ops_l2_stats_transform + ops_l2_stats_transform, + enable_l2 from data_planes - -- Data-planes without configured HMAC keys are presumed to not be ready, - -- and we hold back from processing their L1 derivations. - where hmac_keys != '{}' order by data_plane_name asc; "#, ) @@ -123,6 +121,7 @@ export class Derivation extends Types.IDerivation {"# l2_inferred_transforms.push(models::TransformDef { name: models::Transform::new(&data_plane.ops_l2_inferred_transform), source: models::Source::Collection(data_plane.ops_l1_inferred_name.clone()), + disable: !data_plane.enable_l2, shuffle: models::Shuffle::Key(models::CompositeKey::new([models::JsonPointer::new( "/collection_name", @@ -132,7 +131,6 @@ export class Derivation extends Types.IDerivation {"# )), backfill: 0, - disable: false, priority: 0, read_delay: None, }); @@ -140,15 +138,18 @@ export class Derivation extends Types.IDerivation {"# l2_stats_transforms.push(models::TransformDef { name: models::Transform::new(&data_plane.ops_l2_stats_transform), source: models::Source::Collection(data_plane.ops_l1_stats_name.clone()), + disable: !data_plane.enable_l2, backfill: 0, - disable: false, lambda: models::RawValue::default(), priority: 0, read_delay: None, shuffle: models::Shuffle::Any, }); + if !data_plane.enable_l2 { + l2_stats_module.push_str("\n/*"); + } l2_stats_module.push_str(&format!( r#" {method_name}(read: {{ doc: Types.{type_name}}}): Types.Document[] {{ @@ -160,6 +161,9 @@ export class Derivation extends Types.IDerivation {"# camel_case(&data_plane.ops_l2_stats_transform, true) ) )); + if !data_plane.enable_l2 { + l2_stats_module.push_str("\n*/"); + } } l2_stats_module.push_str("\n}\n"); diff --git a/crates/agent/src/integration_tests/harness.rs b/crates/agent/src/integration_tests/harness.rs index 7f4899a2e4..d4b6650945 100644 --- a/crates/agent/src/integration_tests/harness.rs +++ b/crates/agent/src/integration_tests/harness.rs @@ -181,7 +181,8 @@ impl TestHarness { ops_l2_stats_transform, broker_address, reactor_address, - hmac_keys + hmac_keys, + enable_l2 ) values ( 'ops/dp/public/test', 'test.dp.estuary-data.com', @@ -193,7 +194,8 @@ impl TestHarness { 'from-L1-stats', 'broker:address', 'reactor:address', - '{secret-key}' + '{secret-key}', + false ) on conflict do nothing ) select 1 as "something: bool"; diff --git a/supabase/migrations/63_new_data_plane_columns.sql b/supabase/migrations/63_new_data_plane_columns.sql new file mode 100644 index 0000000000..df643cbbf7 --- /dev/null +++ b/supabase/migrations/63_new_data_plane_columns.sql @@ -0,0 +1,19 @@ +begin; + +alter table data_planes add column aws_iam_user_arn text; +alter table data_planes add column cidr_blocks cidr[] not null default '{}'; +alter table data_planes add column enable_l2 boolean not null default false; +alter table data_planes add column gcp_service_account_email text; +alter table data_planes add column ssh_private_key text; + +-- Must be provided explicitly. +alter table data_planes alter column enable_l2 drop default; + +-- Users may read out details of applied data-plane configuration. +grant select ( + aws_iam_user_arn, + cidr_blocks, + gcp_service_account_email +) on data_planes to authenticated; + +commit; \ No newline at end of file From ddb002b3983f8c584575d787ed256438709c85a5 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Sat, 14 Sep 2024 13:38:05 -0500 Subject: [PATCH 2/8] agent: refator out Snapshot into a separate module Add `evaluate()` for evaluation of a generic policy, while encapsulating Snapshot refresh and retry semantics. We'll use this shortly for additional authorization APIs. --- crates/agent/src/api/authorize.rs | 270 ++---------------------------- crates/agent/src/api/mod.rs | 17 +- crates/agent/src/api/snapshot.rs | 252 ++++++++++++++++++++++++++++ 3 files changed, 283 insertions(+), 256 deletions(-) create mode 100644 crates/agent/src/api/snapshot.rs diff --git a/crates/agent/src/api/authorize.rs b/crates/agent/src/api/authorize.rs index 3b15a384f2..6cebd58ad3 100644 --- a/crates/agent/src/api/authorize.rs +++ b/crates/agent/src/api/authorize.rs @@ -1,4 +1,4 @@ -use super::App; +use super::{App, Snapshot}; use anyhow::Context; use std::sync::Arc; @@ -75,36 +75,19 @@ async fn do_authorize_task(app: &App, Request { token }: &Request) -> anyhow::Re } }; - // Resolve the authorization snapshot against which this request is evaluated. - let snapshot = app.snapshot.read().unwrap(); - - let taken_unix = snapshot - .taken - .duration_since(std::time::UNIX_EPOCH) - .unwrap() - .as_secs(); - - // If the snapshot is too old then the client MUST retry. - if claims.iat > taken_unix + MAX_SNAPSHOT_INTERVAL.as_secs() { - begin_refresh(snapshot, &app.snapshot); - - return Ok(Response { - retry_millis: jitter_millis(), - ..Default::default() - }); - } - - match evaluate_authorization( - &snapshot, - shard_id, - shard_data_plane_fqdn, - token, - &journal_name_or_prefix, - required_role, - ) { + match Snapshot::evaluate(&app.snapshot, claims.iat, |snapshot: &Snapshot| { + evaluate_authorization( + snapshot, + shard_id, + shard_data_plane_fqdn, + token, + &journal_name_or_prefix, + required_role, + ) + }) { Ok((encoding_key, data_plane_fqdn, broker_address)) => { claims.iss = data_plane_fqdn; - claims.exp = claims.iat + exp_seconds(); + claims.exp = claims.iat + super::exp_seconds(); let token = jsonwebtoken::encode(&header, &claims, &encoding_key) .context("failed to encode authorized JWT")?; @@ -115,28 +98,11 @@ async fn do_authorize_task(app: &App, Request { token }: &Request) -> anyhow::Re ..Default::default() }) } - Err(err) if taken_unix > claims.iat => { - // The snapshot was taken AFTER the authorization request was minted, - // which means the request cannot have prior knowledge of upcoming - // state re-configurations, and this is a terminal error. - Err(err) - } - Err(_) => { - let retry_millis = if let Some(remaining) = - MIN_SNAPSHOT_INTERVAL.checked_sub(snapshot.taken.elapsed().unwrap_or_default()) - { - // Our current snapshot isn't old enough. - remaining.as_millis() as u64 - } else { - begin_refresh(snapshot, &app.snapshot); - 0 - } + jitter_millis(); - - Ok(Response { - retry_millis, - ..Default::default() - }) - } + Err(Ok(retry_millis)) => Ok(Response { + retry_millis, + ..Default::default() + }), + Err(Err(err)) => Err(err), } } @@ -281,205 +247,3 @@ fn evaluate_authorization( collection_data_plane.broker_address.clone(), )) } - -// Snapshot is a point-in-time view of control-plane state -// that influences authorization decisions. -pub struct Snapshot { - // Time immediately before the snapshot was taken. - taken: std::time::SystemTime, - // Platform collections, indexed on `journal_template_name`. - collections: Vec, - // Platform data-planes. - data_planes: tables::DataPlanes, - // Platform role grants. - role_grants: tables::RoleGrants, - // Platform tasks, indexed on `shard_template_id`. - tasks: Vec, - // `refresh` is take()-en when the current snapshot should be refreshed. - refresh_tx: Option>, -} - -// SnapshotCollection is the state of a live collection which influences authorization. -// It's indexed on `journal_template_name`. -struct SnapshotCollection { - journal_template_name: String, - collection_name: models::Collection, - data_plane_id: models::Id, -} -// SnapshotTask is the state of a live task which influences authorization. -// It's indexed on `shard_template_id`. -struct SnapshotTask { - shard_template_id: String, - task_name: models::Name, - spec_type: models::CatalogType, - data_plane_id: models::Id, -} - -pub fn seed_snapshot() -> (Snapshot, futures::channel::oneshot::Receiver<()>) { - let (next_tx, next_rx) = futures::channel::oneshot::channel(); - - ( - Snapshot { - taken: std::time::SystemTime::UNIX_EPOCH, - collections: Vec::new(), - data_planes: tables::DataPlanes::default(), - role_grants: tables::RoleGrants::default(), - tasks: Vec::new(), - refresh_tx: Some(next_tx), - }, - next_rx, - ) -} - -pub async fn snapshot_loop(app: Arc, mut refresh_rx: futures::channel::oneshot::Receiver<()>) { - while let Ok(()) = refresh_rx.await { - let (next_tx, next_rx) = futures::channel::oneshot::channel(); - refresh_rx = next_rx; - - match try_fetch_snapshot(&app.pg_pool).await { - Ok(mut snapshot) => { - snapshot.refresh_tx = Some(next_tx); - *app.snapshot.write().unwrap() = snapshot; - } - Err(err) => { - tracing::error!(?err, "failed to fetch snapshot (will retry)"); - () = tokio::time::sleep(MIN_SNAPSHOT_INTERVAL).await; - _ = next_tx.send(()); // Wake ourselves to retry. - } - }; - } -} - -async fn try_fetch_snapshot(pg_pool: &sqlx::PgPool) -> anyhow::Result { - tracing::info!("started to fetch authorization snapshot"); - let taken = std::time::SystemTime::now(); - - let mut collections = sqlx::query_as!( - SnapshotCollection, - r#" - select - journal_template_name as "journal_template_name!", - catalog_name as "collection_name: models::Collection", - data_plane_id as "data_plane_id: models::Id" - from live_specs - where journal_template_name is not null - "#, - ) - .fetch_all(pg_pool) - .await - .context("failed to fetch view of live collections")?; - - let data_planes = sqlx::query_as!( - tables::DataPlane, - r#" - select - id as "control_id: models::Id", - data_plane_name, - data_plane_fqdn, - false as "is_default!: bool", - hmac_keys, - broker_address, - reactor_address, - ops_logs_name as "ops_logs_name: models::Collection", - ops_stats_name as "ops_stats_name: models::Collection" - from data_planes - "#, - ) - .fetch_all(pg_pool) - .await - .context("failed to fetch data_planes")?; - - let role_grants = sqlx::query_as!( - tables::RoleGrant, - r#" - select - subject_role as "subject_role: models::Prefix", - object_role as "object_role: models::Prefix", - capability as "capability: models::Capability" - from role_grants - "#, - ) - .fetch_all(pg_pool) - .await - .context("failed to fetch role_grants")?; - - let mut tasks = sqlx::query_as!( - SnapshotTask, - r#" - select - shard_template_id as "shard_template_id!", - catalog_name as "task_name: models::Name", - spec_type as "spec_type!: models::CatalogType", - data_plane_id as "data_plane_id: models::Id" - from live_specs - where shard_template_id is not null - "#, - ) - .fetch_all(pg_pool) - .await - .context("failed to fetch view of live tasks")?; - - let data_planes = tables::DataPlanes::from_iter(data_planes); - let role_grants = tables::RoleGrants::from_iter(role_grants); - - // Shard ID and journal name templates are prefixes which are always - // extended with a slash-separated suffix. Avoid inadvertent matches - // over path component prefixes. - for task in &mut tasks { - task.shard_template_id.push('/'); - } - for collection in &mut collections { - collection.journal_template_name.push('/'); - } - - tasks.sort_by(|t1, t2| t1.shard_template_id.cmp(&t2.shard_template_id)); - collections.sort_by(|c1, c2| c1.journal_template_name.cmp(&c2.journal_template_name)); - - tracing::info!( - collections = collections.len(), - data_planes = data_planes.len(), - role_grants = role_grants.len(), - tasks = tasks.len(), - "fetched authorization snapshot", - ); - - Ok(Snapshot { - taken, - collections, - data_planes, - role_grants, - tasks, - refresh_tx: None, - }) -} - -fn begin_refresh<'m>( - guard: std::sync::RwLockReadGuard<'_, Snapshot>, - mu: &'m std::sync::RwLock, -) { - // We must release our read-lock before we can acquire a write lock. - std::mem::drop(guard); - - if let Some(tx) = mu.write().unwrap().refresh_tx.take() { - () = tx.send(()).unwrap(); // Begin a refresh. - } -} - -fn jitter_millis() -> u64 { - use rand::Rng; - - // The returned jitter must always be positive. - // In production, it can take a few seconds to fetch a snapshot. - rand::thread_rng().gen_range(500..10_000) -} - -fn exp_seconds() -> u64 { - use rand::Rng; - - // Select a random expiration time in range [40, 80) minutes, - // which spreads out load from re-authorization requests over time. - rand::thread_rng().gen_range(40 * 60..80 * 60) -} - -const MIN_SNAPSHOT_INTERVAL: std::time::Duration = std::time::Duration::from_secs(20); -const MAX_SNAPSHOT_INTERVAL: std::time::Duration = std::time::Duration::from_secs(300); // 5 minutes. diff --git a/crates/agent/src/api/mod.rs b/crates/agent/src/api/mod.rs index 0aa0b1b67f..8f48337bf6 100644 --- a/crates/agent/src/api/mod.rs +++ b/crates/agent/src/api/mod.rs @@ -3,8 +3,11 @@ use std::sync::{Arc, Mutex}; mod authorize; mod create_data_plane; +mod snapshot; mod update_l2_reporting; +use snapshot::Snapshot; + /// Request wraps a JSON-deserialized request type T which /// also implements the validator::Validate trait. #[derive(Debug, Clone, Copy, Default)] @@ -36,7 +39,7 @@ struct App { jwt_validation: jsonwebtoken::Validation, pg_pool: sqlx::PgPool, publisher: crate::publications::Publisher, - snapshot: std::sync::RwLock, + snapshot: std::sync::RwLock, } /// Build the agent's API router. @@ -51,7 +54,7 @@ pub fn build_router( let mut jwt_validation = jsonwebtoken::Validation::default(); jwt_validation.set_audience(&["authenticated"]); - let (snapshot, seed_rx) = authorize::seed_snapshot(); + let (snapshot, seed_rx) = snapshot::seed(); let app = Arc::new(App { id_generator: Mutex::new(id_generator), @@ -61,7 +64,7 @@ pub fn build_router( publisher, snapshot: std::sync::RwLock::new(snapshot), }); - tokio::spawn(authorize::snapshot_loop(app.clone(), seed_rx)); + tokio::spawn(snapshot::fetch_loop(app.clone(), seed_rx)); use axum::routing::post; @@ -155,3 +158,11 @@ async fn authorize( req.extensions_mut().insert(token.claims); next.run(req).await } + +fn exp_seconds() -> u64 { + use rand::Rng; + + // Select a random expiration time in range [40, 80) minutes, + // which spreads out load from re-authorization requests over time. + rand::thread_rng().gen_range(40 * 60..80 * 60) +} diff --git a/crates/agent/src/api/snapshot.rs b/crates/agent/src/api/snapshot.rs new file mode 100644 index 0000000000..e37507b23c --- /dev/null +++ b/crates/agent/src/api/snapshot.rs @@ -0,0 +1,252 @@ +use super::App; +use anyhow::Context; +use std::sync::Arc; + +// Snapshot is a point-in-time view of control-plane state +// that influences authorization decisions. +pub struct Snapshot { + // Time immediately before the snapshot was taken. + pub taken: std::time::SystemTime, + // Platform collections, indexed on `journal_template_name`. + pub collections: Vec, + // Platform data-planes. + pub data_planes: tables::DataPlanes, + // Platform role grants. + pub role_grants: tables::RoleGrants, + // Platform tasks, indexed on `shard_template_id`. + pub tasks: Vec, + // `refresh` is take()-en when the current snapshot should be refreshed. + pub refresh_tx: Option>, +} + +// SnapshotCollection is the state of a live collection which influences authorization. +// It's indexed on `journal_template_name`. +pub struct SnapshotCollection { + // Template journal name which prefixes all journals of the collection. + pub journal_template_name: String, + // Catalog name of the collection. + pub collection_name: models::Collection, + // Data-plane where this collection lives. + pub data_plane_id: models::Id, +} +// SnapshotTask is the state of a live task which influences authorization. +// It's indexed on `shard_template_id`. +pub struct SnapshotTask { + // Template shard ID which prefixes all shard IDs of the task. + pub shard_template_id: String, + // Catalog name of the task. + pub task_name: models::Name, + // Catalog type of the task. + pub spec_type: models::CatalogType, + // Data-plane where this task lives. + pub data_plane_id: models::Id, +} + +impl Snapshot { + pub fn evaluate( + mu: &std::sync::RwLock, + iat: u64, + policy: P, + ) -> Result> + where + P: FnOnce(&Self) -> anyhow::Result, + { + let guard = mu.read().unwrap(); + + let taken_unix = guard + .taken + .duration_since(std::time::UNIX_EPOCH) + .unwrap() + .as_secs(); + + // If the snapshot is too old then the client MUST retry. + if iat > taken_unix + Snapshot::MAX_INTERVAL.as_secs() { + Self::begin_refresh(guard, mu); + + return Err(Ok(jitter_millis())); + } + + match policy(&guard) { + Ok(ok) => Ok(ok), + Err(err) if taken_unix > iat => { + // The snapshot was taken AFTER the authorization request was minted, + // which means the request cannot have prior knowledge of upcoming + // state re-configurations, and this is a terminal error. + Err(Err(err)) + } + Err(_) => { + let retry_millis = if let Some(remaining) = + Snapshot::MIN_INTERVAL.checked_sub(guard.taken.elapsed().unwrap_or_default()) + { + // Our current snapshot isn't old enough. + remaining.as_millis() as u64 + } else { + Snapshot::begin_refresh(guard, mu); + 0 + } + jitter_millis(); + + Err(Ok(retry_millis)) + } + } + } + + fn begin_refresh<'m>( + guard: std::sync::RwLockReadGuard<'_, Self>, + mu: &'m std::sync::RwLock, + ) { + // We must release our read-lock before we can acquire a write lock. + std::mem::drop(guard); + + if let Some(tx) = mu.write().unwrap().refresh_tx.take() { + () = tx.send(()).unwrap(); // Begin a refresh. + } + } + + const MIN_INTERVAL: std::time::Duration = std::time::Duration::from_secs(20); + const MAX_INTERVAL: std::time::Duration = std::time::Duration::from_secs(300); // 5 minutes. +} + +pub fn seed() -> (Snapshot, futures::channel::oneshot::Receiver<()>) { + let (next_tx, next_rx) = futures::channel::oneshot::channel(); + + ( + Snapshot { + taken: std::time::SystemTime::UNIX_EPOCH, + collections: Vec::new(), + data_planes: tables::DataPlanes::default(), + role_grants: tables::RoleGrants::default(), + tasks: Vec::new(), + refresh_tx: Some(next_tx), + }, + next_rx, + ) +} +pub async fn fetch_loop(app: Arc, mut refresh_rx: futures::channel::oneshot::Receiver<()>) { + while let Ok(()) = refresh_rx.await { + let (next_tx, next_rx) = futures::channel::oneshot::channel(); + refresh_rx = next_rx; + + match try_fetch(&app.pg_pool).await { + Ok(mut snapshot) => { + snapshot.refresh_tx = Some(next_tx); + *app.snapshot.write().unwrap() = snapshot; + } + Err(err) => { + tracing::error!(?err, "failed to fetch snapshot (will retry)"); + () = tokio::time::sleep(Snapshot::MIN_INTERVAL).await; + _ = next_tx.send(()); // Wake ourselves to retry. + } + }; + } +} + +async fn try_fetch(pg_pool: &sqlx::PgPool) -> anyhow::Result { + tracing::info!("started to fetch authorization snapshot"); + let taken = std::time::SystemTime::now(); + + let mut collections = sqlx::query_as!( + SnapshotCollection, + r#" + select + journal_template_name as "journal_template_name!", + catalog_name as "collection_name: models::Collection", + data_plane_id as "data_plane_id: models::Id" + from live_specs + where journal_template_name is not null + "#, + ) + .fetch_all(pg_pool) + .await + .context("failed to fetch view of live collections")?; + + let data_planes = sqlx::query_as!( + tables::DataPlane, + r#" + select + id as "control_id: models::Id", + data_plane_name, + data_plane_fqdn, + false as "is_default!: bool", + hmac_keys, + broker_address, + reactor_address, + ops_logs_name as "ops_logs_name: models::Collection", + ops_stats_name as "ops_stats_name: models::Collection" + from data_planes + "#, + ) + .fetch_all(pg_pool) + .await + .context("failed to fetch data_planes")?; + + let role_grants = sqlx::query_as!( + tables::RoleGrant, + r#" + select + subject_role as "subject_role: models::Prefix", + object_role as "object_role: models::Prefix", + capability as "capability: models::Capability" + from role_grants + "#, + ) + .fetch_all(pg_pool) + .await + .context("failed to fetch role_grants")?; + + let mut tasks = sqlx::query_as!( + SnapshotTask, + r#" + select + shard_template_id as "shard_template_id!", + catalog_name as "task_name: models::Name", + spec_type as "spec_type!: models::CatalogType", + data_plane_id as "data_plane_id: models::Id" + from live_specs + where shard_template_id is not null + "#, + ) + .fetch_all(pg_pool) + .await + .context("failed to fetch view of live tasks")?; + + let data_planes = tables::DataPlanes::from_iter(data_planes); + let role_grants = tables::RoleGrants::from_iter(role_grants); + + // Shard ID and journal name templates are prefixes which are always + // extended with a slash-separated suffix. Avoid inadvertent matches + // over path component prefixes. + for task in &mut tasks { + task.shard_template_id.push('/'); + } + for collection in &mut collections { + collection.journal_template_name.push('/'); + } + + tasks.sort_by(|t1, t2| t1.shard_template_id.cmp(&t2.shard_template_id)); + collections.sort_by(|c1, c2| c1.journal_template_name.cmp(&c2.journal_template_name)); + + tracing::info!( + collections = collections.len(), + data_planes = data_planes.len(), + role_grants = role_grants.len(), + tasks = tasks.len(), + "fetched authorization snapshot", + ); + + Ok(Snapshot { + taken, + collections, + data_planes, + role_grants, + tasks, + refresh_tx: None, + }) +} + +fn jitter_millis() -> u64 { + use rand::Rng; + + // The returned jitter must always be positive. + // In production, it can take a few seconds to fetch a snapshot. + rand::thread_rng().gen_range(500..10_000) +} From 2974df673a9558b56cd9f2263d35d24192a16cbb Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Sat, 14 Sep 2024 18:23:45 -0500 Subject: [PATCH 3/8] tables: add UserGrants and refine transitive role search Add UserGrants table. Refactor RBAC search into a joint search that's generalized over both user and role grants. --- Cargo.lock | 1 + crates/tables/Cargo.toml | 1 + crates/tables/src/behaviors.rs | 233 ++++++++++++++++++++++++++------- crates/tables/src/lib.rs | 12 +- 4 files changed, 202 insertions(+), 45 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8c2a421ccf..050886ce84 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6147,6 +6147,7 @@ dependencies = [ "serde_json", "superslice", "url", + "uuid 1.10.0", "xxhash-rust", ] diff --git a/crates/tables/Cargo.toml b/crates/tables/Cargo.toml index 1ce7e06dd2..a613af339d 100644 --- a/crates/tables/Cargo.toml +++ b/crates/tables/Cargo.toml @@ -25,6 +25,7 @@ serde = { workspace = true } serde_json = { workspace = true } superslice = { workspace = true } url = { workspace = true } +uuid = { workspace = true } xxhash-rust = { workspace = true } [dev-dependencies] diff --git a/crates/tables/src/behaviors.rs b/crates/tables/src/behaviors.rs index bca7d9de72..8e31a4e97a 100644 --- a/crates/tables/src/behaviors.rs +++ b/crates/tables/src/behaviors.rs @@ -52,74 +52,149 @@ impl super::Import { } impl super::RoleGrant { - /// Given a task name, enumerate all roles and capabilities granted to the task. + /// Given a role or name, enumerate all granted roles and capabilities. pub fn transitive_roles<'a>( - role_grants: &'a [Self], - task_name: &'a str, - ) -> impl Iterator> + 'a { - let seed = super::RoleGrantRef { - subject_role: "", - object_role: task_name, + role_grants: &'a [super::RoleGrant], + role_or_name: &'a str, + ) -> impl Iterator> + 'a { + let seed = super::GrantRef { + subject_role: role_or_name, + object_role: role_or_name, capability: models::Capability::Admin, }; - pathfinding::directed::bfs::bfs_reach(seed, |f| Self::edges(role_grants, *f)).skip(1) + pathfinding::directed::bfs::bfs_reach(seed, |f| { + grant_edges(*f, role_grants, &[], uuid::Uuid::nil()) + }) + .skip(1) // Skip `seed`. } - /// Given a task name, determine if it's authorized to the object name for the given capability. + /// Given a role or name, determine if it's authorized to the object name for the given capability. pub fn is_authorized<'a>( - role_grants: &'a [Self], - task_name: &'a str, + role_grants: &'a [super::RoleGrant], + role_or_name: &'a str, object_name: &'a str, capability: models::Capability, ) -> bool { - Self::transitive_roles(role_grants, task_name).any(|role_grant| { + Self::transitive_roles(role_grants, role_or_name).any(|role_grant| { object_name.starts_with(role_grant.object_role) && role_grant.capability >= capability }) } - /// Cheaply convert a &RoleGrant into an owned type which holds borrows. - pub fn to_ref<'a>(&'a self) -> super::RoleGrantRef<'a> { - super::RoleGrantRef { + fn to_ref<'a>(&'a self) -> super::GrantRef<'a> { + super::GrantRef { subject_role: self.subject_role.as_str(), object_role: self.object_role.as_str(), capability: self.capability, } } +} - fn edges<'a>( - role_grants: &'a [Self], - from: super::RoleGrantRef<'a>, - ) -> impl Iterator> + 'a { - // Split the source object role into its prefixes: - // "acmeCo/one/two/three" => ["acmeCo/one/two/", "acmeCo/one/", "acmeCo/"]. - let prefixes = from.object_role.char_indices().filter_map(|(ind, chr)| { - if chr == '/' { - Some(&from.object_role[..ind + 1]) - } else { - None - } - }); +impl super::UserGrant { + /// Given a user, enumerate all granted roles and capabilities. + pub fn transitive_roles<'a>( + role_grants: &'a [super::RoleGrant], + user_grants: &'a [super::UserGrant], + user_id: uuid::Uuid, + ) -> impl Iterator> + 'a { + let seed = super::GrantRef { + subject_role: "", + object_role: "", // Empty role causes us to map through user_grants. + capability: models::Capability::Admin, + }; + pathfinding::directed::bfs::bfs_reach(seed, move |f| { + grant_edges(*f, role_grants, user_grants, user_id) + }) + .skip(1) // Skip `seed`. + } - // For each prefix, find all `role_grants` where it's the `subject_role`. - let edges = prefixes - .map(|prefix| { - role_grants - .equal_range_by(|role_grant| role_grant.subject_role.as_str().cmp(prefix)) - }) - .map(|range| role_grants[range].into_iter().map(Self::to_ref)) - .flatten(); + /// Given a user, determine if they're authorized to the object name for the given capability. + pub fn is_authorized<'a>( + role_grants: &'a [super::RoleGrant], + user_grants: &'a [super::UserGrant], + user_id: uuid::Uuid, + object_name: &'a str, + capability: models::Capability, + ) -> bool { + Self::transitive_roles(role_grants, user_grants, user_id).any(|role_grant| { + object_name.starts_with(role_grant.object_role) && role_grant.capability >= capability + }) + } - // Only 'admin' grants are walked transitively. - if from.capability >= models::Capability::Admin { - Some(edges) - } else { - None + fn to_ref<'a>(&'a self) -> super::GrantRef<'a> { + super::GrantRef { + subject_role: "", + object_role: self.object_role.as_str(), + capability: self.capability, } - .into_iter() - .flatten() } } +fn grant_edges<'a>( + from: super::GrantRef<'a>, + role_grants: &'a [super::RoleGrant], + user_grants: &'a [super::UserGrant], + user_id: uuid::Uuid, +) -> impl Iterator> + 'a { + let (user_grants, role_grants, prefixes) = match (from.capability, from.object_role) { + // `from` is a place-holder which kicks of exploration through `user_grants` for `user_id`. + (models::Capability::Admin, "") => { + let range = user_grants.equal_range_by(|user_grant| user_grant.user_id.cmp(&user_id)); + (&user_grants[range], &role_grants[..0], None) + } + // We're an admin of `role_or_name`, and are projecting through + // role_grants to identify other roles and capabilities we take on. + (models::Capability::Admin, role_or_name) => { + // Expand to all roles having a subject_role prefixed by role_or_name. + // In other words, an admin of `acmeCo/org/` may use a role with + // subject `acmeCo/org/team/`. Intuitively, this is because the root + // subject is authorized to create any name under `acmeCo/org/`, + // which implies an ability to create a name under `acmeCo/org/team/`. + let range = role_grants.equal_range_by(|role_grant| { + if role_grant.subject_role.starts_with(role_or_name) { + std::cmp::Ordering::Equal + } else { + role_grant.subject_role.as_str().cmp(role_or_name) + } + }); + // Expand to all roles having a subject_role which prefixes role_or_name. + // In other words, a task `acmeCo/org/task` or admin of `acmeCo/org/` + // may use a role with subject `acmeCo/`. Intuitively, this is because + // the role granted to `acmeCo/` is also granted to any name underneath + // `acmeCo/`, which includes the present role or name. + // + // First split the source object role into its prefixes: + // "acmeCo/one/two/three" => ["acmeCo/one/two/", "acmeCo/one/", "acmeCo/"]. + let prefixes = role_or_name.char_indices().filter_map(|(ind, chr)| { + if chr == '/' { + Some(&role_or_name[..ind + 1]) + } else { + None + } + }); + // Then for each prefix, find all role_grants where it's the exact subject_role. + let edges = prefixes + .map(|prefix| { + role_grants + .equal_range_by(|role_grant| role_grant.subject_role.as_str().cmp(prefix)) + }) + .map(|range| role_grants[range].into_iter().map(super::RoleGrant::to_ref)) + .flatten(); + + (&user_grants[..0], &role_grants[range], Some(edges)) + } + (_not_admin, _) => { + // We perform no expansion through grants which are not Admin. + (&user_grants[..0], &role_grants[..0], None) + } + }; + + let p1 = user_grants.iter().map(super::UserGrant::to_ref); + let p2 = role_grants.iter().map(super::RoleGrant::to_ref); + let p3 = prefixes.into_iter().flatten(); + + p1.chain(p2).chain(p3) +} + impl super::StorageMapping { pub fn scope(&self) -> url::Url { crate::synthetic_scope("storageMapping", &self.catalog_prefix) @@ -128,7 +203,7 @@ impl super::StorageMapping { #[cfg(test)] mod test { - use crate::{Import, Imports, RoleGrant, RoleGrants}; + use crate::{Import, Imports, RoleGrant, RoleGrants, UserGrant, UserGrants}; #[test] fn test_transitive_imports() { @@ -186,6 +261,20 @@ mod test { capability: cap, }), ); + let user_grants = UserGrants::from_iter( + [ + (uuid::Uuid::nil(), "bobCo/", Read), + (uuid::Uuid::nil(), "daveCo/", Admin), + (uuid::Uuid::max(), "aliceCo/widgets/", Admin), + (uuid::Uuid::max(), "carolCo/shared/", Admin), + ] + .into_iter() + .map(|(user_id, obj, cap)| UserGrant { + user_id, + object_role: models::Prefix::new(obj), + capability: cap, + }), + ); insta::assert_json_snapshot!( RoleGrant::transitive_roles(&role_grants, "aliceCo/anvils/thing").collect::>(), @@ -236,6 +325,62 @@ mod test { "carolCo/even/more/hidden/thing", Write )); + + insta::assert_json_snapshot!( + UserGrant::transitive_roles(&role_grants, &user_grants, uuid::Uuid::nil()).collect::>(), + @r###" + [ + { + "subject_role": "", + "object_role": "bobCo/", + "capability": "read" + }, + { + "subject_role": "", + "object_role": "daveCo/", + "capability": "admin" + }, + { + "subject_role": "daveCo/hidden/", + "object_role": "carolCo/hidden/", + "capability": "admin" + }, + { + "subject_role": "carolCo/hidden/", + "object_role": "carolCo/even/more/hidden/", + "capability": "read" + } + ] + "###, + ); + + insta::assert_json_snapshot!( + UserGrant::transitive_roles(&role_grants, &user_grants, uuid::Uuid::max()).collect::>(), + @r###" + [ + { + "subject_role": "", + "object_role": "aliceCo/widgets/", + "capability": "admin" + }, + { + "subject_role": "", + "object_role": "carolCo/shared/", + "capability": "admin" + }, + { + "subject_role": "aliceCo/widgets/", + "object_role": "bobCo/burgers/", + "capability": "admin" + }, + { + "subject_role": "carolCo/shared/", + "object_role": "carolCo/hidden/", + "capability": "read" + } + ] + "###, + ); } #[test] diff --git a/crates/tables/src/lib.rs b/crates/tables/src/lib.rs index 1b40b2d0dd..2943ccdb5a 100644 --- a/crates/tables/src/lib.rs +++ b/crates/tables/src/lib.rs @@ -100,6 +100,15 @@ tables!( val capability: models::Capability, } + table UserGrants (row #[derive(serde::Deserialize, serde::Serialize)] UserGrant, sql "user_grants") { + // User ID to which a capability is bestowed. + key user_id: uuid::Uuid, + // Object of the grant, to which a capability is bestowed upon the subject. + key object_role: models::Prefix, + // Capability of the subject with respect to the object. + val capability: models::Capability, + } + table DraftCaptures (row DraftCapture, sql "draft_captures") { // Catalog name of this capture. key capture: models::Capture, @@ -355,7 +364,7 @@ tables!( ); #[derive(Debug, Copy, Clone, PartialEq, Eq, Hash, serde::Serialize)] -pub struct RoleGrantRef<'a> { +pub struct GrantRef<'a> { subject_role: &'a str, object_role: &'a str, capability: models::Capability, @@ -439,6 +448,7 @@ json_sql_types!( models::Schema, models::TestDef, proto_flow::flow::ContentType, + uuid::Uuid, ); proto_sql_types!( From 244345a57cde0d8aa4a6fd3df1800f4511bf67e2 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Sat, 14 Sep 2024 19:12:11 -0500 Subject: [PATCH 4/8] agent: add `/authorize/user/task` and `/authorize/user/collection` routes `/authorize/user/task` enables UI shard listings/status and retrieval of task logs, as well as access to private connector networking. `/authorize/user/collection` enables UI journal listing and data preview. Both offer temporary support for the current data-plane-gateway, which implements legacy authorization checks using claimed prefixes. Also introduce an address rewrite mechanism for mapping an internal data-plane legacy service address into the data-plane-gateway address in external call contexts. Issue #1627 --- Cargo.lock | 1 + Cargo.toml | 2 +- Tiltfile | 9 +- .../api/{authorize.rs => authorize_task.rs} | 46 +---- .../src/api/authorize_user_collection.rs | 122 +++++++++++++ crates/agent/src/api/authorize_user_task.rs | 162 ++++++++++++++++++ crates/agent/src/api/create_data_plane.rs | 4 +- crates/agent/src/api/mod.rs | 101 +++++++++-- crates/agent/src/api/snapshot.rs | 59 +++++++ crates/agent/src/api/update_l2_reporting.rs | 4 +- crates/agent/src/main.rs | 7 +- crates/models/Cargo.toml | 1 + crates/models/src/authorizations.rs | 140 +++++++++++++++ crates/models/src/lib.rs | 1 + 14 files changed, 598 insertions(+), 61 deletions(-) rename crates/agent/src/api/{authorize.rs => authorize_task.rs} (84%) create mode 100644 crates/agent/src/api/authorize_user_collection.rs create mode 100644 crates/agent/src/api/authorize_user_task.rs create mode 100644 crates/models/src/authorizations.rs diff --git a/Cargo.lock b/Cargo.lock index 050886ce84..7de2c31534 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3770,6 +3770,7 @@ dependencies = [ "time 0.3.36", "unicode-normalization", "url", + "uuid 1.10.0", "validator", ] diff --git a/Cargo.toml b/Cargo.toml index c3ba737ac2..1449c35b93 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -176,7 +176,7 @@ tokio-util = { version = "0.7", features = ["io", "compat"] } tonic = { version = "0.12", features = ["tls", "tls-roots"] } hyper-util = "0.1" tower = { version = "0.5", features = ["util"] } -tower-http = { version = "0.5", features = ["trace"] } +tower-http = { version = "0.5", features = ["cors", "trace"] } tracing = "0.1" tracing-subscriber = { version = "0.3", features = [ "time", diff --git a/Tiltfile b/Tiltfile index 3d776326e6..6025b231ef 100644 --- a/Tiltfile +++ b/Tiltfile @@ -4,9 +4,9 @@ os.putenv("DATABASE_URL", DATABASE_URL) os.putenv("RUST_LOG", "info") os.putenv("DOCKER_DEFAULT_PLATFORM", "linux/amd64") -# Secret used to sign Authorizations within a local data plane, as base64("secret"). +# Secret used to sign Authorizations within a local data plane, as base64("supersecret"). # Also allow requests without an Authorization (to not break data-plane-gateway just yet). -AUTH_KEYS="c2VjcmV0,AA==" +AUTH_KEYS="c3VwZXJzZWNyZXQ=,AA==" os.putenv("CONSUMER_AUTH_KEYS", AUTH_KEYS) os.putenv("BROKER_AUTH_KEYS", AUTH_KEYS) @@ -83,8 +83,9 @@ local_resource('reactor', serve_cmd='%s/flow/.build/package/bin/flowctl-go serve local_resource('agent', serve_cmd='%s/flow/.build/package/bin/agent \ --connector-network supabase_network_flow \ --allow-local \ - --builds-root %s \ + --allow-origin http://localhost:3000 \ --api-port 8675 \ + --builds-root %s \ --serve-handlers \ --bin-dir %s/flow/.build/package/bin' % (REPO_BASE, FLOW_BUILDS_ROOT, REPO_BASE), resource_deps=['reactor', 'gazette'] @@ -101,7 +102,7 @@ local_resource('create-data-plane-local-cluster', "manual": {\ "brokerAddress": "http://localhost:8080",\ "reactorAddress": "http://localhost:9000",\ - "hmacKeys": ["c2VjcmV0"]\ + "hmacKeys": ["c3VwZXJzZWNyZXQ="]\ }\ }\ }\' http://localhost:8675/admin/create-data-plane' % SYSTEM_USER_TOKEN, diff --git a/crates/agent/src/api/authorize.rs b/crates/agent/src/api/authorize_task.rs similarity index 84% rename from crates/agent/src/api/authorize.rs rename to crates/agent/src/api/authorize_task.rs index 6cebd58ad3..d93242e7b5 100644 --- a/crates/agent/src/api/authorize.rs +++ b/crates/agent/src/api/authorize_task.rs @@ -2,24 +2,8 @@ use super::{App, Snapshot}; use anyhow::Context; use std::sync::Arc; -#[derive(Debug, serde::Deserialize, schemars::JsonSchema)] -#[serde(rename_all = "camelCase")] -pub struct Request { - // # JWT token to be authorized and signed. - token: String, -} - -#[derive(Default, Debug, serde::Serialize, schemars::JsonSchema)] -#[serde(rename_all = "camelCase")] -pub struct Response { - // # JWT token which has been authorized for use. - token: String, - // # Address of Gazette brokers for the issued token. - broker_address: String, - // # Number of milliseconds to wait before retrying the request. - // Non-zero if and only if token is not set. - retry_millis: u64, -} +type Request = models::authorizations::TaskAuthorizationRequest; +type Response = models::authorizations::TaskAuthorization; #[axum::debug_handler] pub async fn authorize_task( @@ -108,12 +92,11 @@ async fn do_authorize_task(app: &App, Request { token }: &Request) -> anyhow::Re fn evaluate_authorization( Snapshot { - taken: _, collections, data_planes, role_grants, tasks, - refresh_tx: _, + .. }: &Snapshot, shard_id: &str, shard_data_plane_fqdn: &str, @@ -188,22 +171,12 @@ fn evaluate_authorization( ); }; - let ops_kind = match task.spec_type { - models::CatalogType::Capture => "capture", - models::CatalogType::Collection => "derivation", - models::CatalogType::Materialization => "materialization", - models::CatalogType::Test => "test", - }; - // As a special case outside of the RBAC system, allow a task to write // to its designated partition within its ops collections. if required_role == models::Capability::Write && (collection.collection_name == task_data_plane.ops_logs_name || collection.collection_name == task_data_plane.ops_stats_name) - && journal_name_or_prefix.ends_with(&format!( - "/kind={ops_kind}/name={}/pivot=00", - labels::percent_encoding(&task.task_name).to_string(), - )) + && journal_name_or_prefix.ends_with(&super::ops_suffix(task)) { // Authorized write into designated ops partition. } else if tables::RoleGrant::is_authorized( @@ -214,10 +187,6 @@ fn evaluate_authorization( ) { // Authorized access through RBAC. } else { - let ops_suffix = format!( - "/kind={ops_kind}/name={}/pivot=00", - labels::percent_encoding(&task.task_name).to_string(), - ); tracing::warn!( %task.spec_type, %shard_id, @@ -225,7 +194,7 @@ fn evaluate_authorization( ?required_role, ops_logs=%task_data_plane.ops_logs_name, ops_stats=%task_data_plane.ops_stats_name, - %ops_suffix, + ops_suffix=%super::ops_suffix(task), "task authorization rejection context" ); anyhow::bail!( @@ -244,6 +213,9 @@ fn evaluate_authorization( Ok(( encoding_key, collection_data_plane.data_plane_fqdn.clone(), - collection_data_plane.broker_address.clone(), + super::maybe_rewrite_address( + task.data_plane_id != collection.data_plane_id, + &collection_data_plane.broker_address, + ), )) } diff --git a/crates/agent/src/api/authorize_user_collection.rs b/crates/agent/src/api/authorize_user_collection.rs new file mode 100644 index 0000000000..40e7d1633c --- /dev/null +++ b/crates/agent/src/api/authorize_user_collection.rs @@ -0,0 +1,122 @@ +use super::{App, Snapshot}; +use anyhow::Context; +use std::sync::Arc; + +type Request = models::authorizations::UserCollectionAuthorizationRequest; +type Response = models::authorizations::UserCollectionAuthorization; + +#[tracing::instrument( + skip(snapshot), + err(level = tracing::Level::WARN), +)] +async fn do_authorize_user_collection( + App { snapshot, .. }: &App, + super::ControlClaims { + sub: user_id, + email, + .. + }: super::ControlClaims, + Request { + collection: collection_name, + started_unix, + }: Request, +) -> anyhow::Result { + let (has_started, started_unix) = if started_unix == 0 { + (false, jsonwebtoken::get_current_timestamp()) + } else { + (true, started_unix) + }; + + loop { + match Snapshot::evaluate(snapshot, started_unix, |snapshot: &Snapshot| { + evaluate_authorization(snapshot, user_id, email.as_ref(), &collection_name) + }) { + Ok(response) => return Ok(response), + Err(Ok(retry_millis)) if has_started => { + return Ok(Response { + retry_millis, + ..Default::default() + }) + } + Err(Ok(retry_millis)) => { + () = tokio::time::sleep(std::time::Duration::from_millis(retry_millis)).await; + } + Err(Err(err)) => return Err(err), + } + } +} + +fn evaluate_authorization( + snapshot: &Snapshot, + user_id: uuid::Uuid, + user_email: Option<&String>, + collection_name: &models::Collection, +) -> anyhow::Result { + if !tables::UserGrant::is_authorized( + &snapshot.role_grants, + &snapshot.user_grants, + user_id, + collection_name, + models::Capability::Read, + ) { + anyhow::bail!( + "{} is not authorized to {collection_name}", + user_email.map(String::as_str).unwrap_or("user") + ); + } + + let Some(collection) = snapshot.collection_by_catalog_name(collection_name) else { + anyhow::bail!("collection {collection_name} is not known") + }; + let Some(data_plane) = snapshot.data_planes.get_by_key(&collection.data_plane_id) else { + anyhow::bail!("couldn't resolve collection {collection_name} data-plane") + }; + let Some(encoding_key) = data_plane.hmac_keys.first() else { + anyhow::bail!( + "collection data-plane {} has no configured HMAC keys", + data_plane.data_plane_name + ); + }; + let encoding_key = jsonwebtoken::EncodingKey::from_base64_secret(&encoding_key)?; + + let iat = jsonwebtoken::get_current_timestamp(); + let exp = iat + super::exp_seconds(); + let header = jsonwebtoken::Header::default(); + + let claims = super::DataClaims { + inner: proto_gazette::Claims { + cap: proto_gazette::capability::LIST | proto_gazette::capability::READ, + exp, + iat, + iss: data_plane.data_plane_fqdn.clone(), + sub: user_id.to_string(), + sel: proto_gazette::broker::LabelSelector { + include: Some(labels::build_set([ + ("name:prefix", collection.journal_template_name.as_str()), + (labels::COLLECTION, collection_name.as_str()), + ])), + exclude: None, + }, + }, + // TODO(johnny): Temporary support for data-plane-gateway. + prefixes: vec![collection.journal_template_name.clone()], + }; + let token = jsonwebtoken::encode(&header, &claims, &encoding_key) + .context("failed to encode authorized JWT")?; + + Ok(Response { + broker_address: super::maybe_rewrite_address(true, &data_plane.broker_address), + broker_token: token, + journal_name_prefix: collection.journal_template_name.clone(), + retry_millis: 0, + }) +} + +#[axum::debug_handler] +pub async fn authorize_user_collection( + axum::extract::State(app): axum::extract::State>, + axum::Extension(claims): axum::Extension, + super::Request(request): super::Request, +) -> axum::response::Response { + super::wrap(async move { do_authorize_user_collection(&app, claims, request).await }).await +} diff --git a/crates/agent/src/api/authorize_user_task.rs b/crates/agent/src/api/authorize_user_task.rs new file mode 100644 index 0000000000..06bdb7ac55 --- /dev/null +++ b/crates/agent/src/api/authorize_user_task.rs @@ -0,0 +1,162 @@ +use super::{App, Snapshot}; +use anyhow::Context; +use std::sync::Arc; + +type Request = models::authorizations::UserTaskAuthorizationRequest; +type Response = models::authorizations::UserTaskAuthorization; + +#[tracing::instrument( + skip(snapshot), + err(level = tracing::Level::WARN), +)] +async fn do_authorize_user_task( + App { snapshot, .. }: &App, + super::ControlClaims { + sub: user_id, + email, + .. + }: super::ControlClaims, + Request { + task: task_name, + started_unix, + }: Request, +) -> anyhow::Result { + let (has_started, started_unix) = if started_unix == 0 { + (false, jsonwebtoken::get_current_timestamp()) + } else { + (true, started_unix) + }; + + loop { + match Snapshot::evaluate(snapshot, started_unix, |snapshot: &Snapshot| { + evaluate_authorization(snapshot, user_id, email.as_ref(), &task_name) + }) { + Ok(response) => return Ok(response), + Err(Ok(retry_millis)) if has_started => { + return Ok(Response { + retry_millis, + ..Default::default() + }) + } + Err(Ok(retry_millis)) => { + () = tokio::time::sleep(std::time::Duration::from_millis(retry_millis)).await; + } + Err(Err(err)) => return Err(err), + } + } +} + +fn evaluate_authorization( + snapshot: &Snapshot, + user_id: uuid::Uuid, + user_email: Option<&String>, + task_name: &models::Name, +) -> anyhow::Result { + if !tables::UserGrant::is_authorized( + &snapshot.role_grants, + &snapshot.user_grants, + user_id, + task_name, + models::Capability::Read, + ) { + anyhow::bail!( + "{} is not authorized to {task_name}", + user_email.map(String::as_str).unwrap_or("user") + ); + } + + let Some(task) = snapshot.task_by_catalog_name(task_name) else { + anyhow::bail!("task {task_name} is not known") + }; + let Some(data_plane) = snapshot.data_planes.get_by_key(&task.data_plane_id) else { + anyhow::bail!("couldn't resolve task {task_name} data-plane") + }; + let Some(encoding_key) = data_plane.hmac_keys.first() else { + anyhow::bail!( + "task data-plane {} has no configured HMAC keys", + data_plane.data_plane_name + ); + }; + let encoding_key = jsonwebtoken::EncodingKey::from_base64_secret(&encoding_key)?; + + let (Some(ops_logs), Some(ops_stats)) = ( + snapshot.collection_by_catalog_name(&data_plane.ops_logs_name), + snapshot.collection_by_catalog_name(&data_plane.ops_stats_name), + ) else { + anyhow::bail!( + "couldn't resolve data-plane {} ops collections", + task.data_plane_id + ) + }; + + let ops_suffix = super::ops_suffix(task); + let ops_logs_journal = format!("{}{}", ops_logs.journal_template_name, &ops_suffix[1..]); + let ops_stats_journal = format!("{}{}", ops_stats.journal_template_name, &ops_suffix[1..]); + + let iat = jsonwebtoken::get_current_timestamp(); + let exp = iat + super::exp_seconds(); + let header = jsonwebtoken::Header::default(); + + let claims = super::DataClaims { + inner: proto_gazette::Claims { + cap: proto_gazette::capability::LIST | proto_gazette::capability::READ, + exp, + iat, + iss: data_plane.data_plane_fqdn.clone(), + sub: user_id.to_string(), + sel: proto_gazette::broker::LabelSelector { + include: Some(labels::build_set([ + ("name", ops_logs_journal.as_str()), + ("name", ops_stats_journal.as_str()), + ])), + exclude: None, + }, + }, + // TODO(johnny): Temporary support for data-plane-gateway. + prefixes: vec![ops_logs_journal.clone(), ops_stats_journal.clone()], + }; + let broker_token = jsonwebtoken::encode(&header, &claims, &encoding_key) + .context("failed to encode authorized JWT")?; + + let claims = super::DataClaims { + inner: proto_gazette::Claims { + cap: proto_gazette::capability::LIST + | proto_gazette::capability::READ + | proto_flow::capability::NETWORK_PROXY, + exp, + iat, + iss: claims.inner.iss, + sub: claims.inner.sub, + sel: proto_gazette::broker::LabelSelector { + include: Some(labels::build_set([( + "id:prefix", + task.shard_template_id.as_str(), + )])), + exclude: None, + }, + }, + prefixes: vec![task.task_name.to_string()], + }; + let reactor_token = jsonwebtoken::encode(&header, &claims, &encoding_key) + .context("failed to encode authorized JWT")?; + + Ok(Response { + broker_address: super::maybe_rewrite_address(true, &data_plane.broker_address), + broker_token, + ops_logs_journal, + ops_stats_journal, + reactor_address: super::maybe_rewrite_address(true, &data_plane.reactor_address), + reactor_token, + retry_millis: 0, + shard_id_prefix: task.shard_template_id.clone(), + }) +} + +#[axum::debug_handler] +pub async fn authorize_user_task( + axum::extract::State(app): axum::extract::State>, + axum::Extension(claims): axum::Extension, + super::Request(request): super::Request, +) -> axum::response::Response { + super::wrap(async move { do_authorize_user_task(&app, claims, request).await }).await +} diff --git a/crates/agent/src/api/create_data_plane.rs b/crates/agent/src/api/create_data_plane.rs index 623470ba50..79a1450b1a 100644 --- a/crates/agent/src/api/create_data_plane.rs +++ b/crates/agent/src/api/create_data_plane.rs @@ -56,7 +56,7 @@ async fn do_create_data_plane( id_generator, .. }: &App, - super::Claims { sub: user_id, .. }: super::Claims, + super::ControlClaims { sub: user_id, .. }: super::ControlClaims, Request { name, private, @@ -228,7 +228,7 @@ async fn do_create_data_plane( #[axum::debug_handler] pub async fn create_data_plane( axum::extract::State(app): axum::extract::State>, - axum::Extension(claims): axum::Extension, + axum::Extension(claims): axum::Extension, super::Request(request): super::Request, ) -> axum::response::Response { super::wrap(async move { do_create_data_plane(&app, claims, request).await }).await diff --git a/crates/agent/src/api/mod.rs b/crates/agent/src/api/mod.rs index 8f48337bf6..60e635d845 100644 --- a/crates/agent/src/api/mod.rs +++ b/crates/agent/src/api/mod.rs @@ -1,11 +1,14 @@ use axum::{http::StatusCode, response::IntoResponse}; use std::sync::{Arc, Mutex}; -mod authorize; +mod authorize_task; +mod authorize_user_collection; +mod authorize_user_task; mod create_data_plane; mod snapshot; mod update_l2_reporting; +use anyhow::Context; use snapshot::Snapshot; /// Request wraps a JSON-deserialized request type T which @@ -13,15 +16,19 @@ use snapshot::Snapshot; #[derive(Debug, Clone, Copy, Default)] pub struct Request(pub T); -/// Claims are the JWT claims attached to control-plane access tokens. -#[derive(Debug, Clone, serde::Deserialize)] -pub struct Claims { - // Note that many more fields, such as additional user metadata, - // are available if we choose to parse them. - pub sub: uuid::Uuid, - pub email: String, - pub iat: usize, - pub exp: usize, +/// ControlClaims are claims encoded within control-plane access tokens. +type ControlClaims = models::authorizations::ControlClaims; + +/// DataClaims are claims encoded within data-plane access tokens. +/// TODO(johnny): This should be a bare alias for proto_gazette::Claims. +/// We can do this once data-plane-gateway is updated to be a "dumb" proxy +/// which requires / forwards authorizations but doesn't inspect them. +#[derive(Debug, serde::Serialize, serde::Deserialize)] +struct DataClaims { + #[serde(flatten)] + inner: proto_gazette::Claims, + // prefixes exclusively used by legacy auth checks in data-plane-gateway. + prefixes: Vec, } /// Rejection is an error type of reasons why an API request may fail. @@ -48,7 +55,8 @@ pub fn build_router( jwt_secret: Vec, pg_pool: sqlx::PgPool, publisher: crate::publications::Publisher, -) -> axum::Router<()> { + allow_origin: &[String], +) -> anyhow::Result> { let jwt_secret = jsonwebtoken::DecodingKey::from_secret(&jwt_secret); let mut jwt_validation = jsonwebtoken::Validation::default(); @@ -68,8 +76,45 @@ pub fn build_router( use axum::routing::post; + let allow_origin = allow_origin + .into_iter() + .map(|o| o.parse()) + .collect::, _>>() + .context("failed to parse allowed origins")?; + + let allow_headers = [ + "Cache-Control", + "Content-Language", + "Content-Length", + "Content-Type", + "Expires", + "Last-Modified", + "Pragma", + "Authorization", + ] + .into_iter() + .map(|h| h.parse().unwrap()) + .collect::>(); + + let cors = tower_http::cors::CorsLayer::new() + .allow_methods(tower_http::cors::AllowMethods::mirror_request()) + .allow_origin(tower_http::cors::AllowOrigin::list(allow_origin)) + .allow_headers(allow_headers); + let schema_router = axum::Router::new() - .route("/authorize/task", post(authorize::authorize_task)) + .route("/authorize/task", post(authorize_task::authorize_task)) + .route( + "/authorize/user/task", + post(authorize_user_task::authorize_user_task) + .route_layer(axum::middleware::from_fn_with_state(app.clone(), authorize)) + .options(preflight_handler), + ) + .route( + "/authorize/user/collection", + post(authorize_user_collection::authorize_user_collection) + .route_layer(axum::middleware::from_fn_with_state(app.clone(), authorize)) + .options(preflight_handler), + ) .route( "/admin/create-data-plane", post(create_data_plane::create_data_plane) @@ -81,9 +126,14 @@ pub fn build_router( .route_layer(axum::middleware::from_fn_with_state(app.clone(), authorize)), ) .layer(tower_http::trace::TraceLayer::new_for_http()) + .layer(cors) .with_state(app); - schema_router + Ok(schema_router) +} + +async fn preflight_handler() -> impl IntoResponse { + (StatusCode::NO_CONTENT, "") } #[axum::async_trait] @@ -140,7 +190,7 @@ async fn authorize( mut req: axum::http::Request, next: axum::middleware::Next, ) -> axum::response::Response { - let token = match jsonwebtoken::decode::( + let token = match jsonwebtoken::decode::( bearer.token(), &app.jwt_secret, &app.jwt_validation, @@ -166,3 +216,26 @@ fn exp_seconds() -> u64 { // which spreads out load from re-authorization requests over time. rand::thread_rng().gen_range(40 * 60..80 * 60) } + +fn ops_suffix(task: &snapshot::SnapshotTask) -> String { + let ops_kind = match task.spec_type { + models::CatalogType::Capture => "capture", + models::CatalogType::Collection => "derivation", + models::CatalogType::Materialization => "materialization", + models::CatalogType::Test => "test", + }; + format!( + "/kind={ops_kind}/name={}/pivot=00", + labels::percent_encoding(&task.task_name).to_string(), + ) +} + +// Support the legacy data-plane by re-writing its internal service +// addresses to use the data-plane-gateway in external contexts. +fn maybe_rewrite_address(external: bool, address: &str) -> String { + if external && address.contains("svc.cluster.local:") { + "https://us-central1.v1.estuary-data.dev".to_string() + } else { + address.to_string() + } +} diff --git a/crates/agent/src/api/snapshot.rs b/crates/agent/src/api/snapshot.rs index e37507b23c..f49a587be1 100644 --- a/crates/agent/src/api/snapshot.rs +++ b/crates/agent/src/api/snapshot.rs @@ -9,12 +9,18 @@ pub struct Snapshot { pub taken: std::time::SystemTime, // Platform collections, indexed on `journal_template_name`. pub collections: Vec, + // Indices of `collections`, indexed on `collection_name`. + pub collections_idx_name: Vec, // Platform data-planes. pub data_planes: tables::DataPlanes, // Platform role grants. pub role_grants: tables::RoleGrants, + // Platform user grants. + pub user_grants: tables::UserGrants, // Platform tasks, indexed on `shard_template_id`. pub tasks: Vec, + // Indices of `tasks`, indexed on `task_name`. + pub tasks_idx_name: Vec, // `refresh` is take()-en when the current snapshot should be refreshed. pub refresh_tx: Option>, } @@ -90,6 +96,28 @@ impl Snapshot { } } + pub fn task_by_catalog_name<'s>(&'s self, name: &models::Name) -> Option<&'s SnapshotTask> { + self.tasks_idx_name + .binary_search_by(|i| self.tasks[*i].task_name.as_str().cmp(name)) + .ok() + .map(|index| { + let task = &self.tasks[self.tasks_idx_name[index]]; + assert_eq!(&task.task_name, name); + task + }) + } + + pub fn collection_by_catalog_name<'s>(&'s self, name: &str) -> Option<&'s SnapshotCollection> { + self.collections_idx_name + .binary_search_by(|i| self.collections[*i].collection_name.as_str().cmp(name)) + .ok() + .map(|index| { + let collection = &self.collections[self.collections_idx_name[index]]; + assert_eq!(collection.collection_name.as_str(), name); + collection + }) + } + fn begin_refresh<'m>( guard: std::sync::RwLockReadGuard<'_, Self>, mu: &'m std::sync::RwLock, @@ -113,9 +141,12 @@ pub fn seed() -> (Snapshot, futures::channel::oneshot::Receiver<()>) { Snapshot { taken: std::time::SystemTime::UNIX_EPOCH, collections: Vec::new(), + collections_idx_name: Vec::new(), data_planes: tables::DataPlanes::default(), role_grants: tables::RoleGrants::default(), + user_grants: tables::UserGrants::default(), tasks: Vec::new(), + tasks_idx_name: Vec::new(), refresh_tx: Some(next_tx), }, next_rx, @@ -193,6 +224,20 @@ async fn try_fetch(pg_pool: &sqlx::PgPool) -> anyhow::Result { .await .context("failed to fetch role_grants")?; + let user_grants = sqlx::query_as!( + tables::UserGrant, + r#" + select + user_id as "user_id: uuid::Uuid", + object_role as "object_role: models::Prefix", + capability as "capability: models::Capability" + from user_grants + "#, + ) + .fetch_all(pg_pool) + .await + .context("failed to fetch role_grants")?; + let mut tasks = sqlx::query_as!( SnapshotTask, r#" @@ -211,6 +256,7 @@ async fn try_fetch(pg_pool: &sqlx::PgPool) -> anyhow::Result { let data_planes = tables::DataPlanes::from_iter(data_planes); let role_grants = tables::RoleGrants::from_iter(role_grants); + let user_grants = tables::UserGrants::from_iter(user_grants); // Shard ID and journal name templates are prefixes which are always // extended with a slash-separated suffix. Avoid inadvertent matches @@ -225,20 +271,33 @@ async fn try_fetch(pg_pool: &sqlx::PgPool) -> anyhow::Result { tasks.sort_by(|t1, t2| t1.shard_template_id.cmp(&t2.shard_template_id)); collections.sort_by(|c1, c2| c1.journal_template_name.cmp(&c2.journal_template_name)); + let mut collections_idx_name = Vec::from_iter(0..collections.len()); + collections_idx_name.sort_by(|i1, i2| { + collections[*i1] + .collection_name + .cmp(&collections[*i2].collection_name) + }); + let mut tasks_idx_name = Vec::from_iter(0..tasks.len()); + tasks_idx_name.sort_by(|i1, i2| tasks[*i1].task_name.cmp(&tasks[*i2].task_name)); + tracing::info!( collections = collections.len(), data_planes = data_planes.len(), role_grants = role_grants.len(), tasks = tasks.len(), + user_grants = user_grants.len(), "fetched authorization snapshot", ); Ok(Snapshot { taken, collections, + collections_idx_name, data_planes, role_grants, + user_grants, tasks, + tasks_idx_name, refresh_tx: None, }) } diff --git a/crates/agent/src/api/update_l2_reporting.rs b/crates/agent/src/api/update_l2_reporting.rs index b2fe5e4993..9b122607ae 100644 --- a/crates/agent/src/api/update_l2_reporting.rs +++ b/crates/agent/src/api/update_l2_reporting.rs @@ -28,7 +28,7 @@ async fn do_update_l2_reporting( id_generator, .. }: &App, - super::Claims { sub: user_id, .. }: super::Claims, + super::ControlClaims { sub: user_id, .. }: super::ControlClaims, Request { default_data_plane, dry_run, @@ -228,7 +228,7 @@ export class Derivation extends Types.IDerivation {"# #[axum::debug_handler] pub async fn update_l2_reporting( axum::extract::State(app): axum::extract::State>, - axum::Extension(claims): axum::Extension, + axum::Extension(claims): axum::Extension, super::Request(request): super::Request, ) -> axum::response::Response { super::wrap(async move { do_update_l2_reporting(&app, claims, request).await }).await diff --git a/crates/agent/src/main.rs b/crates/agent/src/main.rs index 5be54e0a72..874b5bb521 100644 --- a/crates/agent/src/main.rs +++ b/crates/agent/src/main.rs @@ -42,8 +42,12 @@ struct Args { /// The port to listen on for API requests. #[clap(long, default_value = "8080", env = "API_PORT")] api_port: u16, + /// Whether to serve job handlers within this agent instance. #[clap(long = "serve-handlers", env = "SERVE_HANDLERS")] serve_handlers: bool, + /// Origin to allow in CORS contexts. May be specified multiple times. + #[clap(long = "allow-origin")] + allow_origin: Vec, } fn main() -> Result<(), anyhow::Error> { @@ -147,7 +151,8 @@ async fn async_main(args: Args) -> Result<(), anyhow::Error> { jwt_secret.into_bytes(), pg_pool.clone(), publisher.clone(), - ); + &args.allow_origin, + )?; let api_server = axum::serve(api_listener, api_router).with_graceful_shutdown(shutdown.clone()); let api_server = async move { anyhow::Result::Ok(api_server.await?) }; diff --git a/crates/models/Cargo.toml b/crates/models/Cargo.toml index f71c681e1e..f2bc214d63 100644 --- a/crates/models/Cargo.toml +++ b/crates/models/Cargo.toml @@ -23,6 +23,7 @@ superslice = { workspace = true } time = { workspace = true } unicode-normalization = { workspace = true } url = { workspace = true } +uuid = { workspace = true } validator = { workspace = true } [dev-dependencies] diff --git a/crates/models/src/authorizations.rs b/crates/models/src/authorizations.rs new file mode 100644 index 0000000000..53bf2d66d5 --- /dev/null +++ b/crates/models/src/authorizations.rs @@ -0,0 +1,140 @@ +use validator::Validate; + +/// ControlClaims are claims encoded within control-plane access tokens. +#[derive(Debug, Clone, serde::Deserialize)] +pub struct ControlClaims { + // Note that many more fields, such as additional user metadata, + // are available if we choose to parse them. + pub sub: uuid::Uuid, + pub email: Option, + pub iat: u64, + pub exp: u64, +} + +// Data-plane claims are represented by proto_gazette::Claims, +// which is not re-exported by this crate. + +/// TaskAuthorizationRequest is sent by data-plane reactors to request +/// an authorization to a collection which is sourced or produced. +#[derive(Debug, serde::Serialize, serde::Deserialize, schemars::JsonSchema)] +#[serde(rename_all = "camelCase")] +pub struct TaskAuthorizationRequest { + /// # JWT token to be authorized and signed. + /// JWT is signed by the requesting data-plane for authorization of a + /// task to a collection. + pub token: String, +} + +/// TaskAuthorization is an authorization granted to a task for the purpose of +/// interacting with collection journals which it sources or produces. +#[derive(Debug, Default, serde::Serialize, serde::Deserialize, schemars::JsonSchema)] +#[serde(rename_all = "camelCase")] +pub struct TaskAuthorization { + /// # JWT token which has been authorized for use. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub token: String, + /// # Address of Gazette brokers for the issued token. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub broker_address: String, + /// # Number of milliseconds to wait before retrying the request. + /// Non-zero if and only if token is not set. + pub retry_millis: u64, +} + +/// UserCollectionAuthorizationRequest requests an authorization to interact +/// with a collection within its data-plane on behalf of a user. +/// It must be accompanied by a control-plane Authorization token. +#[derive( + Debug, serde::Serialize, serde::Deserialize, schemars::JsonSchema, validator::Validate, +)] +#[serde(rename_all = "camelCase")] +pub struct UserCollectionAuthorizationRequest { + /// # Collection name to be authorized. + #[validate] + pub collection: crate::Collection, + /// # Unix timestamp, in seconds, at which the operation started. + /// If this is non-zero, it lower-bounds the time of an authorization + /// snapshot required to definitively reject an authorization. + /// + /// Snapshots taken prior to this time point that reject the request + /// will return a Response asking for the operation to be retried. + /// + /// If zero, the request will block server-side until it can be + /// definitively rejected. + #[serde(default)] + pub started_unix: u64, +} + +/// UserCollectionAuthorization is an authorization granted to a user for the +/// purpose of interacting with collection journals within its data-plane. +#[derive(Debug, Default, serde::Serialize, serde::Deserialize, schemars::JsonSchema)] +#[serde(rename_all = "camelCase")] +pub struct UserCollectionAuthorization { + /// # Address of Gazette brokers for the issued token. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub broker_address: String, + /// # JWT token which has been authorized for use with brokers. + /// The token is capable of LIST and READ for journals + /// of the requested collection. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub broker_token: String, + /// # Prefix of collection Journal names. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub journal_name_prefix: String, + /// # Number of milliseconds to wait before retrying the request. + /// Non-zero if and only if other fields are not set. + #[serde(default)] + pub retry_millis: u64, +} + +#[derive( + Debug, serde::Serialize, serde::Deserialize, schemars::JsonSchema, validator::Validate, +)] +#[serde(rename_all = "camelCase")] +pub struct UserTaskAuthorizationRequest { + /// # Task name to be authorized. + #[validate] + pub task: crate::Name, + /// # Unix timestamp, in seconds, at which the operation started. + /// If this is non-zero, it lower-bounds the time of an authorization + /// snapshot required to definitively reject an authorization. + /// + /// Snapshots taken prior to this time point that reject the request + /// will return a Response asking for the operation to be retried. + /// + /// If zero, the request will block server-side until it can be + /// definitively rejected. + #[serde(default)] + pub started_unix: u64, +} + +#[derive(Debug, Default, serde::Serialize, serde::Deserialize, schemars::JsonSchema)] +#[serde(rename_all = "camelCase")] +pub struct UserTaskAuthorization { + /// # Address of Gazette brokers for the issued token. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub broker_address: String, + /// # JWT token which has been authorized for use with brokers. + /// The token is capable of LIST and READ of task ops journals. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub broker_token: String, + /// # Name of the journal holding task logs. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub ops_logs_journal: String, + /// # Name of the journal holding task stats. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub ops_stats_journal: String, + /// # Address of Reactors for the issued token. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub reactor_address: String, + /// # JWT token which has been authorized for use with reactors. + /// The token is capable of LIST, READ, and NETWORK_PROXY of task shards. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub reactor_token: String, + /// # Number of milliseconds to wait before retrying the request. + /// Non-zero if and only if token is not set. + pub retry_millis: u64, + /// # Prefix of task Shard IDs. + #[serde(default, skip_serializing_if = "String::is_empty")] + pub shard_id_prefix: String, +} diff --git a/crates/models/src/lib.rs b/crates/models/src/lib.rs index 3ebf5a3984..715970dfd5 100644 --- a/crates/models/src/lib.rs +++ b/crates/models/src/lib.rs @@ -1,5 +1,6 @@ use std::collections::BTreeSet; +pub mod authorizations; mod captures; mod catalogs; pub mod collate; From 030133f8295bbc18bafa6183c846bd9a95de5025 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Tue, 17 Sep 2024 15:14:37 -0500 Subject: [PATCH 5/8] gazette: refactor Router to make clients cheap to clone Don't configure Router with a default service address. Instead, journal and shard Client instances are configured with a default service address and metadata which is _used_ by Router when picking a route. This makes it possible to cheaply clone Client instances and give each a different service address and authorization header, while still using the same underlying pool of gRPC connections. --- crates/agent/src/controlplane.rs | 10 +++---- crates/agent/src/publications/builds.rs | 7 ++--- crates/dekaf/src/topology.rs | 10 +++---- crates/gazette/src/journal/list.rs | 2 +- crates/gazette/src/journal/mod.rs | 30 +++++++++++++++++--- crates/gazette/src/journal/read.rs | 5 +++- crates/gazette/src/router.rs | 33 +++++++++++----------- crates/gazette/src/shard/mod.rs | 37 +++++++++++++++++++++---- 8 files changed, 91 insertions(+), 43 deletions(-) diff --git a/crates/agent/src/controlplane.rs b/crates/agent/src/controlplane.rs index 5252eabad1..263fc186b4 100644 --- a/crates/agent/src/controlplane.rs +++ b/crates/agent/src/controlplane.rs @@ -212,14 +212,14 @@ impl PGControlPlane { .context("failed to sign claims for data-plane")?; // Create the journal and shard clients that are used for interacting with the data plane - let journal_router = gazette::Router::new(&data_plane.broker_address, "local")?; + let router = gazette::Router::new("local"); let journal_client = gazette::journal::Client::new( - reqwest::Client::default(), - journal_router, + data_plane.broker_address, metadata.clone(), + router.clone(), ); - let shard_router = gazette::Router::new(&data_plane.reactor_address, "local")?; - let shard_client = gazette::shard::Client::new(shard_router, metadata); + let shard_client = + gazette::shard::Client::new(data_plane.reactor_address, metadata, router); Ok(( shard_client, diff --git a/crates/agent/src/publications/builds.rs b/crates/agent/src/publications/builds.rs index fa8cad2c9c..eeb69d36f0 100644 --- a/crates/agent/src/publications/builds.rs +++ b/crates/agent/src/publications/builds.rs @@ -156,11 +156,10 @@ pub async fn test_catalog( // Activate all derivations. let metadata = gazette::Metadata::default(); - let journal_router = gazette::Router::new(&broker_sock, "local")?; + let router = gazette::Router::new("local"); let journal_client = - gazette::journal::Client::new(Default::default(), journal_router, metadata.clone()); - let shard_router = gazette::Router::new(&consumer_sock, "local")?; - let shard_client = gazette::shard::Client::new(shard_router, metadata); + gazette::journal::Client::new(broker_sock.clone(), metadata.clone(), router.clone()); + let shard_client = gazette::shard::Client::new(consumer_sock.clone(), metadata, router); for built in catalog .built diff --git a/crates/dekaf/src/topology.rs b/crates/dekaf/src/topology.rs index 286f435a1c..66911314f4 100644 --- a/crates/dekaf/src/topology.rs +++ b/crates/dekaf/src/topology.rs @@ -248,7 +248,7 @@ impl Collection { gateway_url: String, } - let auth: [Auth; 1] = client + let [auth]: [Auth; 1] = client .rpc("gateway_auth_token", body) .build() .send() @@ -260,15 +260,15 @@ impl Collection { tracing::debug!( collection, - gateway = auth[0].gateway_url, + gateway = auth.gateway_url, "fetched data-plane token" ); let mut metadata = gazette::Metadata::default(); - metadata.bearer_token(&auth[0].token)?; + metadata.bearer_token(&auth.token)?; - let router = gazette::Router::new(&auth[0].gateway_url, "dekaf")?; - let client = journal::Client::new(Default::default(), router, metadata); + let router = gazette::Router::new("dekaf"); + let client = journal::Client::new(auth.gateway_url, metadata, router); Ok(client) } diff --git a/crates/gazette/src/journal/list.rs b/crates/gazette/src/journal/list.rs index c51c335fb6..142bcce412 100644 --- a/crates/gazette/src/journal/list.rs +++ b/crates/gazette/src/journal/list.rs @@ -51,7 +51,7 @@ impl Client { router: &crate::Router, req: &broker::ListRequest, ) -> crate::Result> { - let mut client = self.into_sub(router.route(None, false).await?); + let mut client = self.into_sub(router.route(None, false, &self.default).await?); Ok(client.list(req.clone()).await?.into_inner()) } } diff --git a/crates/gazette/src/journal/mod.rs b/crates/gazette/src/journal/mod.rs index af13df0203..db47ad7c1d 100644 --- a/crates/gazette/src/journal/mod.rs +++ b/crates/gazette/src/journal/mod.rs @@ -14,22 +14,43 @@ type SubClient = proto_grpc::broker::journal_client::JournalClient< #[derive(Clone)] pub struct Client { + default: broker::process_spec::Id, http: reqwest::Client, metadata: crate::Metadata, router: crate::Router, } impl Client { - pub fn new(http: reqwest::Client, router: crate::Router, metadata: crate::Metadata) -> Self { + /// Build a Client which dispatches request to the given default endpoint with the given Metadata. + /// The provider Router enables re-use of connections to brokers. + pub fn new(endpoint: String, metadata: crate::Metadata, router: crate::Router) -> Self { Self { + default: broker::process_spec::Id { + zone: String::new(), + suffix: endpoint, + }, metadata, - http, + http: reqwest::Client::default(), router, } } + /// Build a new Client which uses a different endpoint and metadata but re-uses underlying connections. + pub fn with_endpoint_and_metadata(&self, endpoint: String, metadata: crate::Metadata) -> Self { + Self { + default: broker::process_spec::Id { + zone: String::new(), + suffix: endpoint, + }, + http: self.http.clone(), + metadata, + router: self.router.clone(), + } + } + + /// Invoke the Gazette journal Apply API. pub async fn apply(&self, req: broker::ApplyRequest) -> crate::Result { - let mut client = self.into_sub(self.router.route(None, false).await?); + let mut client = self.into_sub(self.router.route(None, false, &self.default).await?); let resp = client .apply(req) @@ -40,11 +61,12 @@ impl Client { check_ok(resp.status(), resp) } + /// Invoke the Gazette journal ListFragments API. pub async fn list_fragments( &self, req: broker::FragmentsRequest, ) -> crate::Result { - let mut client = self.into_sub(self.router.route(None, false).await?); + let mut client = self.into_sub(self.router.route(None, false, &self.default).await?); let resp = client .list_fragments(req) diff --git a/crates/gazette/src/journal/read.rs b/crates/gazette/src/journal/read.rs index 443b39a146..ef50199fff 100644 --- a/crates/gazette/src/journal/read.rs +++ b/crates/gazette/src/journal/read.rs @@ -4,6 +4,9 @@ use futures::TryStreamExt; use proto_gazette::broker; impl Client { + /// Invoke the Gazette journal Read API. + /// This routine directly fetches journal fragments from cloud storage where possible, + /// rather than reading through the broker. pub fn read( self, mut req: broker::ReadRequest, @@ -46,7 +49,7 @@ impl Client { write_head: &mut i64, ) -> crate::Result<()> { let route = req.header.as_ref().and_then(|hdr| hdr.route.as_ref()); - let mut client = self.into_sub(self.router.route(route, false).await?); + let mut client = self.into_sub(self.router.route(route, false, &self.default).await?); // Fetch metadata first before we start the actual read. req.metadata_only = true; diff --git a/crates/gazette/src/router.rs b/crates/gazette/src/router.rs index 4ef323957b..5564f4d3df 100644 --- a/crates/gazette/src/router.rs +++ b/crates/gazette/src/router.rs @@ -21,26 +21,21 @@ pub struct Router { } struct Inner { states: std::sync::Mutex>, - default_endpoint: String, zone: String, } impl Router { /// Create a new Router with the given default service endpoint, /// which prefers to route to members in `zone` where possible. - pub fn new(default_endpoint: &str, zone: &str) -> Result { - let (default_endpoint, zone) = (default_endpoint.to_string(), zone.to_string()); + pub fn new(zone: &str) -> Self { + let zone = zone.to_string(); - let _endpoint = tonic::transport::Endpoint::from_shared(default_endpoint.clone()) - .map_err(|_err| Error::InvalidEndpoint(default_endpoint.clone()))?; - - Ok(Self { + Self { inner: Arc::new(Inner { states: Default::default(), - default_endpoint, zone, }), - }) + } } /// Map an optional broker::Route and indication of whether the "primary" @@ -52,10 +47,11 @@ impl Router { &self, route: Option<&broker::Route>, primary: bool, + default: &MemberId, ) -> Result { - let (index, state) = self.pick(route, primary); + let (index, state) = self.pick(route, primary, &default); - // Acquire `id`-specific, async-aware lock. + // Acquire MemberId-specific, async-aware lock. let mut state = state.lock().await; // Fast path: client is dialed and ready. @@ -67,7 +63,7 @@ impl Router { // Slow path: start dialing the endpoint. let channel = super::dial_channel(match index { Some(index) => &route.unwrap().endpoints[index], - None => &self.inner.default_endpoint, + None => &default.suffix, }) .await?; @@ -76,16 +72,19 @@ impl Router { Ok(channel) } - fn pick(&self, route: Option<&broker::Route>, primary: bool) -> (Option, DialState) { + fn pick( + &self, + route: Option<&broker::Route>, + primary: bool, + default: &MemberId, + ) -> (Option, DialState) { // Acquire non-async lock which *cannot* be held across an await point. let mut states = self.inner.states.lock().unwrap(); let index = pick(route, primary, &self.inner.zone, &states); - let default_id = MemberId::default(); - let id = match index { Some(index) => &route.unwrap().members[index], - None => &default_id, + None => default, }; let state = match states.get(id) { @@ -96,7 +95,7 @@ impl Router { (index, state) } - // Identify Channels which have not been used since the preceeding sweep, and close them. + // Identify Channels which have not been used since the preceding sweep, and close them. // As members come and go, Channels may no longer needed. // Call sweep() periodically to clear them out. pub fn sweep(&self) { diff --git a/crates/gazette/src/shard/mod.rs b/crates/gazette/src/shard/mod.rs index 099ec84980..aa54058f77 100644 --- a/crates/gazette/src/shard/mod.rs +++ b/crates/gazette/src/shard/mod.rs @@ -1,4 +1,4 @@ -use proto_gazette::consumer; +use proto_gazette::{broker, consumer}; use tonic::transport::Channel; // SubClient is the routed sub-client of Client. @@ -8,20 +8,43 @@ type SubClient = proto_grpc::consumer::shard_client::ShardClient< #[derive(Clone)] pub struct Client { + default: broker::process_spec::Id, metadata: crate::Metadata, router: crate::Router, } impl Client { - pub fn new(router: crate::Router, metadata: crate::Metadata) -> Self { - Self { metadata, router } + /// Build a Client which dispatches request to the given default endpoint with the given Metadata. + /// The provider Router enables re-use of connections to consumers. + pub fn new(endpoint: String, metadata: crate::Metadata, router: crate::Router) -> Self { + Self { + default: broker::process_spec::Id { + zone: String::new(), + suffix: endpoint, + }, + metadata, + router, + } } + /// Build a new Client which uses a different endpoint and metadata but re-uses underlying connections. + pub fn with_endpoint_and_metadata(&self, endpoint: String, metadata: crate::Metadata) -> Self { + Self { + default: broker::process_spec::Id { + zone: String::new(), + suffix: endpoint, + }, + metadata, + router: self.router.clone(), + } + } + + /// Invoke the Gazette shard List RPC. pub async fn list( &self, req: consumer::ListRequest, ) -> Result { - let mut client = self.into_sub(self.router.route(None, false).await?); + let mut client = self.into_sub(self.router.route(None, false, &self.default).await?); let resp = client .list(req) @@ -32,11 +55,12 @@ impl Client { check_ok(resp.status(), resp) } + /// Invoke the Gazette shard Apply RPC. pub async fn apply( &self, req: consumer::ApplyRequest, ) -> Result { - let mut client = self.into_sub(self.router.route(None, false).await?); + let mut client = self.into_sub(self.router.route(None, false, &self.default).await?); let resp = client .apply(req) @@ -47,11 +71,12 @@ impl Client { check_ok(resp.status(), resp) } + /// Invoke the Gazette shard Unassign RPC. pub async fn unassign( &self, req: consumer::UnassignRequest, ) -> Result { - let mut client = self.into_sub(self.router.route(None, false).await?); + let mut client = self.into_sub(self.router.route(None, false, &self.default).await?); let resp = client .unassign(req) From e91224482460c85ec91362bd3341351a45d402b7 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Tue, 17 Sep 2024 15:17:02 -0500 Subject: [PATCH 6/8] flowctl: refactor config and support new collection & task authorizations This change introduces the agent API to `flowctl`, which is the proverbial straw which motivated a deeper refactor of flowctl configuration. As a headline feature, `flowctl` supports the new task and collection authorization APIs and uses them in support of serving existing subcommands for reading collections, previews, and read ops logs or stats. Clean up management of access and refresh tokens by obtaining access tokens or generating refresh tokens prior to calling into a particular sub-command. Preserve the ability to run `flowctl` in an unauthenticated mode. Make it easier to use `flowctl` against a local stack by introducing alternative defaults when running under a "local" profile. Also fix handling of single-use refresh tokens, where we must retain the updated secret after using it to generate a new access token. We could now consider having `flowctl` create single-use refresh tokens rather than multi-use ones, but I didn't want to take that step just yet. Also fix mis-ordering of output when reading journals. Also fix OffsetNotYetAvailable error when reading a journal in non-blocking mode. Issue #1627 --- crates/flowctl/src/auth/mod.rs | 39 +--- crates/flowctl/src/auth/roles.rs | 15 +- crates/flowctl/src/catalog/delete.rs | 10 +- crates/flowctl/src/catalog/mod.rs | 23 +- crates/flowctl/src/catalog/publish.rs | 24 +- crates/flowctl/src/catalog/pull_specs.rs | 5 +- crates/flowctl/src/catalog/test.rs | 12 +- crates/flowctl/src/client.rs | 209 ++++++++++++++++++ crates/flowctl/src/collection/mod.rs | 46 ++-- crates/flowctl/src/collection/read/mod.rs | 73 +++--- crates/flowctl/src/config.rs | 220 +++++++++++-------- crates/flowctl/src/controlplane.rs | 170 -------------- crates/flowctl/src/dataplane.rs | 66 ------ crates/flowctl/src/draft/author.rs | 15 +- crates/flowctl/src/draft/develop.rs | 7 +- crates/flowctl/src/draft/mod.rs | 46 ++-- crates/flowctl/src/generate/mod.rs | 3 +- crates/flowctl/src/lib.rs | 111 +++++++--- crates/flowctl/src/local_specs.rs | 16 +- crates/flowctl/src/ops.rs | 186 ++-------------- crates/flowctl/src/output.rs | 2 +- crates/flowctl/src/poll.rs | 2 +- crates/flowctl/src/preview/journal_reader.rs | 60 +++-- crates/flowctl/src/preview/mod.rs | 44 +++- crates/flowctl/src/raw/mod.rs | 28 +-- crates/flowctl/src/raw/oauth.rs | 18 +- 26 files changed, 671 insertions(+), 779 deletions(-) create mode 100644 crates/flowctl/src/client.rs delete mode 100644 crates/flowctl/src/controlplane.rs delete mode 100644 crates/flowctl/src/dataplane.rs diff --git a/crates/flowctl/src/auth/mod.rs b/crates/flowctl/src/auth/mod.rs index 4f4c9acb38..17ead583a8 100644 --- a/crates/flowctl/src/auth/mod.rs +++ b/crates/flowctl/src/auth/mod.rs @@ -2,8 +2,6 @@ mod roles; use anyhow::Context; -use crate::controlplane; - #[derive(Debug, clap::Args)] #[clap(rename_all = "kebab-case")] pub struct Auth { @@ -47,21 +45,6 @@ pub enum Command { /// Unlike 'read' or 'write', the subject of an 'admin' grant also inherits /// capabilities granted to the object role from still-other roles. Roles(roles::Roles), - - /// Fetches and prints an auth token that can be used to access a Flow data plane. - /// - /// The returned token can be used to access the Flow data plane with 3rd party tools. - /// For example, you can use curl to access a private port of a running task by running: - /// ```ignore - /// curl -H "Authorization: Bearer $(flowctl auth data-plane-access-token --prefix myTenant/)" https://myPort.myHost.data-plane.example/ - /// ``` - DataPlaneAccessToken(DataPlaneAccessToken), -} - -#[derive(Debug, clap::Args)] -pub struct DataPlaneAccessToken { - #[clap(long, required = true)] - prefix: Vec, } #[derive(Debug, clap::Args)] @@ -76,12 +59,11 @@ impl Auth { match &self.cmd { Command::Login => do_login(ctx).await, Command::Token(Token { token }) => { - controlplane::configure_new_access_token(ctx, token.clone()).await?; + ctx.config.user_access_token = Some(token.clone()); println!("Configured access token."); Ok(()) } Command::Roles(roles) => roles.run(ctx).await, - Command::DataPlaneAccessToken(args) => do_data_plane_access_token(ctx, args).await, } } } @@ -89,7 +71,11 @@ impl Auth { async fn do_login(ctx: &mut crate::CliContext) -> anyhow::Result<()> { use crossterm::tty::IsTty; - let url = ctx.config().get_dashboard_url("/admin/api")?.to_string(); + let url = ctx + .config + .get_dashboard_url() + .join("/admin/api")? + .to_string(); println!("\nOpening browser to: {url}"); if let Err(_) = open::that(&url) { @@ -118,7 +104,7 @@ async fn do_login(ctx: &mut crate::CliContext) -> anyhow::Result<()> { // Copied credentials will often accidentally contain extra whitespace characters. let token = token.trim().to_string(); - controlplane::configure_new_access_token(ctx, token).await?; + ctx.config.user_access_token = Some(token); println!("\nConfigured access token."); Ok(()) } else { @@ -131,14 +117,3 @@ async fn do_login(ctx: &mut crate::CliContext) -> anyhow::Result<()> { ); } } - -async fn do_data_plane_access_token( - ctx: &mut crate::CliContext, - args: &DataPlaneAccessToken, -) -> anyhow::Result<()> { - let client = ctx.controlplane_client().await?; - let access = - crate::dataplane::fetch_data_plane_access_token(client, args.prefix.clone()).await?; - println!("{}", access.auth_token); - Ok(()) -} diff --git a/crates/flowctl/src/auth/roles.rs b/crates/flowctl/src/auth/roles.rs index 132da95a8e..41710db471 100644 --- a/crates/flowctl/src/auth/roles.rs +++ b/crates/flowctl/src/auth/roles.rs @@ -138,8 +138,7 @@ pub async fn do_list(ctx: &mut crate::CliContext) -> anyhow::Result<()> { } } let rows: Vec = api_exec_paginated( - ctx.controlplane_client() - .await? + ctx.client .from("combined_grants_ext") .select( vec![ @@ -177,8 +176,7 @@ pub async fn do_grant( // Upsert user grants to `user_grants` and role grants to `role_grants`. let rows: Vec = if let Some(subject_user_id) = subject_user_id { api_exec_paginated( - ctx.controlplane_client() - .await? + ctx.client .from("user_grants") .select(grant_revoke_columns()) .upsert( @@ -195,8 +193,7 @@ pub async fn do_grant( .await? } else if let Some(subject_role) = subject_role { api_exec_paginated( - ctx.controlplane_client() - .await? + ctx.client .from("role_grants") .select(grant_revoke_columns()) .upsert( @@ -231,8 +228,7 @@ pub async fn do_revoke( // Revoke user grants from `user_grants` and role grants from `role_grants`. let rows: Vec = if let Some(subject_user_id) = subject_user_id { api_exec_paginated( - ctx.controlplane_client() - .await? + ctx.client .from("user_grants") .select(grant_revoke_columns()) .eq("user_id", subject_user_id.to_string()) @@ -242,8 +238,7 @@ pub async fn do_revoke( .await? } else if let Some(subject_role) = subject_role { api_exec_paginated( - ctx.controlplane_client() - .await? + ctx.client .from("role_grants") .select(grant_revoke_columns()) .eq("subject_role", subject_role) diff --git a/crates/flowctl/src/catalog/delete.rs b/crates/flowctl/src/catalog/delete.rs index 6cb84be8fd..e501377632 100644 --- a/crates/flowctl/src/catalog/delete.rs +++ b/crates/flowctl/src/catalog/delete.rs @@ -69,9 +69,8 @@ pub async fn do_delete( type_selector: type_selector.clone(), }; - let client = ctx.controlplane_client().await?; let specs = catalog::fetch_live_specs::( - client.clone(), + &ctx.client, &list_args, vec![ "id", @@ -98,7 +97,7 @@ pub async fn do_delete( anyhow::bail!("delete operation cancelled"); } - let draft = draft::create_draft(client.clone()) + let draft = draft::create_draft(&ctx.client) .await .context("failed to create draft")?; println!( @@ -121,8 +120,7 @@ pub async fn do_delete( .collect::>(); api_exec::>( - ctx.controlplane_client() - .await? + ctx.client .from("draft_specs") //.select("catalog_name,spec_type") .upsert(serde_json::to_string(&draft_specs).unwrap()) @@ -131,7 +129,7 @@ pub async fn do_delete( .await?; tracing::debug!("added deletions to draft"); - draft::publish(client.clone(), "", draft.id, false).await?; + draft::publish(&ctx.client, "", draft.id, false).await?; // extra newline before, since `publish` will output a bunch of logs println!("\nsuccessfully deleted {} spec(s)", draft_specs.len()); diff --git a/crates/flowctl/src/catalog/mod.rs b/crates/flowctl/src/catalog/mod.rs index 2fc3ebd70d..4b27ad6c3c 100644 --- a/crates/flowctl/src/catalog/mod.rs +++ b/crates/flowctl/src/catalog/mod.rs @@ -4,7 +4,7 @@ mod pull_specs; mod test; use crate::{ - api_exec, api_exec_paginated, controlplane, + api_exec, api_exec_paginated, output::{to_table_row, CliOutput, JsonCell}, }; use anyhow::Context; @@ -226,7 +226,7 @@ impl Catalog { /// # Panics /// If the name_selector `name` and `prefix` are both non-empty. pub async fn fetch_live_specs( - cp_client: controlplane::Client, + client: &crate::Client, list: &List, columns: Vec<&'static str>, ) -> anyhow::Result> @@ -242,7 +242,7 @@ where panic!("cannot specify both 'name' and 'prefix' for filtering live specs"); } - let builder = cp_client.from("live_specs_ext").select(columns.join(",")); + let builder = client.from("live_specs_ext").select(columns.join(",")); let builder = list.type_selector.add_spec_type_filters(builder); // Drive the actual request(s) based on the name selector, since the arguments there may @@ -448,8 +448,7 @@ async fn do_list(ctx: &mut crate::CliContext, list_args: &List) -> anyhow::Resul columns.push("reads_from"); columns.push("writes_to"); } - let client = ctx.controlplane_client().await?; - let rows = fetch_live_specs::(client, list_args, columns).await?; + let rows = fetch_live_specs::(&ctx.client, list_args, columns).await?; ctx.write_all(rows, list_args.flows) } @@ -499,8 +498,7 @@ async fn do_history(ctx: &mut crate::CliContext, History { name }: &History) -> } } let rows: Vec = api_exec_paginated( - ctx.controlplane_client() - .await? + ctx.client .from("publication_specs_ext") .like("catalog_name", format!("{name}%")) .select( @@ -531,7 +529,7 @@ async fn do_draft( publication_id, }: &Draft, ) -> anyhow::Result<()> { - let draft_id = ctx.config().cur_draft()?; + let draft_id = ctx.config.selected_draft()?; #[derive(Deserialize)] struct Row { @@ -550,8 +548,7 @@ async fn do_draft( spec_type, } = if let Some(publication_id) = publication_id { api_exec( - ctx.controlplane_client() - .await? + ctx.client .from("publication_specs_ext") .eq("catalog_name", name) .eq("pub_id", publication_id.to_string()) @@ -561,8 +558,7 @@ async fn do_draft( .await? } else { api_exec( - ctx.controlplane_client() - .await? + ctx.client .from("live_specs") .eq("catalog_name", name) .not("is", "spec_type", "null") @@ -596,8 +592,7 @@ async fn do_draft( tracing::debug!(?draft_spec, "inserting draft"); let rows: Vec = api_exec( - ctx.controlplane_client() - .await? + ctx.client .from("draft_specs") .select("catalog_name,spec_type") .upsert(serde_json::to_string(&draft_spec).unwrap()) diff --git a/crates/flowctl/src/catalog/publish.rs b/crates/flowctl/src/catalog/publish.rs index 2e620235e4..ec3b6f9ec7 100644 --- a/crates/flowctl/src/catalog/publish.rs +++ b/crates/flowctl/src/catalog/publish.rs @@ -1,4 +1,4 @@ -use crate::{catalog::SpecSummaryItem, controlplane, draft, local_specs, CliContext}; +use crate::{catalog::SpecSummaryItem, draft, local_specs, CliContext}; use anyhow::Context; #[derive(Debug, clap::Args)] @@ -24,19 +24,17 @@ pub async fn do_publish(ctx: &mut CliContext, args: &Publish) -> anyhow::Result< // in common error scenarios. For example, we don't create the draft until after bundling, because // then we'd have to clean up the empty draft if the bundling fails. The very first thing is to create the client, // since that can fail due to missing/expired credentials. - let client = ctx.controlplane_client().await?; - anyhow::ensure!(args.auto_approve || std::io::stdin().is_tty(), "The publish command must be run interactively unless the `--auto-approve` flag is provided"); let (draft_catalog, _validations) = - local_specs::load_and_validate(client.clone(), &args.source).await?; + local_specs::load_and_validate(&ctx.client, &args.source).await?; - let draft = draft::create_draft(client.clone()).await?; + let draft = draft::create_draft(&ctx.client).await?; println!("Created draft: {}", &draft.id); tracing::info!(draft_id = %draft.id, "created draft"); - draft::upsert_draft_specs(client.clone(), draft.id, &draft_catalog).await?; + draft::upsert_draft_specs(&ctx.client, draft.id, &draft_catalog).await?; - let removed = draft::remove_unchanged(&client, draft.id).await?; + let removed = draft::remove_unchanged(&ctx.client, draft.id).await?; if !removed.is_empty() { println!("The following specs are identical to the currently published specs, and have been pruned from the draft:"); for name in removed.iter() { @@ -50,7 +48,7 @@ pub async fn do_publish(ctx: &mut CliContext, args: &Publish) -> anyhow::Result< if summary.is_empty() { println!("No specs would be changed by this publication, nothing to publish."); - try_delete_draft(client, draft.id).await; + try_delete_draft(&ctx.client, draft.id).await; return Ok(()); } @@ -59,17 +57,17 @@ pub async fn do_publish(ctx: &mut CliContext, args: &Publish) -> anyhow::Result< if !(args.auto_approve || prompt_to_continue().await) { println!("\nCancelling"); - try_delete_draft(client.clone(), draft.id).await; + try_delete_draft(&ctx.client, draft.id).await; anyhow::bail!("publish cancelled"); } println!("Proceeding to publish..."); let publish_result = - draft::publish(client.clone(), &args.default_data_plane, draft.id, false).await; + draft::publish(&ctx.client, &args.default_data_plane, draft.id, false).await; // The draft will have been deleted automatically if the publish was successful. if let Err(err) = publish_result.as_ref() { tracing::error!(draft_id = %draft.id, error = %err, "publication error"); - try_delete_draft(client, draft.id).await; + try_delete_draft(&ctx.client, draft.id).await; } publish_result.context("Publish failed")?; println!("\nPublish successful"); @@ -90,8 +88,8 @@ async fn prompt_to_continue() -> bool { } } -async fn try_delete_draft(client: controlplane::Client, draft_id: models::Id) { - if let Err(del_err) = draft::delete_draft(client.clone(), draft_id).await { +async fn try_delete_draft(client: &crate::Client, draft_id: models::Id) { + if let Err(del_err) = draft::delete_draft(client, draft_id).await { tracing::error!(draft_id = %draft_id, error = %del_err, "failed to delete draft"); } } diff --git a/crates/flowctl/src/catalog/pull_specs.rs b/crates/flowctl/src/catalog/pull_specs.rs index 45ab0d0936..1edf371893 100644 --- a/crates/flowctl/src/catalog/pull_specs.rs +++ b/crates/flowctl/src/catalog/pull_specs.rs @@ -23,10 +23,9 @@ pub struct PullSpecs { } pub async fn do_pull_specs(ctx: &mut CliContext, args: &PullSpecs) -> anyhow::Result<()> { - let client = ctx.controlplane_client().await?; // Retrieve identified live specifications. let live_specs = fetch_live_specs::( - client.clone(), + &ctx.client, &List { flows: false, name_selector: args.name_selector.clone(), @@ -58,7 +57,7 @@ pub async fn do_pull_specs(ctx: &mut CliContext, args: &PullSpecs) -> anyhow::Re let sources = local_specs::indirect_and_write_resources(sources)?; println!("Wrote {count} specifications under {target}."); - let () = local_specs::generate_files(client, sources).await?; + let () = local_specs::generate_files(&ctx.client, sources).await?; Ok(()) } diff --git a/crates/flowctl/src/catalog/test.rs b/crates/flowctl/src/catalog/test.rs index 41e9126c89..b5a01f7f53 100644 --- a/crates/flowctl/src/catalog/test.rs +++ b/crates/flowctl/src/catalog/test.rs @@ -16,24 +16,22 @@ pub struct TestArgs { /// and discoverable to users. There's also no need for any confirmation steps, since we're not /// actually modifying the published specs. pub async fn do_test(ctx: &mut CliContext, args: &TestArgs) -> anyhow::Result<()> { - let client = ctx.controlplane_client().await?; - let (draft_catalog, _validations) = - local_specs::load_and_validate(client.clone(), &args.source).await?; + local_specs::load_and_validate(&ctx.client, &args.source).await?; - let draft = draft::create_draft(client.clone()).await?; + let draft = draft::create_draft(&ctx.client).await?; println!("Created draft: {}", &draft.id); tracing::info!(draft_id = %draft.id, "created draft"); - let spec_rows = draft::upsert_draft_specs(client.clone(), draft.id, &draft_catalog).await?; + let spec_rows = draft::upsert_draft_specs(&ctx.client, draft.id, &draft_catalog).await?; println!("Running tests for catalog items:"); ctx.write_all(spec_rows, ())?; println!("Starting tests..."); // Technically, test is just a publish with the dry-run flag set to true. let publish_result = - draft::publish(client.clone(), &args.default_data_plane, draft.id, true).await; + draft::publish(&ctx.client, &args.default_data_plane, draft.id, true).await; - if let Err(del_err) = draft::delete_draft(client.clone(), draft.id).await { + if let Err(del_err) = draft::delete_draft(&ctx.client, draft.id).await { tracing::error!(draft_id = %draft.id, error = %del_err, "failed to delete draft"); } publish_result.context("Tests failed")?; diff --git a/crates/flowctl/src/client.rs b/crates/flowctl/src/client.rs new file mode 100644 index 0000000000..2f1fe540c3 --- /dev/null +++ b/crates/flowctl/src/client.rs @@ -0,0 +1,209 @@ +/// Client encapsulates sub-clients for various control-plane +/// and data-plane services that `flowctl` interacts with. +#[derive(Clone)] +pub struct Client { + // URL of the control-plane agent API. + 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, + // 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, +} + +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}")); + } + + // 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 + // underlying connections. + 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 { + agent_endpoint: config.get_agent_url().clone(), + http_client: reqwest::Client::new(), + journal_client, + pg_client, + shard_client, + user_access_token, + } + } + + pub fn from(&self, table: &str) -> postgrest::Builder { + self.pg_client.from(table) + } + + pub fn rpc(&self, function: &str, params: String) -> postgrest::Builder { + self.pg_client.rpc(function, params) + } + + pub fn is_authenticated(&self) -> bool { + self.user_access_token.is_some() + } + + pub async fn agent_unary( + &self, + path: &str, + request: &Request, + ) -> anyhow::Result + where + Request: serde::Serialize, + Response: serde::de::DeserializeOwned, + { + let mut builder = self + .http_client + .post(self.agent_endpoint.join(path)?) + .json(request); + + if let Some(token) = &self.user_access_token { + builder = builder.bearer_auth(token); + } + + let response = self + .http_client + .execute(builder.build()?) + .await? + .error_for_status()? + .json() + .await?; + + Ok(response) + } +} + +#[tracing::instrument(skip(client), err)] +pub async fn fetch_task_authorization( + client: &Client, + task: &str, +) -> anyhow::Result<( + String, + String, + String, + gazette::shard::Client, + gazette::journal::Client, +)> { + let started_unix = std::time::SystemTime::now() + .duration_since(std::time::SystemTime::UNIX_EPOCH) + .unwrap() + .as_secs(); + + let models::authorizations::UserTaskAuthorization { + broker_address, + broker_token, + ops_logs_journal, + ops_stats_journal, + reactor_address, + reactor_token, + shard_id_prefix, + retry_millis: _, + } = loop { + let response: models::authorizations::UserTaskAuthorization = client + .agent_unary( + "/authorize/user/task", + &models::authorizations::UserTaskAuthorizationRequest { + started_unix, + task: models::Name::new(task), + }, + ) + .await?; + + if response.retry_millis != 0 { + tracing::debug!(response.retry_millis, "sleeping before retrying request"); + () = tokio::time::sleep(std::time::Duration::from_millis(response.retry_millis)).await; + continue; + } + break response; + }; + + let mut md = gazette::Metadata::default(); + md.bearer_token(&reactor_token)?; + + let shard_client = client + .shard_client + .with_endpoint_and_metadata(reactor_address, md); + + let mut md = gazette::Metadata::default(); + md.bearer_token(&broker_token)?; + + let journal_client = client + .journal_client + .with_endpoint_and_metadata(broker_address, md); + + Ok(( + shard_id_prefix, + ops_logs_journal, + ops_stats_journal, + shard_client, + journal_client, + )) +} + +#[tracing::instrument(skip(client), err)] +pub async fn fetch_collection_authorization( + client: &Client, + collection: &str, +) -> anyhow::Result<(String, gazette::journal::Client)> { + let started_unix = std::time::SystemTime::now() + .duration_since(std::time::SystemTime::UNIX_EPOCH) + .unwrap() + .as_secs(); + + let models::authorizations::UserCollectionAuthorization { + broker_address, + broker_token, + journal_name_prefix, + retry_millis: _, + } = loop { + let response: models::authorizations::UserCollectionAuthorization = client + .agent_unary( + "/authorize/user/collection", + &models::authorizations::UserCollectionAuthorizationRequest { + started_unix, + collection: models::Collection::new(collection), + }, + ) + .await?; + + if response.retry_millis != 0 { + tracing::debug!(response.retry_millis, "sleeping before retrying request"); + () = tokio::time::sleep(std::time::Duration::from_millis(response.retry_millis)).await; + continue; + } + break response; + }; + + let mut md = gazette::Metadata::default(); + md.bearer_token(&broker_token)?; + + let journal_client = client + .journal_client + .with_endpoint_and_metadata(broker_address, md); + + Ok((journal_name_prefix, journal_client)) +} diff --git a/crates/flowctl/src/collection/mod.rs b/crates/flowctl/src/collection/mod.rs index 80adf777ca..6088a947f3 100644 --- a/crates/flowctl/src/collection/mod.rs +++ b/crates/flowctl/src/collection/mod.rs @@ -6,7 +6,6 @@ use proto_flow::flow; use proto_gazette::broker; use time::OffsetDateTime; -use crate::dataplane::journal_client_for; use crate::output::{to_table_row, CliOutput, JsonCell}; use self::read::ReadArgs; @@ -30,12 +29,13 @@ fn parse_partition_selector(arg: &str) -> Result broker::LabelSelector { + pub fn build_label_selector(&self, journal_name_prefix: String) -> broker::LabelSelector { assemble::journal_selector( + // Synthesize a minimal CollectionSpec to satisfy `journal_selector()`. &flow::CollectionSpec { name: self.collection.to_string(), partition_template: Some(broker::JournalSpec { - name: self.collection.to_string(), + name: journal_name_prefix, ..Default::default() }), ..Default::default() @@ -110,13 +110,15 @@ pub struct ListFragmentsArgs { #[clap(flatten)] pub selector: CollectionJournalSelector, - /// If provided, then the frament listing will include a pre-signed URL for each fragment, which is valid for the given duration. + /// If provided, then the fragment listing will include a pre-signed URL for each fragment, + /// which is valid for the given duration. /// This can be used to fetch fragment data directly from cloud storage. #[clap(long)] pub signature_ttl: Option, /// Only include fragments which were written within the provided duration from the present. - /// For example, `--since 10m` will only output fragments that have been written within the last 10 minutes. + /// For example, `--since 10m` will only output fragments that have been written within + /// the last 10 minutes. #[clap(long)] pub since: Option, } @@ -176,22 +178,23 @@ impl CliOutput for broker::fragments_response::Fragment { async fn do_list_fragments( ctx: &mut crate::CliContext, - args: &ListFragmentsArgs, + ListFragmentsArgs { + selector, + signature_ttl, + since, + }: &ListFragmentsArgs, ) -> Result<(), anyhow::Error> { - let client = journal_client_for( - ctx.controlplane_client().await?, - vec![args.selector.collection.clone()], - ) - .await?; + let (journal_name_prefix, client) = + crate::client::fetch_collection_authorization(&ctx.client, &selector.collection).await?; let list_resp = client .list(broker::ListRequest { - selector: Some(args.selector.build_label_selector()), + selector: Some(selector.build_label_selector(journal_name_prefix)), ..Default::default() }) .await?; - let start_time = if let Some(since) = args.since { + let start_time = if let Some(since) = *since { let timepoint = OffsetDateTime::now_utc() - *since; tracing::debug!(%since, begin_mod_time = %timepoint, "resolved --since to begin_mod_time"); timepoint.unix_timestamp() @@ -199,9 +202,7 @@ async fn do_list_fragments( 0 }; - let signature_ttl = args - .signature_ttl - .map(|ttl| std::time::Duration::from(*ttl).into()); + let signature_ttl = signature_ttl.map(|ttl| std::time::Duration::from(*ttl).into()); let mut fragments = Vec::with_capacity(32); for journal in list_resp.journals { let req = broker::FragmentsRequest { @@ -216,22 +217,19 @@ async fn do_list_fragments( fragments.extend(frag_resp.fragments); } - ctx.write_all(fragments, args.signature_ttl.is_some()) + ctx.write_all(fragments, signature_ttl.is_some()) } async fn do_list_journals( ctx: &mut crate::CliContext, - args: &CollectionJournalSelector, + selector: &CollectionJournalSelector, ) -> Result<(), anyhow::Error> { - let client = journal_client_for( - ctx.controlplane_client().await?, - vec![args.collection.clone()], - ) - .await?; + let (journal_name_prefix, client) = + crate::client::fetch_collection_authorization(&ctx.client, &selector.collection).await?; let list_resp = client .list(broker::ListRequest { - selector: Some(args.build_label_selector()), + selector: Some(selector.build_label_selector(journal_name_prefix)), ..Default::default() }) .await?; diff --git a/crates/flowctl/src/collection/read/mod.rs b/crates/flowctl/src/collection/read/mod.rs index 2ebc4ae953..81171247bd 100644 --- a/crates/flowctl/src/collection/read/mod.rs +++ b/crates/flowctl/src/collection/read/mod.rs @@ -1,11 +1,10 @@ -use crate::dataplane::{self}; use crate::{collection::CollectionJournalSelector, output::OutputType}; use anyhow::Context; use futures::StreamExt; use gazette::journal::ReadJsonLine; use proto_gazette::broker; +use std::io::Write; use time::OffsetDateTime; -use tokio::io::AsyncWriteExt; #[derive(clap::Args, Default, Debug, Clone)] pub struct ReadArgs { @@ -20,8 +19,6 @@ pub struct ReadArgs { /// the default. #[clap(long)] pub uncommitted: bool, - #[clap(skip)] - pub auth_prefixes: Vec, } /// Common definition for arguments specifying the begin and and bounds of a read command. @@ -42,18 +39,21 @@ pub struct ReadBounds { /// - Only uncommitted reads are supported /// - Any acknowledgements (documents with `/_meta/ack` value `true`) are also printed /// These limitations should all be addressed in the future when we add support for committed reads. -pub async fn read_collection(ctx: &mut crate::CliContext, args: &ReadArgs) -> anyhow::Result<()> { - if !args.uncommitted { +pub async fn read_collection( + ctx: &mut crate::CliContext, + ReadArgs { + selector, + bounds, + uncommitted, + }: &ReadArgs, +) -> anyhow::Result<()> { + if !uncommitted { anyhow::bail!("missing the `--uncommitted` flag. This flag is currently required, though a future release will add support for committed reads, which will be the default."); } // output can be either None or Some(OutputType::Json), but cannot be explicitly set to // anything else. _Eventually_, we may want to support outputting collection data as yaml // or a table, but certainly not right now. - if let Some(naughty_output_type) = ctx - .output_args() - .output - .filter(|ot| *ot != OutputType::Json) - { + if let Some(naughty_output_type) = ctx.output.output.filter(|ot| *ot != OutputType::Json) { let clap_enum = clap::ValueEnum::to_possible_value(&naughty_output_type) .expect("possible value cannot be None"); let name = clap_enum.get_name(); @@ -62,17 +62,12 @@ pub async fn read_collection(ctx: &mut crate::CliContext, args: &ReadArgs) -> an ); } - let auth_prefixes = if args.auth_prefixes.is_empty() { - vec![args.selector.collection.clone()] - } else { - args.auth_prefixes.clone() - }; - let cp_client = ctx.controlplane_client().await?; - let client = dataplane::journal_client_for(cp_client, auth_prefixes).await?; + let (journal_name_prefix, journal_client) = + crate::client::fetch_collection_authorization(&ctx.client, &selector.collection).await?; - let list_resp = client + let list_resp = journal_client .list(broker::ListRequest { - selector: Some(args.selector.build_label_selector()), + selector: Some(selector.build_label_selector(journal_name_prefix)), ..Default::default() }) .await @@ -84,7 +79,7 @@ pub async fn read_collection(ctx: &mut crate::CliContext, args: &ReadArgs) -> an .map(|j| j.spec.unwrap()) .collect::>(); - tracing::debug!(journal_count = journals.len(), collection = %args.selector.collection, "listed journals"); + tracing::debug!(journal_count = journals.len(), collection = %selector.collection, "listed journals"); let maybe_journal = journals.pop(); if !journals.is_empty() { // TODO: implement a sequencer and allow reading from multiple journals @@ -94,11 +89,19 @@ pub async fn read_collection(ctx: &mut crate::CliContext, args: &ReadArgs) -> an let journal = maybe_journal.ok_or_else(|| { anyhow::anyhow!( "collection '{}' does not exist or has never been written to (it has no journals)", - args.selector.collection + selector.collection ) })?; - let begin_mod_time = if let Some(since) = args.bounds.since { + read_collection_journal(journal_client, &journal.name, bounds).await +} + +pub async fn read_collection_journal( + journal_client: gazette::journal::Client, + journal_name: &str, + bounds: &ReadBounds, +) -> anyhow::Result<()> { + let begin_mod_time = if let Some(since) = bounds.since { let start_time = OffsetDateTime::now_utc() - *since; tracing::debug!(%since, begin_mod_time = %start_time, "resolved --since to begin_mod_time"); (start_time - OffsetDateTime::UNIX_EPOCH).as_seconds_f64() as i64 @@ -106,27 +109,23 @@ pub async fn read_collection(ctx: &mut crate::CliContext, args: &ReadArgs) -> an 0 }; - let mut lines = client.read_json_lines( + let mut lines = journal_client.read_json_lines( broker::ReadRequest { - journal: journal.name.clone(), + journal: journal_name.to_string(), offset: 0, - block: args.bounds.follow, + block: bounds.follow, begin_mod_time, ..Default::default() }, 1, ); - tracing::debug!(journal = %journal.name, "starting read of journal"); + tracing::debug!(%journal_name, "starting read of journal"); let policy = doc::SerPolicy::noop(); + let mut stdout = std::io::stdout(); while let Some(line) = lines.next().await { match line { - Err(err) if err.is_transient() => { - tracing::warn!(%err, "error reading collection (will retry)"); - tokio::time::sleep(std::time::Duration::from_secs(5)).await; - } - Err(err) => anyhow::bail!(err), Ok(ReadJsonLine::Meta(_)) => (), Ok(ReadJsonLine::Doc { root, @@ -134,8 +133,16 @@ pub async fn read_collection(ctx: &mut crate::CliContext, args: &ReadArgs) -> an }) => { let mut v = serde_json::to_vec(&policy.on(root.get())).unwrap(); v.push(b'\n'); - tokio::io::stdout().write_all(&v).await?; + () = stdout.write_all(&v)?; + } + Err(gazette::Error::BrokerStatus(broker::Status::OffsetNotYetAvailable)) => { + break; // Graceful EOF of non-blocking read. } + Err(err) if err.is_transient() => { + tracing::warn!(%err, "error reading collection (will retry)"); + tokio::time::sleep(std::time::Duration::from_secs(5)).await; + } + Err(err) => anyhow::bail!(err), } } diff --git a/crates/flowctl/src/config.rs b/crates/flowctl/src/config.rs index 6a7959c560..c03d4a6c9f 100644 --- a/crates/flowctl/src/config.rs +++ b/crates/flowctl/src/config.rs @@ -1,29 +1,114 @@ use anyhow::Context; -use serde::{Deserialize, Serialize}; use std::path::PathBuf; -lazy_static::lazy_static! { - static ref DEFAULT_DASHBOARD_URL: url::Url = url::Url::parse("https://dashboard.estuary.dev/").unwrap(); -} - -#[derive(Debug, Serialize, Deserialize, Default)] +/// Configuration of `flowctl`. +/// +/// We generally keep this minimal and prefer to use built-in default +/// or local value fallbacks, because that means we can update these +/// defaults in future releases of flowctl without breaking local +/// User configuration. +#[derive(Debug, Default, serde::Serialize, serde::Deserialize)] pub struct Config { + /// URL endpoint of the Flow control-plane Agent API. + #[serde(default, skip_serializing_if = "Option::is_none")] + pub agent_url: Option, /// URL of the Flow UI, which will be used as a base when flowctl generates links to it. + #[serde(default, skip_serializing_if = "Option::is_none")] pub dashboard_url: Option, /// ID of the current draft, or None if no draft is configured. + #[serde(default, skip_serializing_if = "Option::is_none")] pub draft: Option, - // Current access token, or None if no token is set. - pub api: Option, + /// Public (shared) anonymous token of the control-plane API. + #[serde(default, skip_serializing_if = "Option::is_none")] + pub pg_public_token: Option, + /// URL endpoint of the Flow control-plane PostgREST API. + #[serde(default, skip_serializing_if = "Option::is_none")] + pub pg_url: Option, + /// Users's access token for the control-plane API. + #[serde(default, skip_serializing_if = "Option::is_none")] + pub user_access_token: Option, + /// User's refresh token for the control-plane API, + /// used to generate access_token when it's unset or expires. + #[serde(default, skip_serializing_if = "Option::is_none")] + pub user_refresh_token: Option, + + #[serde(skip)] + is_local: bool, + + // Legacy API stanza, which is being phased out. + #[serde(default, skip_serializing)] + 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)] + endpoint: url::Url, + #[allow(dead_code)] + public_token: String, + access_token: String, + refresh_token: Option, } impl Config { + pub fn selected_draft(&self) -> anyhow::Result { + self.draft + .ok_or(anyhow::anyhow!("No draft is currently selected")) + } + + pub fn get_agent_url(&self) -> &url::Url { + if let Some(agent_url) = &self.agent_url { + agent_url + } else if self.is_local { + &LOCAL_AGENT_URL + } else { + &DEFAULT_AGENT_URL + } + } + + pub fn get_dashboard_url(&self) -> &url::Url { + if let Some(dashboard_url) = &self.dashboard_url { + dashboard_url + } else if self.is_local { + &LOCAL_DASHBOARD_URL + } else { + &DEFAULT_DASHBOARD_URL + } + } + + pub fn get_pg_public_token(&self) -> &str { + if let Some(pg_public_token) = &self.pg_public_token { + pg_public_token + } else if self.is_local { + LOCAL_PG_PUBLIC_TOKEN + } else { + DEFAULT_PG_PUBLIC_TOKEN + } + } + + pub fn get_pg_url(&self) -> &url::Url { + if let Some(pg_url) = &self.pg_url { + pg_url + } else if self.is_local { + &LOCAL_PG_URL + } else { + &DEFAULT_PG_URL + } + } + /// Loads the config corresponding to the given named `profile`. /// This loads from: /// - $HOME/.config/flowctl/${profile}.json on linux /// - $HOME/Library/Application Support/flowctl/${profile}.json on macos pub fn load(profile: &str) -> anyhow::Result { let config_file = Config::file_path(profile)?; - let config = match std::fs::read(&config_file) { + let mut config = match std::fs::read(&config_file) { Ok(v) => { let cfg = serde_json::from_slice(&v).with_context(|| { format!( @@ -43,9 +128,35 @@ impl Config { Config::default() } Err(err) => { - return Err(err).context("opening config"); + return Err(err).context("failed to read config"); } }; + + // Migrate legacy portions of the config. + if let Some(DeprecatedAPISection { + endpoint: _, + public_token: _, + access_token, + refresh_token, + }) = config.api.take() + { + config.user_access_token = Some(access_token); + config.user_refresh_token = refresh_token; + } + + // If a refresh token is not defined, attempt to parse one from the environment. + if config.user_refresh_token.is_none() { + if let Ok(env_token) = std::env::var(FLOW_AUTH_TOKEN) { + let decoded = base64::decode(env_token).context("FLOW_AUTH_TOKEN is not base64")?; + let token: RefreshToken = + serde_json::from_slice(&decoded).context("FLOW_AUTH_TOKEN is invalid JSON")?; + + tracing::info!("using refresh token from environment variable {FLOW_AUTH_TOKEN}"); + config.user_refresh_token = Some(token); + } + } + config.is_local = profile == "local"; + Ok(config) } @@ -83,86 +194,21 @@ impl Config { let path = Config::config_dir()?.join(format!("{profile}.json")); Ok(path) } - - pub fn cur_draft(&self) -> anyhow::Result { - match self.draft { - Some(draft) => Ok(draft), - None => { - anyhow::bail!("You must create or select a draft"); - } - } - } - - pub fn set_access_token(&mut self, access_token: String) { - // Don't overwrite the other fields of api if they are already present. - if let Some(api) = self.api.as_mut() { - api.access_token = access_token; - } else { - self.api = Some(API::managed(access_token)); - } - } - - pub fn set_refresh_token(&mut self, refresh_token: RefreshToken) { - // Don't overwrite the other fields of api if they are already present. - if let Some(api) = self.api.as_mut() { - api.refresh_token = Some(refresh_token); - } - } - - pub fn get_dashboard_url(&self, path: &str) -> anyhow::Result { - let base = self - .dashboard_url - .as_ref() - .unwrap_or(&*DEFAULT_DASHBOARD_URL); - let url = base.join(path).context( - "failed to join path to configured dashboard_url, the dashboard_url is likely invalid", - )?; - Ok(url) - } -} - -#[derive(Deserialize, Serialize, Debug)] -pub struct RefreshToken { - pub id: String, - pub secret: String, } -impl RefreshToken { - pub fn from_base64(encoded_token: &str) -> anyhow::Result { - let decoded = base64::decode(encoded_token).context("invalid base64")?; - let tk: RefreshToken = serde_json::from_slice(&decoded)?; - Ok(tk) - } - - pub fn to_base64(&self) -> anyhow::Result { - let ser = serde_json::to_vec(self)?; - Ok(base64::encode(&ser)) - } -} +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(); -#[derive(Debug, Serialize, Deserialize)] -pub struct API { - // URL endpoint of the Flow control-plane Rest API. - pub endpoint: url::Url, - // Public (shared) anonymous token of the control-plane API. - pub public_token: String, - // Secret access token of the control-plane API. - pub access_token: String, - // Secret refresh token of the control-plane API, used to generate access_token when it expires. - pub refresh_token: Option, + // 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(); } -pub const PUBLIC_TOKEN: &str = "eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJzdXBhYmFzZSIsInJlZiI6ImV5cmNubXV6enlyaXlwZGFqd2RrIiwicm9sZSI6ImFub24iLCJpYXQiOjE2NDg3NTA1NzksImV4cCI6MTk2NDMyNjU3OX0.y1OyXD3-DYMz10eGxzo1eeamVMMUwIIeOoMryTRAoco"; - -pub const ENDPOINT: &str = "https://eyrcnmuzzyriypdajwdk.supabase.co/rest/v1"; +const DEFAULT_PG_PUBLIC_TOKEN: &str = "eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJzdXBhYmFzZSIsInJlZiI6ImV5cmNubXV6enlyaXlwZGFqd2RrIiwicm9sZSI6ImFub24iLCJpYXQiOjE2NDg3NTA1NzksImV4cCI6MTk2NDMyNjU3OX0.y1OyXD3-DYMz10eGxzo1eeamVMMUwIIeOoMryTRAoco"; +const LOCAL_PG_PUBLIC_TOKEN: &str = "eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJzdXBhYmFzZS1kZW1vIiwicm9sZSI6ImFub24iLCJleHAiOjE5ODM4MTI5OTZ9.CRXP1A7WOeoJeXxjNni43kdQwgnWNReilDMblYTn_I0"; -impl API { - fn managed(access_token: String) -> Self { - Self { - endpoint: url::Url::parse(ENDPOINT).unwrap(), - public_token: PUBLIC_TOKEN.to_string(), - access_token, - refresh_token: None, - } - } -} +// 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/controlplane.rs b/crates/flowctl/src/controlplane.rs deleted file mode 100644 index bfaf4cb9ee..0000000000 --- a/crates/flowctl/src/controlplane.rs +++ /dev/null @@ -1,170 +0,0 @@ -use crate::config::{RefreshToken, ENDPOINT, PUBLIC_TOKEN}; -use crate::{api_exec, CliContext}; -use anyhow::Context; -use serde::Deserialize; -use std::fmt::{self, Debug}; -use std::ops::Deref; -use std::sync::Arc; - -/// A wafer-thin wrapper around a `Postgrest` client that makes it -/// cheaply cloneable and implements `Debug`. This allows us to create -/// a client once and then store it in the `CliContext` for future re-use. -/// This client implements `Deref`, so you can use it -/// just like you would the normal `Postgrest` client. -#[derive(Clone)] -pub struct Client(Arc, bool); - -impl Client { - /// Is this client authenticated (versus being an anonymous user)? - pub fn is_authenticated(&self) -> bool { - self.1 - } -} - -impl Debug for Client { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - // We can't really give a better debug impl since Postgrest - // keeps all of its members private. - f.write_str("controlplane::Client") - } -} - -impl Deref for Client { - type Target = postgrest::Postgrest; - - fn deref(&self) -> &Self::Target { - self.0.deref() - } -} - -#[derive(Deserialize)] -struct AccessTokenResponse { - access_token: String, -} - -/// Creates a new client. **you should instead call `CliContext::controlplane_client(&mut Self)`**, which -/// will re-use the existing client if possible. -// TODO(johnny): This really needs a deep overhaul. We're not updating refresh -// tokens as we should be, and the structure of the Config is problematic. -// I'm resisting refactoring it more substantially right now, but it needs it. -pub(crate) async fn new_client(ctx: &mut CliContext) -> anyhow::Result { - match ctx.config_mut().api { - Some(ref mut api) => { - let client = postgrest::Postgrest::new(api.endpoint.as_str()); - let client = client.insert_header("apikey", &api.public_token); - - // Try to give users a more friendly error message if we know their credentials are expired. - if let Err(e) = check_access_token(&api.access_token) { - if let Some(refresh_token) = &api.refresh_token { - let response = api_exec::(client.rpc( - "generate_access_token", - format!( - r#"{{"refresh_token_id": "{}", "secret": "{}"}}"#, - refresh_token.id, refresh_token.secret - ), - )) - .await?; - api.access_token = response.access_token; - } else { - return Err(e); - } - } - let client = - client.insert_header("Authorization", format!("Bearer {}", &api.access_token)); - Ok(Client(Arc::new(client), true)) - } - None => { - // If there has been no prior login, but FLOW_AUTH_TOKEN is available, we use that to - // generate an access_token and automatically login the user - if let Ok(env_token) = std::env::var(FLOW_AUTH_TOKEN) { - let client = postgrest::Postgrest::new(ENDPOINT); - let client = client.insert_header("apikey", PUBLIC_TOKEN); - - let refresh_token = RefreshToken::from_base64(&env_token)?; - let response = api_exec::(client.rpc( - "generate_access_token", - format!( - r#"{{"refresh_token_id": "{}", "secret": "{}"}}"#, - refresh_token.id, refresh_token.secret - ), - )) - .await?; - - let _jwt = check_access_token(&response.access_token)?; - ctx.config_mut() - .set_access_token(response.access_token.clone()); - - let client = client - .insert_header("Authorization", format!("Bearer {}", response.access_token)); - Ok(Client(Arc::new(client), true)) - } else { - tracing::warn!("You are not authenticated. Run `auth login` to login to Flow."); - - let client = postgrest::Postgrest::new(ENDPOINT); - let client = client.insert_header("apikey", PUBLIC_TOKEN); - Ok(Client(Arc::new(client), false)) - } - } - } -} - -pub async fn configure_new_access_token( - ctx: &mut CliContext, - access_token: String, -) -> anyhow::Result<()> { - let jwt = check_access_token(&access_token)?; - ctx.config_mut().set_access_token(access_token); - let client = ctx.controlplane_client().await?; - - let refresh_token = api_exec::(client.rpc( - "create_refresh_token", - r#"{"multi_use": true, "valid_for": "90d", "detail": "Created by flowctl"}"#, - )) - .await?; - ctx.config_mut().set_refresh_token(refresh_token); - - let message = if let Some(email) = jwt.email { - format!("Configured access token for user '{email}'") - } else { - "Configured access token".to_string() - }; - println!("{}", message); - Ok(()) -} - -fn check_access_token(access_token: &str) -> anyhow::Result { - let jwt = parse_jwt(access_token).context("invalid access_token")?; - // Try to give users a more friendly error message if we know their credentials are expired. - if jwt.is_expired() { - anyhow::bail!("access token is expired, please re-authenticate and then try again"); - } - Ok(jwt) -} - -const FLOW_AUTH_TOKEN: &str = "FLOW_AUTH_TOKEN"; -#[derive(Deserialize)] -struct JWT { - exp: i64, - email: Option, -} - -impl JWT { - fn is_expired(&self) -> bool { - let exp = time::OffsetDateTime::from_unix_timestamp(self.exp).unwrap_or_else(|err| { - tracing::error!(exp = self.exp, error = %err, "invalid exp in JWT"); - time::OffsetDateTime::UNIX_EPOCH - }); - time::OffsetDateTime::now_utc() >= exp - } -} - -fn parse_jwt(jwt: &str) -> anyhow::Result { - let payload = jwt - .split('.') - .nth(1) - .ok_or_else(|| anyhow::anyhow!("invalid JWT"))?; - let json_data = - base64::decode_config(payload, base64::URL_SAFE_NO_PAD).context("invalid JWT")?; - let data: JWT = serde_json::from_slice(&json_data).context("parsing JWT data")?; - Ok(data) -} diff --git a/crates/flowctl/src/dataplane.rs b/crates/flowctl/src/dataplane.rs deleted file mode 100644 index c10e0c5313..0000000000 --- a/crates/flowctl/src/dataplane.rs +++ /dev/null @@ -1,66 +0,0 @@ -use crate::controlplane; -use anyhow::Context; -use serde::Deserialize; - -#[derive(Deserialize, Clone, PartialEq, Debug)] -pub struct DataPlaneAccess { - #[serde(rename = "token")] - pub auth_token: String, - pub gateway_url: String, -} - -/// Fetches connection info for accessing a data plane for the given catalog namespace prefixes. -pub async fn fetch_data_plane_access_token( - client: controlplane::Client, - prefixes: Vec, -) -> anyhow::Result { - tracing::debug!(?prefixes, "requesting data-plane access token for prefixes"); - - let body = serde_json::to_string(&serde_json::json!({ - "prefixes": prefixes, - })) - .context("serializing prefix parameters")?; - - let req = client.rpc("gateway_auth_token", body).build(); - tracing::trace!(?req, "built request to execute"); - let resp = req - .send() - .await - .and_then(|r| r.error_for_status()) - .context("requesting data plane gateway auth token")?; - let json: serde_json::Value = resp.json().await?; - tracing::trace!(response_body = ?json, "got response from control-plane"); - let mut auths: Vec = - serde_json::from_value(json).context("failed to decode response")?; - let access = auths.pop().ok_or_else(|| { - anyhow::anyhow!( - "no data-plane access tokens were returned for the given prefixes, access is denied" - ) - })?; - if !auths.is_empty() { - let num_tokens = auths.len() + 1; - anyhow::bail!("received {} tokens for the given set of prefixes: {:?}. This is not yet implemented in flowctl", num_tokens, prefixes); - } - Ok(access) -} - -/// Returns an authenticated journal client that's authorized to the given prefixes. -pub async fn journal_client_for( - cp_client: controlplane::Client, - prefixes: Vec, -) -> anyhow::Result { - let DataPlaneAccess { - auth_token, - gateway_url, - } = fetch_data_plane_access_token(cp_client, prefixes).await?; - tracing::debug!(%gateway_url, "acquired data-plane-gateway access token"); - - let mut metadata = gazette::Metadata::default(); - metadata.bearer_token(&auth_token)?; - - let router = gazette::Router::new(&gateway_url, "local")?; - let client = gazette::journal::Client::new(Default::default(), router, metadata); - - tracing::debug!(%gateway_url, "connected data-plane client"); - Ok(client) -} diff --git a/crates/flowctl/src/draft/author.rs b/crates/flowctl/src/draft/author.rs index b65b105244..1b628bad82 100644 --- a/crates/flowctl/src/draft/author.rs +++ b/crates/flowctl/src/draft/author.rs @@ -1,4 +1,4 @@ -use crate::{api_exec, catalog::SpecSummaryItem, controlplane, local_specs}; +use crate::{api_exec, catalog::SpecSummaryItem, local_specs}; use anyhow::Context; use futures::{stream::FuturesOrdered, StreamExt}; use serde::Serialize; @@ -11,7 +11,7 @@ pub struct Author { source: String, } -pub async fn clear_draft(client: controlplane::Client, draft_id: models::Id) -> anyhow::Result<()> { +pub async fn clear_draft(client: &crate::Client, draft_id: models::Id) -> anyhow::Result<()> { tracing::info!(%draft_id, "clearing existing specs from draft"); api_exec::>( client @@ -25,7 +25,7 @@ pub async fn clear_draft(client: controlplane::Client, draft_id: models::Id) -> } pub async fn upsert_draft_specs( - client: controlplane::Client, + client: &crate::Client, draft_id: models::Id, draft: &tables::DraftCatalog, ) -> anyhow::Result> { @@ -130,12 +130,11 @@ pub async fn do_author( ctx: &mut crate::CliContext, Author { source }: &Author, ) -> anyhow::Result<()> { - let client = ctx.controlplane_client().await?; - let draft_id = ctx.config().cur_draft()?; - let (draft, _) = local_specs::load_and_validate(client.clone(), &source).await?; + let draft_id = ctx.config.selected_draft()?; + let (draft, _) = local_specs::load_and_validate(&ctx.client, &source).await?; - clear_draft(client.clone(), draft_id).await?; - let rows = upsert_draft_specs(client, draft_id, &draft).await?; + clear_draft(&ctx.client, draft_id).await?; + let rows = upsert_draft_specs(&ctx.client, draft_id, &draft).await?; ctx.write_all(rows, ()) } diff --git a/crates/flowctl/src/draft/develop.rs b/crates/flowctl/src/draft/develop.rs index 0d24d8f143..03c46d2d47 100644 --- a/crates/flowctl/src/draft/develop.rs +++ b/crates/flowctl/src/draft/develop.rs @@ -24,10 +24,9 @@ pub async fn do_develop( flat, }: &Develop, ) -> anyhow::Result<()> { - let draft_id = ctx.config().cur_draft()?; - let client = ctx.controlplane_client().await?; + let draft_id = ctx.config.selected_draft()?; let rows: Vec = api_exec_paginated( - client + ctx.client .from("draft_specs") .select("catalog_name,spec,spec_type,expect_pub_id") .not("is", "spec_type", "null") @@ -46,7 +45,7 @@ pub async fn do_develop( let sources = local_specs::indirect_and_write_resources(sources)?; println!("Wrote {count} specifications under {target}."); - let () = local_specs::generate_files(client, sources).await?; + let () = local_specs::generate_files(&ctx.client, sources).await?; Ok(()) } diff --git a/crates/flowctl/src/draft/mod.rs b/crates/flowctl/src/draft/mod.rs index f24960b23f..c577146272 100644 --- a/crates/flowctl/src/draft/mod.rs +++ b/crates/flowctl/src/draft/mod.rs @@ -2,7 +2,6 @@ use std::collections::BTreeSet; use crate::{ api_exec, api_exec_paginated, - controlplane::Client, output::{to_table_row, CliOutput, JsonCell}, }; use anyhow::Context; @@ -128,7 +127,7 @@ impl CliOutput for DraftRow { } } -pub async fn create_draft(client: Client) -> Result { +pub async fn create_draft(client: &crate::Client) -> Result { let row: DraftRow = api_exec( client .from("drafts") @@ -141,7 +140,10 @@ pub async fn create_draft(client: Client) -> Result { Ok(row) } -pub async fn delete_draft(client: Client, draft_id: models::Id) -> Result { +pub async fn delete_draft( + client: &crate::Client, + draft_id: models::Id, +) -> Result { let row: DraftRow = api_exec( client .from("drafts") @@ -156,10 +158,9 @@ pub async fn delete_draft(client: Client, draft_id: models::Id) -> Result anyhow::Result<()> { - let client = ctx.controlplane_client().await?; - let row = create_draft(client).await?; + let row = create_draft(&ctx.client).await?; - ctx.config_mut().draft = Some(row.id.clone()); + ctx.config.draft = Some(row.id.clone()); ctx.write_all(Some(row), ()) } @@ -181,11 +182,10 @@ async fn do_delete(ctx: &mut crate::CliContext) -> anyhow::Result<()> { to_table_row(self, &["/id", "/updated_at"]) } } - let client = ctx.controlplane_client().await?; - let draft_id = ctx.config().cur_draft()?; - let row = delete_draft(client, draft_id).await?; + let draft_id = ctx.config.selected_draft()?; + let row = delete_draft(&ctx.client, draft_id).await?; - ctx.config_mut().draft.take(); + ctx.config.draft.take(); ctx.write_all(Some(row), ()) } @@ -223,8 +223,7 @@ async fn do_describe(ctx: &mut crate::CliContext) -> anyhow::Result<()> { } } let rows: Vec = api_exec_paginated( - ctx.controlplane_client() - .await? + ctx.client .from("draft_specs_ext") .select( vec![ @@ -237,7 +236,7 @@ async fn do_describe(ctx: &mut crate::CliContext) -> anyhow::Result<()> { ] .join(","), ) - .eq("draft_id", ctx.config().cur_draft()?.to_string()), + .eq("draft_id", ctx.config.selected_draft()?.to_string()), ) .await?; @@ -269,8 +268,7 @@ async fn do_list(ctx: &mut crate::CliContext) -> anyhow::Result<()> { } } let rows: Vec = api_exec_paginated( - ctx.controlplane_client() - .await? + ctx.client .from("drafts_ext") .select("created_at,detail,id,num_specs,updated_at"), ) @@ -278,7 +276,7 @@ async fn do_list(ctx: &mut crate::CliContext) -> anyhow::Result<()> { // Decorate the id to mark the selected draft, but only if we're outputting a table let cur_draft = ctx - .config() + .config .draft .map(|id| id.to_string()) .unwrap_or_default(); @@ -298,7 +296,7 @@ async fn do_list(ctx: &mut crate::CliContext) -> anyhow::Result<()> { /// that are identical to their live specs, accounting for changes to inferred schemas. /// Returns the set of specs that were removed from the draft (as a `BTreeSet` so they're ordered). pub async fn remove_unchanged( - client: &Client, + client: &crate::Client, draft_id: models::Id, ) -> anyhow::Result> { #[derive(Deserialize)] @@ -321,8 +319,7 @@ async fn do_select( Select { id: select_id }: &Select, ) -> anyhow::Result<()> { let matched: Vec = api_exec_paginated( - ctx.controlplane_client() - .await? + ctx.client .from("drafts") .eq("id", select_id.to_string()) .select("id"), @@ -333,7 +330,7 @@ async fn do_select( anyhow::bail!("draft {select_id} does not exist"); } - ctx.config_mut().draft = Some(select_id.clone()); + ctx.config.draft = Some(select_id.clone()); do_list(ctx).await } @@ -342,19 +339,18 @@ async fn do_publish( data_plane_name: &str, dry_run: bool, ) -> anyhow::Result<()> { - let draft_id = ctx.config().cur_draft()?; - let client = ctx.controlplane_client().await?; + let draft_id = ctx.config.selected_draft()?; - publish(client, data_plane_name, draft_id, dry_run).await?; + publish(&ctx.client, data_plane_name, draft_id, dry_run).await?; if !dry_run { - ctx.config_mut().draft.take(); + ctx.config.draft.take(); } Ok(()) } pub async fn publish( - client: Client, + client: &crate::Client, default_data_plane_name: &str, draft_id: models::Id, dry_run: bool, diff --git a/crates/flowctl/src/generate/mod.rs b/crates/flowctl/src/generate/mod.rs index bdf4ea9d59..f30142ce3e 100644 --- a/crates/flowctl/src/generate/mod.rs +++ b/crates/flowctl/src/generate/mod.rs @@ -44,8 +44,7 @@ impl Generate { build::write_files(&project_root, files)?; - let client = ctx.controlplane_client().await?; - let () = local_specs::generate_files(client, draft).await?; + let () = local_specs::generate_files(&ctx.client, draft).await?; Ok(()) } } diff --git a/crates/flowctl/src/lib.rs b/crates/flowctl/src/lib.rs index e967436543..59649aaf11 100644 --- a/crates/flowctl/src/lib.rs +++ b/crates/flowctl/src/lib.rs @@ -5,10 +5,9 @@ use clap::Parser; mod auth; mod catalog; +mod client; mod collection; mod config; -mod controlplane; -mod dataplane; mod draft; mod generate; mod local_specs; @@ -19,8 +18,8 @@ mod poll; mod preview; mod raw; +use client::Client; use output::{Output, OutputType}; -use pagination::into_items; use poll::poll_while_queued; /// A command-line tool for working with Estuary Flow. @@ -97,38 +96,14 @@ pub enum Command { Raw(raw::Advanced), } -#[derive(Debug)] pub struct CliContext { + client: Client, config: config::Config, output: output::Output, - controlplane_client: Option, } impl CliContext { - /// Returns a client to the controlplane, creating a new one if necessary. - /// This function will return an error if the authentication credentials - /// are missing, invalid, or expired. - pub async fn controlplane_client(&mut self) -> anyhow::Result { - if self.controlplane_client.is_none() { - let client = controlplane::new_client(self).await?; - self.controlplane_client = Some(client.clone()) - } - Ok(self.controlplane_client.clone().unwrap()) - } - - pub fn config_mut(&mut self) -> &mut config::Config { - &mut self.config - } - - pub fn config(&self) -> &config::Config { - &self.config - } - - pub fn output_args(&self) -> &output::Output { - &self.output - } - - pub fn write_all(&mut self, items: I, table_alt: T::TableAlt) -> anyhow::Result<()> + fn write_all(&mut self, items: I, table_alt: T::TableAlt) -> anyhow::Result<()> where T: output::CliOutput, I: IntoIterator, @@ -140,7 +115,7 @@ impl CliContext { } } - pub fn get_output_type(&mut self) -> OutputType { + fn get_output_type(&mut self) -> OutputType { use crossterm::tty::IsTty; if let Some(ty) = self.output.output { @@ -157,12 +132,71 @@ impl CliContext { impl Cli { pub async fn run(&self) -> anyhow::Result<()> { - let config = config::Config::load(&self.profile)?; + 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 now = time::OffsetDateTime::now_utc(); + let exp = time::OffsetDateTime::from_unix_timestamp(claims.exp as i64).unwrap(); + + 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"); + } else { + tracing::warn!("You are not authenticated. Run `auth login` to login to Flow."); + } + let mut context = CliContext { + client: Client::new(&config), config, output, - controlplane_client: None, }; match &self.cmd { @@ -176,7 +210,7 @@ impl Cli { Command::Raw(advanced) => advanced.run(&mut context).await, }?; - context.config().write(&self.profile)?; + context.config.write(&self.profile)?; Ok(()) } @@ -213,7 +247,7 @@ where { use futures::TryStreamExt; - let pages = into_items(b).try_collect().await?; + let pages = pagination::into_items(b).try_collect().await?; Ok(pages) } @@ -258,3 +292,12 @@ 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/local_specs.rs b/crates/flowctl/src/local_specs.rs index 9ea708bf16..1f4ef366df 100644 --- a/crates/flowctl/src/local_specs.rs +++ b/crates/flowctl/src/local_specs.rs @@ -6,7 +6,7 @@ use tables::CatalogResolver; /// Load and validate sources and derivation connectors (only). /// Capture and materialization connectors are not validated. pub(crate) async fn load_and_validate( - client: crate::controlplane::Client, + client: &crate::Client, source: &str, ) -> anyhow::Result<(tables::DraftCatalog, tables::Validations)> { let source = build::arg_source_to_url(source, false)?; @@ -17,7 +17,7 @@ pub(crate) async fn load_and_validate( /// Load and validate sources and all connectors. pub(crate) async fn load_and_validate_full( - client: crate::controlplane::Client, + client: &crate::Client, source: &str, network: &str, ) -> anyhow::Result<(tables::DraftCatalog, tables::Validations)> { @@ -29,7 +29,7 @@ pub(crate) async fn load_and_validate_full( /// Generate connector files by validating sources with derivation connectors. pub(crate) async fn generate_files( - client: crate::controlplane::Client, + client: &crate::Client, sources: tables::DraftCatalog, ) -> anyhow::Result<()> { let (mut draft, built) = validate(client, true, false, true, sources, "").await; @@ -67,7 +67,7 @@ pub(crate) async fn load(source: &url::Url) -> tables::DraftCatalog { } async fn validate( - client: crate::controlplane::Client, + client: &crate::Client, noop_captures: bool, noop_derivations: bool, noop_materializations: bool, @@ -77,7 +77,11 @@ async fn validate( let source = &draft.fetches[0].resource.clone(); let project_root = build::project_root(source); - let mut live = Resolver { client }.resolve(draft.all_catalog_names()).await; + let mut live = Resolver { + client: client.clone(), + } + .resolve(draft.all_catalog_names()) + .await; let output = if !live.errors.is_empty() { // If there's a live catalog resolution error, surface it through built tables. @@ -191,7 +195,7 @@ pub(crate) fn pick_policy( } pub(crate) struct Resolver { - pub client: crate::controlplane::Client, + pub client: crate::Client, } impl tables::CatalogResolver for Resolver { diff --git a/crates/flowctl/src/ops.rs b/crates/flowctl/src/ops.rs index beb0b5fc9d..7649323d68 100644 --- a/crates/flowctl/src/ops.rs +++ b/crates/flowctl/src/ops.rs @@ -1,9 +1,4 @@ -use serde_json::Value; - -use crate::collection::{ - read::{read_collection, ReadArgs, ReadBounds}, - CollectionJournalSelector, -}; +use crate::collection::read::ReadBounds; #[derive(clap::Args, Debug)] pub struct Logs { @@ -14,17 +9,23 @@ pub struct Logs { pub bounds: ReadBounds, } +/// Selects a Flow task. +#[derive(clap::Args, Debug, Default, Clone)] +pub struct TaskSelector { + /// The name of the task + #[clap(long)] + pub task: String, +} + impl Logs { pub async fn run(&self, ctx: &mut crate::CliContext) -> anyhow::Result<()> { - let uncommitted = true; // logs reads are always 'uncommitted' because logs aren't written inside transactions. - let read_args = read_args( + read_task_ops_journal( + &ctx.client, &self.task.task, OpsCollection::Logs, &self.bounds, - uncommitted, - ); - read_collection(ctx, &read_args).await?; - Ok(()) + ) + .await } } @@ -34,161 +35,18 @@ pub enum OpsCollection { Stats, } -pub fn read_args( +pub async fn read_task_ops_journal( + client: &crate::Client, task_name: &str, collection: OpsCollection, bounds: &ReadBounds, - uncommitted: bool, -) -> ReadArgs { - let logs_or_stats = match collection { - OpsCollection::Logs => "logs", - OpsCollection::Stats => "stats", - }; - // Once we implement federated data planes, we'll need to update this to - // fetch the name of the data plane based on the tenant. - let collection = format!("ops.us-central1.v1/{logs_or_stats}"); +) -> 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?; - let mut include = std::collections::BTreeMap::new(); - include.insert( - "name".to_string(), - vec![Value::String(task_name.to_string())], - ); - let selector = CollectionJournalSelector { - collection, - partitions: Some(models::PartitionSelector { - include, - exclude: Default::default(), - }), - ..Default::default() + let journal_name = match collection { + OpsCollection::Logs => ops_logs_journal, + OpsCollection::Stats => ops_stats_journal, }; - ReadArgs { - selector, - uncommitted, - bounds: bounds.clone(), - auth_prefixes: vec![task_name.to_string()], - } -} - -/// Selects one or more Flow tasks within a single tenant. -#[derive(clap::Args, Debug, Default, Clone)] -pub struct TaskSelector { - /// The name of the task - #[clap(long)] - pub task: String, - // Selects all tasks with the given type - // - // Requires the `--tenant ` argument - //#[clap(long, arg_enum, requires("tenant"))] - //pub task_type: Option, - - // Selects all tasks within the given tenant - // - // The `--task-type` may also be specified to further limit the selection to only tasks of the given - // type. - //#[clap(long)] - //pub tenant: Option, -} - -/* -#[derive(Debug, clap::ArgEnum, PartialEq, Eq, Clone, Copy)] -pub enum TaskType { - Capture, - Derivation, - Materialization, -} - -impl TaskType { - fn label_value(&self) -> &'static str { - match self { - TaskType::Capture => "capture", - TaskType::Derivation => "derivation", - TaskType::Materialization => "materialization", - } - } -} - -impl TaskSelector { - fn tenant_name(&self) -> Result<&str, anyhow::Error> { - self.tenant - .as_deref() - .or_else(|| self.task.as_deref().map(tenant)) - .ok_or_else(|| anyhow::anyhow!("missing required task selector argument")) - } -} - -*/ - -#[cfg(test)] -mod test { - // use super::*; - - /* - #[test] - fn logs_translates_into_journals_read_commands() { - assert_logs_command( - TaskSelector { - task: Some(String::from("acmeCo/test/capture")), - ..Default::default() - }, - "estuary.dev/collection=ops/acmeCo/logs,estuary.dev/field/name=acmeCo%2Ftest%2Fcapture", - ); - assert_logs_command( - TaskSelector { - task_type: Some(TaskType::Capture), - tenant: Some("acmeCo".to_owned()), - task: None, - }, - "estuary.dev/collection=ops/acmeCo/logs,estuary.dev/field/kind=capture", - ); - assert_logs_command( - TaskSelector { - task_type: Some(TaskType::Derivation), - tenant: Some("acmeCo".to_owned()), - task: None, - }, - "estuary.dev/collection=ops/acmeCo/logs,estuary.dev/field/kind=derivation", - ); - assert_logs_command( - TaskSelector { - task_type: Some(TaskType::Materialization), - tenant: Some("acmeCo".to_owned()), - task: None, - }, - "estuary.dev/collection=ops/acmeCo/logs,estuary.dev/field/kind=materialization", - ); - assert_logs_command( - TaskSelector { - tenant: Some(String::from("acmeCo")), - ..Default::default() - }, - "estuary.dev/collection=ops/acmeCo/logs", - ); - } - - fn assert_logs_command(selector: TaskSelector, expected_label_selector: &str) { - let args = Args { - task: selector.clone(), - // Any extra arguments should be appended to whatever is generated - other: vec![String::from("an extra arg")], - }; - let cmd = args - .try_into_exec_external() - .expect("failed to convert args"); - let expected = ExecExternal::from(( - GO_FLOWCTL, - vec![ - "journals", - "read", - "--selector", - expected_label_selector, - "an extra arg", - ], - )); - assert_eq!( - expected, cmd, - "expected selector: {:?} to return journal selector: '{}', but got: {:?}", - selector, expected_label_selector, cmd - ); - } - */ + crate::collection::read::read_collection_journal(journal_client, &journal_name, bounds).await } diff --git a/crates/flowctl/src/output.rs b/crates/flowctl/src/output.rs index b43366cccb..e911668db0 100644 --- a/crates/flowctl/src/output.rs +++ b/crates/flowctl/src/output.rs @@ -15,7 +15,7 @@ pub enum OutputType { Json, /// Format output as YAML Yaml, - /// Format the output as a prett-printed table + /// Format the output as a pretty-printed table Table, } diff --git a/crates/flowctl/src/poll.rs b/crates/flowctl/src/poll.rs index 5159159052..07db9b1764 100644 --- a/crates/flowctl/src/poll.rs +++ b/crates/flowctl/src/poll.rs @@ -3,7 +3,7 @@ use serde::Deserialize; // Poll an async task in `table` having `id` until it's no longer queued. // While we're waiting print out logs under `logs_token`. pub async fn poll_while_queued( - client: &postgrest::Postgrest, + client: &crate::Client, table: &str, id: models::Id, logs_token: &str, diff --git a/crates/flowctl/src/preview/journal_reader.rs b/crates/flowctl/src/preview/journal_reader.rs index 9fa27d9234..2181bf2192 100644 --- a/crates/flowctl/src/preview/journal_reader.rs +++ b/crates/flowctl/src/preview/journal_reader.rs @@ -7,7 +7,7 @@ use proto_gazette::{broker, consumer}; /// collection journals. #[derive(Clone)] pub struct Reader { - control_plane: crate::controlplane::Client, + client: crate::Client, delay: std::time::Duration, } @@ -25,9 +25,9 @@ impl Reader { /// /// `delay` is an artificial, injected delay between a read and a subsequent checkpoint. /// It emulates back-pressure and encourages amortized transactions and reductions. - pub fn new(control_plane: crate::controlplane::Client, delay: std::time::Duration) -> Self { + pub fn new(client: &crate::Client, delay: std::time::Duration) -> Self { Self { - control_plane, + client: client.clone(), delay, } } @@ -38,50 +38,42 @@ impl Reader { mut resume: proto_gazette::consumer::Checkpoint, ) -> mpsc::Receiver> { let reader = coroutines::try_coroutine(move |mut co| async move { - // We must be able to access all sourced collections. - let access_prefixes = sources - .iter() - .map(|source| source.collection.clone()) - .collect(); - - let data_plane_client = - crate::dataplane::journal_client_for(self.control_plane, access_prefixes).await?; + // 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) + .map_ok(move |(_journal_name_prefix, client)| (source, client)) + })) + .await?; // Concurrently list the journals of every Source. - let journals: Vec<(&Source, Vec)> = - futures::future::try_join_all(sources.iter().map(|source| { - Self::list_journals(source, data_plane_client.clone()) - .map_ok(move |l| (source, l)) + let journals: Vec<(&Source, Vec, &gazette::journal::Client)> = + futures::future::try_join_all(sources.iter().map(|(source, client)| { + Self::list_journals(*source, client).map_ok(move |l| (*source, l, client)) })) .await?; - // Flatten into (binding, source, journal). - let journals: Vec<(u32, &Source, String)> = journals - .into_iter() + // Flatten into (binding, source, journal, client). + let journals: Vec<(u32, &Source, String, &gazette::journal::Client)> = journals + .iter() .enumerate() - .flat_map(|(binding, (source, journals))| { + .flat_map(|(binding, (source, journals, client))| { journals.into_iter().map(move |journal| { ( binding as u32, - source, + *source, format!("{};{}", journal.name, source.read_suffix), + *client, ) }) }) .collect(); // Map into a stream that yields lines from across all journals, as they're ready. - let mut journals = - futures::stream::select_all(journals.iter().map(|(binding, source, journal)| { - Self::read_journal_lines( - *binding, - data_plane_client.clone(), - journal, - &resume, - source, - ) - .boxed() - })); + let mut journals = futures::stream::select_all(journals.iter().map( + |(binding, source, journal, client)| { + Self::read_journal_lines(*binding, client, journal, &resume, source).boxed() + }, + )); // Reset-able timer for delivery of delayed checkpoints. let deadline = tokio::time::sleep(std::time::Duration::MAX); @@ -147,7 +139,7 @@ impl Reader { async fn list_journals( source: &Source, - client: gazette::journal::Client, + client: &gazette::journal::Client, ) -> anyhow::Result> { let resp = client .list(broker::ListRequest { @@ -179,7 +171,7 @@ impl Reader { fn read_journal_lines<'s>( binding: u32, - client: gazette::journal::Client, + client: &gazette::journal::Client, journal: &'s String, resume: &consumer::Checkpoint, source: &Source, @@ -198,7 +190,7 @@ impl Reader { .map(|b| b.seconds) .unwrap_or_default(); - let mut lines = client.read_json_lines( + let mut lines = client.clone().read_json_lines( broker::ReadRequest { journal: journal.clone(), offset, diff --git a/crates/flowctl/src/preview/mod.rs b/crates/flowctl/src/preview/mod.rs index 9568d11aa9..3d295d72ec 100644 --- a/crates/flowctl/src/preview/mod.rs +++ b/crates/flowctl/src/preview/mod.rs @@ -90,11 +90,10 @@ impl Preview { } = self; let source = build::arg_source_to_url(source, false)?; - let client = ctx.controlplane_client().await?; // TODO(johnny): validate only `name`, if presented. let (_sources, validations) = - local_specs::load_and_validate_full(client, source.as_str(), &network).await?; + local_specs::load_and_validate_full(&ctx.client, source.as_str(), &network).await?; let runtime = runtime::Runtime::new( true, // Allow local. @@ -134,7 +133,7 @@ impl Preview { } else { None }; - let journal_reader = journal_reader::Reader::new(ctx.controlplane_client().await?, delay); + let journal_reader = journal_reader::Reader::new(&ctx.client, delay); let initial_state = models::RawValue::from_str(initial_state).context("initial state is not valid JSON")?; @@ -274,8 +273,16 @@ async fn preview_capture( output_state: bool, output_apply: bool, ) -> anyhow::Result<()> { - let responses_rx = - runtime::harness::run_capture(delay, runtime, sessions, &spec, state, state_dir, timeout, output_apply); + let responses_rx = runtime::harness::run_capture( + delay, + runtime, + sessions, + &spec, + state, + state_dir, + timeout, + output_apply, + ); tokio::pin!(responses_rx); while let Some(response) = responses_rx.try_next().await? { @@ -303,7 +310,11 @@ async fn preview_capture( internal.checkpoint.unwrap_or_default(); let collection = "connectorState"; - let state_json = state.as_ref().map(|s| serde_json::to_string(s)).transpose()?.unwrap_or("{}".to_string()); + let state_json = state + .as_ref() + .map(|s| serde_json::to_string(s)) + .transpose()? + .unwrap_or("{}".to_string()); if output_state { print!("[{collection:?},{state_json}]\n"); @@ -350,7 +361,11 @@ async fn preview_derivation( tracing::debug!(stats=?ops::DebugJson(stats), "flushed"); } else if let Some(derive::response::StartedCommit { state }) = response.started_commit { let collection = "connectorState"; - let state_json = state.as_ref().map(|s| serde_json::to_string(s)).transpose()?.unwrap_or("{}".to_string()); + let state_json = state + .as_ref() + .map(|s| serde_json::to_string(s)) + .transpose()? + .unwrap_or("{}".to_string()); if output_state { print!("[{collection:?},{state_json}]\n"); } @@ -373,7 +388,14 @@ async fn preview_materialization( output_apply: bool, ) -> anyhow::Result<()> { let responses_rx = runtime::harness::run_materialize( - reader, runtime, sessions, &spec, state, state_dir, timeout, output_apply, + reader, + runtime, + sessions, + &spec, + state, + state_dir, + timeout, + output_apply, ); tokio::pin!(responses_rx); @@ -389,7 +411,11 @@ async fn preview_materialization( } else if let Some(materialize::response::StartedCommit { state }) = response.started_commit { let collection = "connectorState"; - let state_json = state.as_ref().map(|s| serde_json::to_string(s)).transpose()?.unwrap_or("{}".to_string()); + let state_json = state + .as_ref() + .map(|s| serde_json::to_string(s)) + .transpose()? + .unwrap_or("{}".to_string()); if output_state { print!("[{collection:?},{state_json}]\n"); } diff --git a/crates/flowctl/src/raw/mod.rs b/crates/flowctl/src/raw/mod.rs index 9214e11c12..e977e950d1 100644 --- a/crates/flowctl/src/raw/mod.rs +++ b/crates/flowctl/src/raw/mod.rs @@ -1,5 +1,5 @@ use crate::{ - collection::read::{read_collection, ReadBounds}, + collection::read::ReadBounds, local_specs, ops::{OpsCollection, TaskSelector}, }; @@ -163,14 +163,13 @@ pub struct Stats { impl Stats { pub async fn run(&self, ctx: &mut crate::CliContext) -> anyhow::Result<()> { - let read_args = crate::ops::read_args( + crate::ops::read_task_ops_journal( + &ctx.client, &self.task.task, OpsCollection::Stats, &self.bounds, - self.uncommitted, - ); - read_collection(ctx, &read_args).await?; - Ok(()) + ) + .await } } @@ -199,8 +198,7 @@ impl Advanced { } async fn do_get(ctx: &mut crate::CliContext, Get { table, query }: &Get) -> anyhow::Result<()> { - let client = ctx.controlplane_client().await?; - let req = client.from(table).build().query(query); + let req = ctx.client.from(table).build().query(query); tracing::debug!(?req, "built request to execute"); println!("{}", req.send().await?.text().await?); @@ -211,8 +209,7 @@ async fn do_update( ctx: &mut crate::CliContext, Update { table, query, body }: &Update, ) -> anyhow::Result<()> { - let client = ctx.controlplane_client().await?; - let req = client.from(table).update(body).build().query(query); + let req = ctx.client.from(table).update(body).build().query(query); tracing::debug!(?req, "built request to execute"); println!("{}", req.send().await?.text().await?); @@ -227,8 +224,7 @@ async fn do_rpc( body, }: &Rpc, ) -> anyhow::Result<()> { - let client = ctx.controlplane_client().await?; - let req = client.rpc(function, body).build().query(query); + let req = ctx.client.rpc(function, body.clone()).build().query(query); tracing::debug!(?req, "built request to execute"); println!("{}", req.send().await?.text().await?); @@ -236,8 +232,9 @@ async fn do_rpc( } async fn do_build(ctx: &mut crate::CliContext, build: &Build) -> anyhow::Result<()> { - let client = ctx.controlplane_client().await?; - let resolver = local_specs::Resolver { client }; + let resolver = local_specs::Resolver { + client: ctx.client.clone(), + }; let Build { db_path, @@ -299,8 +296,7 @@ async fn do_combine( ctx: &mut crate::CliContext, Combine { source, collection }: &Combine, ) -> anyhow::Result<()> { - let (_sources, validations) = - local_specs::load_and_validate(ctx.controlplane_client().await?, source).await?; + let (_sources, validations) = local_specs::load_and_validate(&ctx.client, source).await?; let collection = match validations .built_collections diff --git a/crates/flowctl/src/raw/oauth.rs b/crates/flowctl/src/raw/oauth.rs index 112e262e91..1736827d34 100644 --- a/crates/flowctl/src/raw/oauth.rs +++ b/crates/flowctl/src/raw/oauth.rs @@ -54,6 +54,9 @@ pub async fn do_oauth( injected_values, }: &Oauth, ) -> anyhow::Result<()> { + let Some(user_access_token) = &ctx.config.user_access_token else { + anyhow::bail!("This comment can only be run when authenticated"); + }; let source = build::arg_source_to_url(source, false)?; let draft = local_specs::surface_errors(local_specs::load(&source).await.into_result())?; @@ -175,13 +178,8 @@ pub async fn do_oauth( tracing::warn!( "Make sure that your application has {redirect_uri} set as an allowed redirect URL" ); - let api = ctx - .config - .api - .as_ref() - .expect("Cannot connect to edge functions"); - let mut oauth_endpoint = api.endpoint.clone(); + let mut oauth_endpoint = ctx.config.get_pg_url().clone(); oauth_endpoint.set_path("functions/v1/oauth"); #[derive(serde::Deserialize, serde::Serialize)] @@ -192,8 +190,8 @@ pub async fn do_oauth( let authorize_response_bytes = reqwest::Client::new() .post(oauth_endpoint.clone()) - .bearer_auth(api.access_token.to_owned()) - .header("apikey", api.public_token.to_owned()) + .bearer_auth(user_access_token) + .header("apikey", ctx.config.get_pg_public_token()) .json(&serde_json::json!({ "operation": "auth-url", "connector_config": { @@ -253,8 +251,8 @@ pub async fn do_oauth( let code_response = reqwest::Client::new() .post(oauth_endpoint) - .bearer_auth(api.access_token.to_owned()) - .header("apikey", api.public_token.to_owned()) + .bearer_auth(user_access_token) + .header("apikey", ctx.config.get_pg_public_token()) .json(&code_request_body) .send() .await? From 7b659c6bb1fec0240a3c93692bd71af3987a41b3 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Fri, 20 Sep 2024 00:13:49 -0500 Subject: [PATCH 7/8] go/network: refactored and updated connector networking feature Move connector networking entirely into this repo, from the legacy data-plane-gatweay repo, and significantly retool it along the way to: * Improve latency and throughput of HTTP reverse-proxy cases, by allowing the reverse proxy to use multiple pooled connections built atop network proxy RPCs with reasonable idle timeouts. This improves concurrency as many HTTP/2 requests can be in flight at once, and improves latency to the end user by ammortizing connections to reduce aggregate TCP and TLS startup time. * Improve user-facing error experience around misconfigurations, by often assuming an HTTP protocol and yielding a more informative error. * Overhauling metrics that we collect. * Updating the authorization flow, laying groundwork for the UI to use the /authorize/user/task API (but not requiring it just yet). --- go/network/auth.go | 120 ++++++ go/network/frontend.go | 392 ++++++++++++++++++ go/network/metrics.go | 51 +++ go/network/proxy_client.go | 196 +++++++++ .../proxy.go => network/proxy_server.go} | 55 +-- go/network/sni.go | 115 +++++ go/network/tap.go | 50 +++ go/runtime/flow_consumer.go | 41 +- go/runtime/task.go | 2 +- 9 files changed, 972 insertions(+), 50 deletions(-) create mode 100644 go/network/auth.go create mode 100644 go/network/frontend.go create mode 100644 go/network/metrics.go create mode 100644 go/network/proxy_client.go rename go/{runtime/proxy.go => network/proxy_server.go} (71%) create mode 100644 go/network/sni.go create mode 100644 go/network/tap.go diff --git a/go/network/auth.go b/go/network/auth.go new file mode 100644 index 0000000000..8f7c581293 --- /dev/null +++ b/go/network/auth.go @@ -0,0 +1,120 @@ +package network + +import ( + "errors" + "fmt" + "net/http" + "net/url" + + pb "go.gazette.dev/core/broker/protocol" + "google.golang.org/grpc/metadata" +) + +// verifyAuthorization ensures the request has an authorization which +// is valid for capability NETWORK_PROXY to `taskName`. +func verifyAuthorization(req *http.Request, verifier pb.Verifier, taskName string) error { + var bearer = req.Header.Get("authorization") + if bearer != "" { + // Pass. + } else if cookie, err := req.Cookie(AuthCookieName); err == nil { + bearer = fmt.Sprintf("Bearer %s", cookie.Value) + } else { + return errors.New("missing authorization") + } + + var _, cancel, claims, err = verifier.Verify( + metadata.NewIncomingContext( + req.Context(), + metadata.Pairs("authorization", bearer), + ), + 0, // TODO(johnny): Should be pf.Capability_NETWORK_PROXY. + ) + if err != nil { + return err + } + cancel() // We don't use the returned context. + + /* TODO(johnny): Inspect claims once UI is updated to use /authorize/user/task API. + if !claims.Selector.Matches(pb.MustLabelSet( + labels.TaskName, taskName, + )) { + return fmt.Errorf("invalid authorization for task %s (%s)", taskName, bearer) + } + */ + _ = claims + + return nil +} + +// startAuthRedirect redirect an interactive user to the dashboard, which will +// obtain a user task authorization and redirect back to us with it. +func startAuthRedirect(w http.ResponseWriter, req *http.Request, err error, dashboard *url.URL, taskName string) { + var query = make(url.Values) + query.Add("orig_url", "https://"+req.Host+req.URL.Path) + query.Add("task", taskName) + query.Add("prefix", taskName) + query.Add("err", err.Error()) // Informational. + + var target = dashboard.JoinPath("/data-plane-auth-req") + target.RawQuery = query.Encode() + + http.Redirect(w, req, target.String(), http.StatusTemporaryRedirect) +} + +// completeAuthRedirect handles path "/auth-redirect" as part of a redirect chain +// back from the dashboard. It expects a token parameter, which is set as a cookie, +// and an original URL which it in-turn redirects to. +func completeAuthRedirect(w http.ResponseWriter, req *http.Request) { + var params = req.URL.Query() + + var token = params.Get("token") + if token == "" { + http.Error(w, "URL is missing required `token` parameter", http.StatusBadRequest) + return + } + var origUrl = params.Get("orig_url") + if origUrl == "" { + http.Error(w, "URL is missing required `orig_url` parameter", http.StatusBadRequest) + return + } + + var cookie = &http.Cookie{ + Name: AuthCookieName, + Value: token, + Secure: true, + HttpOnly: true, + Path: "/", + } + http.SetCookie(w, cookie) + + http.Redirect(w, req, origUrl, http.StatusTemporaryRedirect) +} + +func scrubProxyRequest(req *http.Request, public bool) { + if _, ok := req.Header["User-Agent"]; !ok { + req.Header.Set("User-Agent", "") // Omit auto-added User-Agent. + } + + if public { + return // All done. + } + + // Scrub authentication token(s) from the request. + req.Header.Del("Authorization") + + // There's no `DeleteCookie` function, so we parse them, delete them all, and + // add them back in while filtering out the flow_auth cookie. + var cookies = req.Cookies() + req.Header.Del("Cookie") + + for _, cookie := range cookies { + if cookie.Name != AuthCookieName { + req.AddCookie(cookie) + } + } +} + +// AuthCookieName is the name of the cookie that we use for passing the JWT for interactive logins. +// It's name begins with '__Host-' in order to opt in to some additional security restrictions. +// See: https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Set-Cookie#cookie_prefixes +const AuthCookieName = "__Host-flow_auth" diff --git a/go/network/frontend.go b/go/network/frontend.go new file mode 100644 index 0000000000..5c2fbffc1e --- /dev/null +++ b/go/network/frontend.go @@ -0,0 +1,392 @@ +package network + +import ( + "context" + "crypto/tls" + "errors" + "fmt" + "io" + "math" + "net" + "net/http" + "net/http/httputil" + "net/url" + "reflect" + "slices" + "strings" + "sync" + "time" + + pf "github.com/estuary/flow/go/protocols/flow" + lru "github.com/hashicorp/golang-lru/v2" + pb "go.gazette.dev/core/broker/protocol" + pc "go.gazette.dev/core/consumer/protocol" + "golang.org/x/net/http2" +) + +// Frontend accepts connections over its configured Listener, +// matches on the TLS ServerName (SNI), and either: +// +// - Passes unmatched connections on to the tapped Listener +// - Attaches the connection to a connector via TCP proxy +// - Serves the connection as HTTP/2 using an authorizing reverse proxy +// - Or, returns an HTTP/1.1 descriptive error about what went wrong +// +// If Frontend is not running in a TLS context, all connections are +// trivially passed to the tapped Listener. +type Frontend struct { + controlAPI *url.URL + dashboard *url.URL + domains []string + networkClient pf.NetworkProxyClient + shardClient pc.ShardClient + verifier pb.Verifier + + listener net.Listener // Tapped listener. + tlsConfig *tls.Config // TLS config for accepted connections. + + // Forwarding channel for connections that are passed on. + fwdCh chan<- net.Conn + fwdErr *error + + // Cache of mappings from parsed to resolved SNIs. + sniCache *lru.Cache[parsedSNI, resolvedSNI] + + // Map of frontend connections currently undergoing TLS handshake. + handshake map[uintptr]*frontendConn + handshakeMu sync.Mutex +} + +// frontendConn is the state of a connection initiated +// by a user into the Frontend. +type frontendConn struct { + id uintptr + ctx context.Context + + // Raw and TLS-wrapped connections to the user. + raw net.Conn + tls *tls.Conn + + pass bool + parsed parsedSNI + resolved resolvedSNI + + // Error while resolving SNI to a mapped task: + // the SNI is invalid with respect to current task config. + sniErr error + // proxyClient dialed during TLS handshake. + // If set, we are acting as a TCP proxy. + dialed *proxyClient + // Error while dialing a shard for TCP proxy: + // this is an internal and usually-temporary error. + dialErr error +} + +func NewFrontend( + tap *Tap, + fqdn string, + controlAPI *url.URL, + dashboard *url.URL, + networkClient pf.NetworkProxyClient, + shardClient pc.ShardClient, + verifier pb.Verifier, +) (*Frontend, error) { + if tap.raw == nil { + return nil, fmt.Errorf("Tap has not tapped a raw net.Listener") + } + + var domains, fqdnParts []string = nil, strings.Split(fqdn, ".") + for i := range fqdnParts { + domains = append(domains, strings.Join(fqdnParts[i:], ".")) + } + var sniCache, err = lru.New[parsedSNI, resolvedSNI](1024) + if err != nil { + panic(err) + } + + var proxy = &Frontend{ + controlAPI: controlAPI, + dashboard: dashboard, + domains: domains, + networkClient: networkClient, + shardClient: shardClient, + verifier: verifier, + listener: tap.raw, + tlsConfig: tap.config, + fwdCh: tap.fwdCh, + fwdErr: &tap.fwdErr, + sniCache: sniCache, + handshake: make(map[uintptr]*frontendConn), + } + if proxy.tlsConfig != nil { + proxy.tlsConfig.GetConfigForClient = proxy.getTLSConfigForClient + } + + return proxy, nil +} + +func (p *Frontend) Serve(ctx context.Context) (_err error) { + defer func() { + // Forward terminal error to callers of Tap.Accept(). + *p.fwdErr = _err + close(p.fwdCh) + }() + + for { + var raw, err = p.listener.Accept() + if err != nil { + return err + } + if p.tlsConfig == nil { + p.fwdCh <- raw // Not serving TLS. + continue + } + go p.serveConn(ctx, raw) + } +} + +func (p *Frontend) serveConn(ctx context.Context, raw net.Conn) { + var conn = &frontendConn{ + id: reflect.ValueOf(raw).Pointer(), + ctx: ctx, + raw: raw, + tls: tls.Server(raw, p.tlsConfig), + } + + // Push `conn` onto the map of current handshakes. + p.handshakeMu.Lock() + p.handshake[conn.id] = conn + p.handshakeMu.Unlock() + + // The TLS handshake machinery will next call into getTLSConfigForClient(). + var err = conn.tls.HandshakeContext(conn.ctx) + + // Clear `conn` from the map of current handshakes. + p.handshakeMu.Lock() + delete(p.handshake, conn.id) + p.handshakeMu.Unlock() + + if err != nil { + handshakeCounter.WithLabelValues(err.Error()).Inc() // `err` is low-variance. + p.serveConnErr(conn.raw, 421, "This service may only be accessed using TLS, such as through an https:// URL.\n") + return + } + if conn.pass { + handshakeCounter.WithLabelValues("OKPass").Inc() + p.fwdCh <- conn.tls // Connection is not for us. + return + } + + if conn.sniErr != nil { + handshakeCounter.WithLabelValues("ErrSNI").Inc() + p.serveConnErr(conn.tls, 404, fmt.Sprintf("Failed to match the connection to a task:\n\t%s\n", conn.sniErr)) + } else if conn.dialErr != nil { + handshakeCounter.WithLabelValues("ErrDial").Inc() + p.serveConnErr(conn.tls, 503, fmt.Sprintf("Failed to connect to a task shard:\n\t%s\n", conn.dialErr)) + } else if conn.dialed != nil { + handshakeCounter.WithLabelValues("OkTCP").Inc() + p.serveConnTCP(conn) + } else { + handshakeCounter.WithLabelValues("OkHTTP").Inc() + p.serveConnHTTP(conn) + } +} + +func (p *Frontend) getTLSConfigForClient(hello *tls.ClientHelloInfo) (*tls.Config, error) { + p.handshakeMu.Lock() + var conn = p.handshake[reflect.ValueOf(hello.Conn).Pointer()] + p.handshakeMu.Unlock() + + // Exact match of the FQDN or a parent domain means it's not for us. + if slices.Contains(p.domains, hello.ServerName) { + conn.pass = true + return nil, nil + } + + var ok bool + var target, service, _ = strings.Cut(hello.ServerName, ".") + + // This block parses the SNI `target` and matches it to shard configuration. + if !slices.Contains(p.domains, service) { + conn.sniErr = fmt.Errorf("TLS ServerName %s is an invalid domain", hello.ServerName) + } else if conn.parsed, conn.sniErr = parseSNI(target); conn.sniErr != nil { + // No need to wrap error. + } else if conn.resolved, ok = p.sniCache.Get(conn.parsed); !ok { + // We didn't hit cache while resolving the parsed SNI. + // We must fetch matching shard specs to inspect their shard ID prefix and port config. + var shards []pc.ListResponse_Shard + shards, conn.sniErr = listShards(hello.Context(), p.shardClient, conn.parsed, "") + + if conn.sniErr != nil { + conn.sniErr = fmt.Errorf("fetching shards: %w", conn.sniErr) + } else if len(shards) == 0 { + conn.sniErr = errors.New("the requested subdomain does not match a known task and port combination") + } else { + conn.resolved = newResolvedSNI(conn.parsed, &shards[0].Spec) + p.sniCache.Add(conn.parsed, conn.resolved) + } + } + + if conn.sniErr == nil && conn.resolved.portProtocol != "" { + // We intend to TCP proxy to the connector. Dial the shard now so that + // we fail-fast during TLS handshake, instead of letting the client + // think it has a good connection. + var addr = conn.raw.RemoteAddr().String() + conn.dialed, conn.dialErr = dialShard( + conn.ctx, p.networkClient, p.shardClient, conn.parsed, conn.resolved, addr) + } + + var nextProtos []string + if conn.sniErr != nil || conn.dialErr != nil { + nextProtos = []string{"http/1.1"} // We'll send a descriptive HTTP/1.1 error. + } else if conn.dialed == nil { + nextProtos = []string{"h2"} // We'll reverse-proxy. The user MUST speak HTTP/2. + } else { + nextProtos = []string{conn.resolved.portProtocol} // We'll TCP proxy. + } + + return &tls.Config{ + Certificates: p.tlsConfig.Certificates, + NextProtos: nextProtos, + }, nil +} + +func (p *Frontend) serveConnTCP(user *frontendConn) { + var task, port, proto = user.resolved.taskName, user.parsed.port, user.resolved.portProtocol + userStartedCounter.WithLabelValues(task, port, proto).Inc() + + // Enable TCP keep-alive to ensure broken user connections are closed. + if tcpConn, ok := user.raw.(*net.TCPConn); ok { + tcpConn.SetKeepAlive(true) + tcpConn.SetKeepAlivePeriod(time.Minute) + } + + var ( + done = make(chan struct{}) + errBack error + errFwd error + shard = user.dialed + ) + + // Backward loop that reads from `shard` and writes to `user`. + // This may be sitting in a call to shard.Read() which races shard.Close(). + go func() { + _, errBack = io.Copy(user.tls, shard) + _ = user.tls.CloseWrite() + close(done) + }() + + // Forward loop that reads from `user` and writes to `shard`. + if _, errFwd = io.Copy(shard, user.tls); errFwd == nil { + _ = shard.rpc.CloseSend() // Allow reads to drain. + } else { + // `shard` write RST or `user` read error. + // Either way, we want to abort reads from `shard` => `user`. + _ = shard.Close() + } + <-done + + // If errBack is: + // - nil, then we read a clean EOF from shard and wrote it all to the user. + // - A user write RST, then errFwd MUST be an error read from the user and shard.Close() was already called. + // - A shard read error, then the shard RPC is already done. + _ = user.tls.Close() + + var status string + if errFwd != nil && errBack != nil { + status = "Err" + } else if errFwd != nil { + status = "ErrUser" + } else if errBack != nil { + status = "ErrShard" + } else { + status = "OK" + } + userHandledCounter.WithLabelValues(task, port, proto, status).Inc() +} + +func (p *Frontend) serveConnHTTP(user *frontendConn) { + var task, port, proto = user.resolved.taskName, user.parsed.port, user.resolved.portProtocol + userStartedCounter.WithLabelValues(task, port, proto).Inc() + + var transport = &http.Transport{ + DialTLSContext: func(ctx context.Context, network string, addr string) (net.Conn, error) { + return dialShard(ctx, p.networkClient, p.shardClient, user.parsed, user.resolved, user.raw.RemoteAddr().String()) + }, + // Connections are fairly cheap because they're "dialed" over an + // established gRPC HTTP/2 transport, but they do require a + // Open / Opened round trip and we'd like to re-use them. + // Note also that the maximum number of connections is implicitly + // bounded by http2.Server's MaxConcurrentStreams (default: 100), + // and the gRPC transport doesn't bound the number of streams. + IdleConnTimeout: 5 * time.Second, + MaxConnsPerHost: 0, // No limit. + MaxIdleConns: 0, // No limit. + MaxIdleConnsPerHost: math.MaxInt, + } + + var reverse = httputil.ReverseProxy{ + Director: func(req *http.Request) { + req.URL.Scheme = "https" + req.URL.Host = user.parsed.hostname + scrubProxyRequest(req, user.resolved.portIsPublic) + }, + ErrorHandler: func(w http.ResponseWriter, r *http.Request, err error) { + var body = fmt.Sprintf("Service temporarily unavailable: %s\nPlease retry in a moment.", err) + http.Error(w, body, http.StatusServiceUnavailable) + httpHandledCounter.WithLabelValues(task, port, "ErrProxy").Inc() + }, + ModifyResponse: func(r *http.Response) error { + httpHandledCounter.WithLabelValues(task, port, r.Status).Inc() + return nil + }, + Transport: transport, + } + + var handle = func(w http.ResponseWriter, req *http.Request) { + httpStartedCounter.WithLabelValues(task, port, req.Method).Inc() + + if user.resolved.portIsPublic { + reverse.ServeHTTP(w, req) + } else if req.URL.Path == "/auth-redirect" { + completeAuthRedirect(w, req) + httpHandledCounter.WithLabelValues(task, port, "CompleteAuth").Inc() + } else if err := verifyAuthorization(req, p.verifier, user.resolved.taskName); err == nil { + reverse.ServeHTTP(w, req) + } else if req.Method == "GET" && strings.Contains(req.Header.Get("accept"), "html") { + // Presence of "html" in Accept means this is probably a browser. + // Start a redirect chain to obtain an authorization cookie. + startAuthRedirect(w, req, err, p.dashboard, user.resolved.taskName) + httpHandledCounter.WithLabelValues(task, port, "StartAuth").Inc() + } else { + http.Error(w, err.Error(), http.StatusForbidden) + httpHandledCounter.WithLabelValues(task, port, "MissingAuth").Inc() + } + } + + (&http2.Server{ + // IdleTimeout can be generous: it's intended to catch broken TCP transports. + // MaxConcurrentStreams is an important setting left as the default (100). + IdleTimeout: time.Minute, + }).ServeConn(user.tls, &http2.ServeConnOpts{ + Context: user.ctx, + Handler: http.HandlerFunc(handle), + }) + + userHandledCounter.WithLabelValues(task, port, proto, "OK").Inc() +} + +func (f *Frontend) serveConnErr(conn net.Conn, status int, body string) { + // We're terminating this connection and sending a best-effort error. + // We don't know what the client is, or even if they speak HTTP, + // but we do only offer `http/1.1` during ALPN under an error condition. + var resp, _ = httputil.DumpResponse(&http.Response{ + ProtoMajor: 1, ProtoMinor: 1, + StatusCode: status, + Body: io.NopCloser(strings.NewReader(body)), + Close: true, + }, true) + + _, _ = conn.Write(resp) + _ = conn.Close() +} diff --git a/go/network/metrics.go b/go/network/metrics.go new file mode 100644 index 0000000000..73856f7461 --- /dev/null +++ b/go/network/metrics.go @@ -0,0 +1,51 @@ +package network + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" +) + +var handshakeCounter = promauto.NewCounterVec(prometheus.CounterOpts{ + Name: "flow_net_proxy_handshake_total", + Help: "counter of connections which attempted TLS handshake with the connector network proxy frontend", +}, []string{"status"}) + +var userStartedCounter = promauto.NewCounterVec(prometheus.CounterOpts{ + Name: "flow_net_proxy_user_started_total", + Help: "counter of started user-initiated connections to the connector network proxy frontend", +}, []string{"task", "port", "proto"}) + +var userHandledCounter = promauto.NewCounterVec(prometheus.CounterOpts{ + Name: "flow_net_proxy_user_handled_total", + Help: "counter of handled user-initiated connections to the connector network proxy frontend", +}, []string{"task", "port", "proto", "status"}) + +var shardStartedCounter = promauto.NewCounterVec(prometheus.CounterOpts{ + Name: "flow_net_proxy_shard_started_total", + Help: "counter of started shard connector client connections initiated by the network proxy", +}, []string{"task", "port", "proto"}) + +var shardHandledCounter = promauto.NewCounterVec(prometheus.CounterOpts{ + Name: "flow_net_proxy_shard_handled_total", + Help: "counter of handled shard connector client connections initiated by the network proxy", +}, []string{"task", "port", "proto", "status"}) + +var httpStartedCounter = promauto.NewCounterVec(prometheus.CounterOpts{ + Name: "flow_net_proxy_http_started_total", + Help: "counter of started reverse-proxy connector HTTP requests initiated by the network proxy", +}, []string{"task", "port", "method"}) + +var httpHandledCounter = promauto.NewCounterVec(prometheus.CounterOpts{ + Name: "flow_net_proxy_http_handled_total", + Help: "counter of handled reverse-proxy connector HTTP requests initiated by the network proxy", +}, []string{"task", "port", "status"}) + +var bytesReceivedCounter = promauto.NewCounterVec(prometheus.CounterOpts{ + Name: "flow_net_proxy_bytes_received_total", + Help: "counter of bytes received from user connections by the connector network proxy frontend", +}, []string{"task", "port", "proto"}) + +var bytesSentCounter = promauto.NewCounterVec(prometheus.CounterOpts{ + Name: "flow_net_proxy_bytes_sent_total", + Help: "counter of bytes sent to user connections by the connector network proxy frontend", +}, []string{"task", "port", "proto"}) diff --git a/go/network/proxy_client.go b/go/network/proxy_client.go new file mode 100644 index 0000000000..eeaa1c2c6d --- /dev/null +++ b/go/network/proxy_client.go @@ -0,0 +1,196 @@ +package network + +import ( + "context" + "fmt" + "io" + "math/rand/v2" + "net" + "strconv" + "time" + + pf "github.com/estuary/flow/go/protocols/flow" + "github.com/prometheus/client_golang/prometheus" + pb "go.gazette.dev/core/broker/protocol" + pc "go.gazette.dev/core/consumer/protocol" +) + +// proxyClient is a connection between the frontend +// and the shard assignment currently hosting the connector. +type proxyClient struct { + buf []byte // Unread remainder of last response. + picked pc.ListResponse_Shard // Picked primary shard assignment and route. + rpc pf.NetworkProxy_ProxyClient // Running RPC. + rxCh chan struct{} // Token for a capability to read from `rpc`. + labels []string // Metric labels. + nWrite prometheus.Counter // Accumulates bytes written. + nRead prometheus.Counter // Accumulates bytes read. +} + +func dialShard( + ctx context.Context, + networkClient pf.NetworkProxyClient, + shardClient pc.ShardClient, + parsed parsedSNI, + resolved resolvedSNI, + userAddr string, +) (*proxyClient, error) { + var labels = []string{resolved.taskName, parsed.port, resolved.portProtocol} + shardStartedCounter.WithLabelValues(labels...).Inc() + + var fetched, err = listShards(ctx, shardClient, parsed, resolved.shardIDPrefix) + if err != nil { + shardHandledCounter.WithLabelValues(append(labels, "ErrList")...).Inc() + return nil, fmt.Errorf("failed to list matching task shards: %w", err) + } + + // Pick a random primary. + rand.Shuffle(len(fetched), func(i, j int) { fetched[i], fetched[j] = fetched[j], fetched[i] }) + + var primary = -1 + for i := range fetched { + if fetched[i].Route.Primary != -1 { + primary = i + break + } + } + if primary == -1 { + shardHandledCounter.WithLabelValues(append(labels, "ErrNoPrimary")...).Inc() + return nil, fmt.Errorf("task has no ready primary shard assignment") + } + + var claims = pb.Claims{ + Capability: pf.Capability_NETWORK_PROXY, + Selector: pb.LabelSelector{ + Include: pb.MustLabelSet("id:prefix", resolved.shardIDPrefix), + }, + } + var picked = fetched[primary] + + rpc, err := networkClient.Proxy( + pb.WithDispatchRoute( + pb.WithClaims(ctx, claims), + picked.Route, + picked.Route.Members[picked.Route.Primary], + ), + ) + if err != nil { + shardHandledCounter.WithLabelValues(append(labels, "ErrProxy")...).Inc() + return nil, fmt.Errorf("failed to start network proxy RPC to task shard: %w", err) + } + + var port, _ = strconv.ParseUint(parsed.port, 10, 16) // parseSNI() already verified. + var openErr = rpc.Send(&pf.TaskNetworkProxyRequest{ + Open: &pf.TaskNetworkProxyRequest_Open{ + ShardId: picked.Spec.Id, + TargetPort: uint32(port), + ClientAddr: userAddr, + }, + }) + + opened, err := rpc.Recv() + if err != nil { + err = fmt.Errorf("failed to read opened response from task shard: %w", err) + } else if opened.OpenResponse == nil { + err = fmt.Errorf("task shard proxy RPC is missing expected OpenResponse") + } else if status := opened.OpenResponse.Status; status != pf.TaskNetworkProxyResponse_OK { + err = fmt.Errorf("task shard proxy RPC has non-ready status: %s", status) + } else if openErr != nil { + err = fmt.Errorf("failed to send open request: %w", err) + } + + if err != nil { + rpc.CloseSend() + _, _ = rpc.Recv() + shardHandledCounter.WithLabelValues(append(labels, "ErrOpen")...).Inc() + return nil, err + } + + var rxCh = make(chan struct{}, 1) + rxCh <- struct{}{} + + // Received and sent from the user's perspective. + var nWrite = bytesReceivedCounter.WithLabelValues(labels...) + var nRead = bytesSentCounter.WithLabelValues(labels...) + + return &proxyClient{ + buf: nil, + picked: picked, + rpc: rpc, + rxCh: rxCh, + labels: labels, + nWrite: nWrite, + nRead: nRead, + }, nil +} + +// Write to the shard proxy client. MUST not be called concurrently with Close. +func (pc *proxyClient) Write(b []byte) (n int, err error) { + if err = pc.rpc.Send(&pf.TaskNetworkProxyRequest{Data: b}); err != nil { + return 0, err + } + pc.nWrite.Add(float64(len(b))) + return len(b), nil +} + +// Read from the shard proxy client. MAY be called concurrently with Close. +func (pc *proxyClient) Read(b []byte) (n int, err error) { + if len(pc.buf) == 0 { + if _, ok := <-pc.rxCh; !ok { + return 0, io.EOF // RPC already completed. + } + + if rx, err := pc.rpc.Recv(); err != nil { + close(pc.rxCh) + + if err == io.EOF { + shardHandledCounter.WithLabelValues(append(pc.labels, "OK")...).Inc() + } else { + shardHandledCounter.WithLabelValues(append(pc.labels, "ErrRead")...).Inc() + } + return 0, err + } else { + pc.buf = rx.Data + pc.rxCh <- struct{}{} // Yield token. + pc.nRead.Add(float64(len(rx.Data))) + } + } + + var i = copy(b, pc.buf) + pc.buf = pc.buf[i:] + return i, nil +} + +// Close the proxy client. MAY be called concurrently with Read. +func (pc *proxyClient) Close() error { + // Note that http.Transport in particular will sometimes but not always race + // calls of Read() and Close(). We must ensure the RPC reads a final error as + // part of Close(), because we can't guarantee a current or future call to + // Read() will occur, but there may also be a raced Read() which will receive + // EOF after we CloseSend() -- and if we naively attempted another pc.rpc.Recv() + // it would block forever. + var _ = pc.rpc.CloseSend() + + if _, ok := <-pc.rxCh; !ok { + return nil // Read already completed. + } + close(pc.rxCh) // Future Read()'s return EOF. + + for { + if _, err := pc.rpc.Recv(); err == io.EOF { + shardHandledCounter.WithLabelValues(append(pc.labels, "OK")...).Inc() + return nil + } else if err != nil { + shardHandledCounter.WithLabelValues(append(pc.labels, "ErrClose")...).Inc() + return err + } + } +} + +func (sc *proxyClient) LocalAddr() net.Addr { return nil } +func (sc *proxyClient) RemoteAddr() net.Addr { return nil } +func (sc *proxyClient) SetDeadline(t time.Time) error { return nil } +func (sc *proxyClient) SetReadDeadline(t time.Time) error { return nil } +func (sc *proxyClient) SetWriteDeadline(t time.Time) error { return nil } + +var _ net.Conn = &proxyClient{} diff --git a/go/runtime/proxy.go b/go/network/proxy_server.go similarity index 71% rename from go/runtime/proxy.go rename to go/network/proxy_server.go index 379f9d329f..cbae98a3b5 100644 --- a/go/runtime/proxy.go +++ b/go/network/proxy_server.go @@ -1,28 +1,29 @@ -package runtime +package network import ( "context" "fmt" "io" "net" - "strconv" "sync/atomic" pf "github.com/estuary/flow/go/protocols/flow" "github.com/estuary/flow/go/protocols/ops" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" + pr "github.com/estuary/flow/go/protocols/runtime" pb "go.gazette.dev/core/broker/protocol" "go.gazette.dev/core/consumer" pc "go.gazette.dev/core/consumer/protocol" "golang.org/x/net/trace" ) -type proxyServer struct { - resolver *consumer.Resolver +// ProxyServer is the "backend" of connector networking. +// It accepts Proxy requests from the Frontend and connects them to the +// corresponding TCP port of a running connector container. +type ProxyServer struct { + Resolver *consumer.Resolver } -func (ps *proxyServer) Proxy(claims pb.Claims, stream pf.NetworkProxy_ProxyServer) (_err error) { +func (ps *ProxyServer) Proxy(claims pb.Claims, stream pf.NetworkProxy_ProxyServer) (_err error) { var ctx = stream.Context() var open, err = stream.Recv() @@ -31,11 +32,8 @@ func (ps *proxyServer) Proxy(claims pb.Claims, stream pf.NetworkProxy_ProxyServe } else if err := validateOpen(open); err != nil { return fmt.Errorf("invalid open proxy message: %w", err) } - var labels = []string{ - open.Open.ShardId.String(), strconv.Itoa(int(open.Open.TargetPort)), - } - resolution, err := ps.resolver.Resolve(consumer.ResolveArgs{ + resolution, err := ps.Resolver.Resolve(consumer.ResolveArgs{ Context: ctx, Claims: claims, MayProxy: false, @@ -58,7 +56,9 @@ func (ps *proxyServer) Proxy(claims pb.Claims, stream pf.NetworkProxy_ProxyServe } // Resolve the target port to the current container. - var container, publisher = resolution.Store.(Application).proxyHook() + var container, publisher = resolution.Store.(interface { + ProxyHook() (*pr.Container, ops.Publisher) + }).ProxyHook() resolution.Done() if tr, ok := trace.FromContext(ctx); ok { @@ -109,8 +109,6 @@ func (ps *proxyServer) Proxy(claims pb.Claims, stream pf.NetworkProxy_ProxyServe "clientAddr", open.Open.ClientAddr, "targetPort", open.Open.TargetPort, ) - proxyConnectionsAcceptedCounter.WithLabelValues(labels...).Inc() - var inbound, outbound uint64 defer func() { @@ -121,18 +119,12 @@ func (ps *proxyServer) Proxy(claims pb.Claims, stream pf.NetworkProxy_ProxyServe "byteOut", outbound, "error", _err, ) - if _err == nil { - proxyConnectionsClosedCounter.WithLabelValues(append(labels, "ok")...).Inc() - } else { - proxyConnectionsClosedCounter.WithLabelValues(append(labels, "error")...).Inc() - } }() // Forward loop that proxies from `client` => `delegate`. go func() { defer delegate.CloseWrite() - var counter = proxyConnBytesInboundCounter.WithLabelValues(labels...) for { if request, err := stream.Recv(); err != nil { err = pf.UnwrapGRPCError(err) @@ -151,7 +143,6 @@ func (ps *proxyServer) Proxy(claims pb.Claims, stream pf.NetworkProxy_ProxyServe return } else { atomic.AddUint64(&inbound, uint64(n)) - counter.Add(float64(n)) } } }() @@ -160,7 +151,6 @@ func (ps *proxyServer) Proxy(claims pb.Claims, stream pf.NetworkProxy_ProxyServe // When this loop completes, so does the Proxy RPC. var buffer = make([]byte, 1<<14) // 16KB. - var counter = proxyConnBytesOutboundCounter.WithLabelValues(labels...) for { if n, err := delegate.Read(buffer); err == io.EOF { return nil @@ -172,7 +162,6 @@ func (ps *proxyServer) Proxy(claims pb.Claims, stream pf.NetworkProxy_ProxyServe return nil } else { outbound += uint64(n) - counter.Add(float64(n)) } } } @@ -200,25 +189,5 @@ func validateOpen(req *pf.TaskNetworkProxyRequest) error { return nil } -// Prometheus metrics for connector TCP networking. -// These metrics match those collected by data-plane-gateway. -var proxyConnectionsAcceptedCounter = promauto.NewCounterVec(prometheus.CounterOpts{ - Name: "net_proxy_conns_accept_total", - Help: "counter of proxy connections that have been accepted", -}, []string{"shard", "port"}) -var proxyConnectionsClosedCounter = promauto.NewCounterVec(prometheus.CounterOpts{ - Name: "net_proxy_conns_closed_total", - Help: "counter of proxy connections that have completed and closed", -}, []string{"shard", "port", "status"}) - -var proxyConnBytesInboundCounter = promauto.NewCounterVec(prometheus.CounterOpts{ - Name: "net_proxy_conn_inbound_bytes_total", - Help: "total bytes proxied from client to container", -}, []string{"shard", "port"}) -var proxyConnBytesOutboundCounter = promauto.NewCounterVec(prometheus.CounterOpts{ - Name: "net_proxy_conn_outbound_bytes_total", - Help: "total bytes proxied from container to client", -}, []string{"shard", "port"}) - // See crates/runtime/src/container.rs const connectorInitPort = 49092 diff --git a/go/network/sni.go b/go/network/sni.go new file mode 100644 index 0000000000..1d34fdf64b --- /dev/null +++ b/go/network/sni.go @@ -0,0 +1,115 @@ +package network + +import ( + "context" + "errors" + "fmt" + "strconv" + "strings" + + "github.com/estuary/flow/go/labels" + pb "go.gazette.dev/core/broker/protocol" + pc "go.gazette.dev/core/consumer/protocol" +) + +// Parsed portions of the TLS ServerName which are used to map to a shard. +type parsedSNI struct { + hostname string + port string + keyBegin string + rClockBegin string +} + +// Resolved task shard metadata which allow us to complete TLS handshake. +type resolvedSNI struct { + portIsPublic bool + portProtocol string + shardIDPrefix string + taskName string +} + +// parseSNI parses a `target` into a parsedSNI. +// We accept two forms of targets: +// * d7f4a9d02b48c1a-6789 (hostname and port) +// * d7f4a9d02b48c1a-00000000-80000000-6789 (hostname, key begin, r-clock begin, and port). +func parseSNI(target string) (parsedSNI, error) { + var parts = strings.Split(target, "-") + var hostname, port, keyBegin, rClockBegin string + + if len(parts) == 2 { + hostname = parts[0] + port = parts[1] + } else if len(parts) == 4 { + hostname = parts[0] + keyBegin = parts[1] + rClockBegin = parts[2] + port = parts[3] + } else { + return parsedSNI{}, fmt.Errorf("expected two or for subdomain components, not %d", len(parts)) + } + + var _, err = strconv.ParseUint(port, 10, 16) + if err != nil { + return parsedSNI{}, fmt.Errorf("failed to parse subdomain port number: %w", err) + } + + return parsedSNI{ + hostname: hostname, + port: port, + keyBegin: keyBegin, + rClockBegin: rClockBegin, + }, nil +} + +func newResolvedSNI(parsed parsedSNI, shard *pc.ShardSpec) resolvedSNI { + var shardIDPrefix = shard.Id.String() + if ind := strings.LastIndexByte(shardIDPrefix, '/'); ind != -1 { + shardIDPrefix = shardIDPrefix[:ind+1] // Including trailing '/'. + } + + var portProtocol = shard.LabelSet.ValueOf(labels.PortProtoPrefix + parsed.port) + var portIsPublic = shard.LabelSet.ValueOf(labels.PortPublicPrefix+parsed.port) == "true" + + // Private ports MUST use the HTTP/1.1 reverse proxy. + if !portIsPublic { + portProtocol = "" + } + + return resolvedSNI{ + shardIDPrefix: shardIDPrefix, + portProtocol: portProtocol, + portIsPublic: portIsPublic, + taskName: shard.LabelSet.ValueOf(labels.TaskName), + } +} + +func listShards(ctx context.Context, shards pc.ShardClient, parsed parsedSNI, shardIDPrefix string) ([]pc.ListResponse_Shard, error) { + var include = []pb.Label{ + {Name: labels.ExposePort, Value: parsed.port}, + {Name: labels.Hostname, Value: parsed.hostname}, + } + if parsed.keyBegin != "" { + include = append(include, pb.Label{Name: labels.KeyBegin, Value: parsed.keyBegin}) + } + if parsed.rClockBegin != "" { + include = append(include, pb.Label{Name: labels.RClockBegin, Value: parsed.rClockBegin}) + } + if shardIDPrefix != "" { + include = append(include, pb.Label{Name: "id", Value: shardIDPrefix, Prefix: true}) + } + + var resp, err = shards.List( + pb.WithDispatchDefault(ctx), + &pc.ListRequest{ + Selector: pb.LabelSelector{Include: pb.LabelSet{Labels: include}}, + }, + ) + if err == nil && resp.Status != pc.Status_OK { + err = errors.New(resp.Status.String()) + } + if err != nil { + return nil, err + } + + return resp.Shards, nil +} diff --git a/go/network/tap.go b/go/network/tap.go new file mode 100644 index 0000000000..2bbf66f10c --- /dev/null +++ b/go/network/tap.go @@ -0,0 +1,50 @@ +package network + +import ( + "crypto/tls" + "net" +) + +// Tap is an adapter which retains a tapped net.Listener and itself acts as a +// net.Listener, with an Accept() that communicates over a forwarding channel. +// It's used for late binding of a Proxy to a pre-created Listener, +// and to hand off connections which are not intended for Proxy. +type Tap struct { + raw net.Listener + config *tls.Config + fwdCh chan net.Conn + fwdErr error +} + +func NewTap() *Tap { + return &Tap{ + raw: nil, // Set by Tap(). + config: nil, // Set by Tap(). + fwdCh: make(chan net.Conn, 4), + fwdErr: nil, + } +} + +func (tap *Tap) Wrap(tapped net.Listener, config *tls.Config) (net.Listener, error) { + tap.raw = tapped + tap.config = config + return tap, nil +} + +func (tap *Tap) Accept() (net.Conn, error) { + if conn, ok := <-tap.fwdCh; ok { + return conn, nil + } else { + return nil, tap.fwdErr + } +} + +func (tap *Tap) Close() error { + return tap.raw.Close() +} + +func (tap *Tap) Addr() net.Addr { + return tap.raw.Addr() +} + +var _ net.Listener = &Tap{} diff --git a/go/runtime/flow_consumer.go b/go/runtime/flow_consumer.go index fa8042e604..e1c8211361 100644 --- a/go/runtime/flow_consumer.go +++ b/go/runtime/flow_consumer.go @@ -10,6 +10,7 @@ import ( "github.com/estuary/flow/go/bindings" "github.com/estuary/flow/go/flow" "github.com/estuary/flow/go/labels" + "github.com/estuary/flow/go/network" "github.com/estuary/flow/go/protocols/capture" "github.com/estuary/flow/go/protocols/derive" pf "github.com/estuary/flow/go/protocols/flow" @@ -36,6 +37,7 @@ type FlowConsumerConfig struct { AllowLocal bool `long:"allow-local" env:"ALLOW_LOCAL" description:"Allow local connectors. True for local stacks, and false otherwise."` BuildsRoot string `long:"builds-root" required:"true" env:"BUILDS_ROOT" description:"Base URL for fetching Flow catalog builds"` ControlAPI pb.Endpoint `long:"control-api" env:"CONTROL_API" description:"Address of the control-plane API"` + Dashboard pb.Endpoint `long:"dashboard" env:"DASHBOARD" description:"Address of the Estuary dashboard"` DataPlaneFQDN string `long:"data-plane-fqdn" env:"DATA_PLANE_FQDN" description:"Fully-qualified domain name of the data-plane to which this reactor belongs"` Network string `long:"network" description:"The Docker network that connector containers are given access to. Defaults to the bridge network"` ProxyRuntimes int `long:"proxy-runtimes" default:"2" description:"The number of proxy connector runtimes that may run concurrently"` @@ -45,7 +47,14 @@ type FlowConsumerConfig struct { // Execute delegates to runconsumer.Cmd.Execute. func (c *FlowConsumerConfig) Execute(args []string) error { - return runconsumer.Cmd{Cfg: c, App: new(FlowConsumer)}.Execute(args) + var app = &FlowConsumer{ + Tap: network.NewTap(), + } + return runconsumer.Cmd{ + Cfg: c, + App: app, + WrapListener: app.Tap.Wrap, + }.Execute(args) } // FlowConsumer implements the Estuary Flow Consumer. @@ -65,6 +74,8 @@ type FlowConsumer struct { // It's important that we use a Context that's scoped to the life of the process, // rather than the lives of individual shards, so we don't lose logs. OpsContext context.Context + // Network listener tap. + Tap *network.Tap } // Application is the interface implemented by Flow shard task stores. @@ -81,9 +92,9 @@ type Application interface { ReplayRange(_ consumer.Shard, _ pb.Journal, begin, end pb.Offset) message.Iterator ReadThrough(pb.Offsets) (pb.Offsets, error) - // proxyHook exposes a current Container and ops.Publisher - // for use by the network proxy server. - proxyHook() (*pr.Container, ops.Publisher) + // ProxyHook exposes a current Container and ops.Publisher + // for use by network.ProxyServer. + ProxyHook() (*pr.Container, ops.Publisher) } var _ consumer.Application = (*FlowConsumer)(nil) @@ -197,7 +208,9 @@ func (f *FlowConsumer) InitApplication(args runconsumer.InitArgs) error { return fmt.Errorf("catalog builds service: %w", err) } - if keyedAuth, ok := args.Service.Authorizer.(*auth.KeyedAuth); ok && !config.Flow.TestAPIs { + var localAuthorizer = args.Service.Authorizer + + if keyedAuth, ok := localAuthorizer.(*auth.KeyedAuth); ok && !config.Flow.TestAPIs { // Wrap the underlying KeyedAuth Authorizer to use the control-plane's Authorize API. args.Service.Authorizer = NewControlPlaneAuthorizer( keyedAuth, @@ -242,7 +255,7 @@ func (f *FlowConsumer) InitApplication(args runconsumer.InitArgs) error { pr.NewVerifiedShufflerServer(shuffle.NewAPI(args.Service.Resolver), f.Service.Verifier)) pf.RegisterNetworkProxyServer(args.Server.GRPCServer, - pf.NewVerifiedNetworkProxyServer(&proxyServer{resolver: args.Service.Resolver}, f.Service.Verifier)) + pf.NewVerifiedNetworkProxyServer(&network.ProxyServer{Resolver: args.Service.Resolver}, f.Service.Verifier)) var connectorProxy = &connectorProxy{ address: args.Server.Endpoint(), @@ -255,5 +268,21 @@ func (f *FlowConsumer) InitApplication(args runconsumer.InitArgs) error { derive.RegisterConnectorServer(args.Server.GRPCServer, connectorProxy) materialize.RegisterConnectorServer(args.Server.GRPCServer, connectorProxy) + networkProxy, err := network.NewFrontend( + f.Tap, + config.Consumer.Host, + config.Flow.ControlAPI.URL(), + config.Flow.Dashboard.URL(), + pf.NewAuthNetworkProxyClient(pf.NewNetworkProxyClient(args.Server.GRPCLoopback), localAuthorizer), + pc.NewAuthShardClient(pc.NewShardClient(args.Server.GRPCLoopback), localAuthorizer), + args.Service.Verifier, + ) + if err != nil { + return fmt.Errorf("failed to build network proxy: %w", err) + } + args.Tasks.Queue("network-proxy-frontend", func() error { + return networkProxy.Serve(args.Tasks.Context()) + }) + return nil } diff --git a/go/runtime/task.go b/go/runtime/task.go index 48e0c02eeb..edf433ae64 100644 --- a/go/runtime/task.go +++ b/go/runtime/task.go @@ -148,7 +148,7 @@ func (t *taskBase[TaskSpec]) initTerm(shard consumer.Shard) error { return nil } -func (t *taskBase[TaskSpec]) proxyHook() (*pr.Container, ops.Publisher) { +func (t *taskBase[TaskSpec]) ProxyHook() (*pr.Container, ops.Publisher) { return t.container.Load(), t.opsPublisher } From b1bbdde7c391ec31788c6f02acb0e1337face956 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Sun, 15 Sep 2024 20:54:01 -0500 Subject: [PATCH 8/8] go.mod/Tiltfile: update for grpc-web and connector networking Bring in Gazette updates for grpc-web gateways and net.Listener customization. Deeply rework Tiltfile to remove data-plane-gateway, by creating a self-signed TLS CA and Certificate that are used by the broker and reactor (and may be used by other services if desired). Use a naming strategy of `thing.flow.localhost`, because many libraries like rustls don't accept wildcard certs of a toplevel `*.localhost` but will happily accept `*.flow.localhost`. Update for changes to --broker.allow-origin, --consumer.allow-origin, and --flow.dashboard, as well as extracting explicit arguments into environment variables. --- Tiltfile | 214 ++++++++++++++++----------- go.mod | 7 +- go.sum | 20 ++- go/flowctl-go/cmd-temp-data-plane.go | 3 + 4 files changed, 154 insertions(+), 90 deletions(-) diff --git a/Tiltfile b/Tiltfile index 6025b231ef..0a408c9095 100644 --- a/Tiltfile +++ b/Tiltfile @@ -1,38 +1,79 @@ # This file is interpreted by `tilt`, and describes how to get a local flow environment running. -DATABASE_URL="postgresql://postgres:postgres@localhost:5432/postgres" -os.putenv("DATABASE_URL", DATABASE_URL) -os.putenv("RUST_LOG", "info") -os.putenv("DOCKER_DEFAULT_PLATFORM", "linux/amd64") +DATABASE_URL="postgresql://postgres:postgres@db.flow.localhost:5432/postgres" # Secret used to sign Authorizations within a local data plane, as base64("supersecret"). # Also allow requests without an Authorization (to not break data-plane-gateway just yet). AUTH_KEYS="c3VwZXJzZWNyZXQ=,AA==" -os.putenv("CONSUMER_AUTH_KEYS", AUTH_KEYS) -os.putenv("BROKER_AUTH_KEYS", AUTH_KEYS) - REPO_BASE= '%s/..' % os.getcwd() TEST_KMS_KEY="projects/helpful-kingdom-273219/locations/us-central1/keyRings/dev/cryptoKeys/testing" -HOME_DIR=os.getenv("HOME") -FLOW_DIR=os.getenv("FLOW_DIR", os.path.join(HOME_DIR, "flow-local")) +FLOW_DIR=os.getenv("FLOW_DIR", os.path.join(os.getenv("HOME"), "flow-local")) ETCD_DATA_DIR=os.path.join(FLOW_DIR, "etcd") FLOW_BUILDS_ROOT="file://"+os.path.join(FLOW_DIR, "builds")+"/" -# Or alternatively, use an actual bucket when testing with external data-planes: -# FLOW_BUILDS_ROOT="gs://example/builds/" # A token for the local-stack system user signed against the local-stack # supabase secret (super-secret-jwt-token-with-at-least-32-characters-long). SYSTEM_USER_TOKEN="eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJodHRwOi8vMTI3LjAuMC4xOjU0MzEvYXV0aC92MSIsInN1YiI6ImZmZmZmZmZmLWZmZmYtZmZmZi1mZmZmLWZmZmZmZmZmZmZmZiIsImF1ZCI6ImF1dGhlbnRpY2F0ZWQiLCJleHAiOjI3MDAwMDAwMDAsImlhdCI6MTcwMDAwMDAwMCwiZW1haWwiOiJzdXBwb3J0QGVzdHVhcnkuZGV2Iiwicm9sZSI6ImF1dGhlbnRpY2F0ZWQiLCJpc19hbm9ueW1vdXMiOmZhbHNlfQ.Nb-N4s_YnObBHGivSTe_8FEniVUUpehzrRkF5JgNWWU" -# Start supabase, which is needed in order to compile the agent -local_resource('supabase', cmd='supabase start', links='http://localhost:5433') +# Paths for CA and server certificates +CA_KEY_PATH = "%s/ca.key" % FLOW_DIR +CA_CERT_PATH = "%s/ca.crt" % FLOW_DIR +TLS_KEY_PATH = "%s/server.key" % FLOW_DIR +TLS_CERT_PATH = "%s/server.crt" % FLOW_DIR + +local_resource( + 'supabase', + cmd='supabase start', + links='http://db.flow.localhost:5433', +) -# Builds many of the binaries that we'll need -local_resource('make', cmd='make', resource_deps=['supabase']) +local_resource( + 'make', + cmd='make', + resource_deps=['supabase'], +) -local_resource('etcd', serve_cmd='%s/flow/.build/package/bin/etcd \ +local_resource( + 'self-signed-tls-cert', + dir=REPO_BASE, + cmd='[ -f "%s" ] && [ -f "%s" ] || (\ + openssl req -x509 -nodes -days 3650 \ + -subj "/C=US/ST=QC/O=Estuary/CN=Estuary Root CA" \ + -addext basicConstraints=critical,CA:TRUE,pathlen:0 \ + -newkey ec -pkeyopt ec_paramgen_curve:P-256 \ + -keyout "%s" \ + -out "%s" \ + ) && (\ + openssl req -nodes -newkey ec -pkeyopt ec_paramgen_curve:P-256 \ + -subj "/C=US/ST=QC/O=Estuary/CN=flow.localhost" \ + -addext "subjectAltName=DNS:flow.localhost,DNS:*.flow.localhost,IP:127.0.0.1" \ + -keyout "%s" -out server.csr \ + ) && (\ + echo "subjectAltName=DNS:flow.localhost,DNS:*.flow.localhost,IP:127.0.0.1" > extfile.txt && \ + echo "basicConstraints=CA:FALSE" >> extfile.txt && \ + openssl x509 -req -days 365 \ + -in server.csr -CA "%s" -CAkey "%s" -CAcreateserial \ + -out "%s" \ + -extfile extfile.txt \ + ) && (\ + rm server.csr extfile.txt \ + )' % ( + TLS_CERT_PATH, # Check if server certificate already exists + TLS_KEY_PATH, # Check if server key already exists + CA_KEY_PATH, # CA key output path (ECDSA) + CA_CERT_PATH, # CA certificate output path + TLS_KEY_PATH, # Server key output path (ECDSA) + CA_CERT_PATH, # CA certificate input path + CA_KEY_PATH, # CA key input path + TLS_CERT_PATH # Server certificate output path + ) +) + +local_resource( + 'etcd', + serve_cmd='%s/flow/.build/package/bin/etcd \ --data-dir %s \ --log-level info \ --logger zap' % (REPO_BASE, ETCD_DATA_DIR), @@ -43,56 +84,89 @@ local_resource('etcd', serve_cmd='%s/flow/.build/package/bin/etcd \ ) ) -local_resource('gazette', serve_cmd='%s/flow/.build/package/bin/gazette serve \ - --broker.port=8080 \ - --broker.host=localhost \ +local_resource( + 'gazette', + serve_cmd='%s/flow/.build/package/bin/gazette serve \ + --broker.allow-origin http://localhost:3000 \ --broker.disable-stores \ - --broker.max-replication=1 \ - --log.level=info' % REPO_BASE, - links='http://localhost:8080/debug/pprof', + --broker.host gazette.flow.localhost \ + --broker.max-replication 1 \ + --broker.port 8080 \ + --etcd.address http://etcd.flow.localhost:2379 \ + --log.level=info \ + ' % REPO_BASE, + serve_env={ + "BROKER_AUTH_KEYS": AUTH_KEYS, + "BROKER_PEER_CA_FILE": CA_CERT_PATH, + "BROKER_SERVER_CERT_FILE": TLS_CERT_PATH, + "BROKER_SERVER_CERT_KEY_FILE": TLS_KEY_PATH, + }, + links='https://gazette.flow.localhost:8080/debug/pprof', resource_deps=['etcd'], readiness_probe=probe( initial_delay_secs=5, - http_get=http_get_action(port=8080, path='/debug/ready') + http_get=http_get_action(port=8080, path='/debug/ready', scheme='https') ) ) -local_resource('reactor', serve_cmd='%s/flow/.build/package/bin/flowctl-go serve consumer \ - --flow.allow-local \ - --broker.address http://localhost:8080 \ +local_resource( + 'reactor', + serve_cmd='%s/flow/.build/package/bin/flowctl-go serve consumer \ + --broker.address https://gazette.flow.localhost:8080 \ --broker.cache.size 128 \ - --consumer.host localhost \ + --consumer.allow-origin http://localhost:3000 \ + --consumer.host reactor.flow.localhost \ --consumer.limit 1024 \ --consumer.max-hot-standbys 0 \ --consumer.port 9000 \ - --etcd.address http://localhost:2379 \ - --flow.builds-root %s \ - --flow.network supabase_network_flow \ - --flow.control-api http://localhost:8675 \ + --etcd.address http://etcd.flow.localhost:2379 \ + --flow.allow-local \ + --flow.control-api http://agent.flow.localhost:8675 \ + --flow.dashboard http://localhost:3000 \ --flow.data-plane-fqdn local-cluster.dp.estuary-data.com \ - --log.format text \ - --log.level info' % (REPO_BASE, FLOW_BUILDS_ROOT), - links='http://localhost:9000/debug/pprof', + --flow.network supabase_network_flow \ + --log.level info \ + ' % (REPO_BASE), + serve_env={ + "BROKER_AUTH_KEYS": AUTH_KEYS, + "BROKER_TRUSTED_CA_FILE": CA_CERT_PATH, + "CONSUMER_AUTH_KEYS": AUTH_KEYS, + "CONSUMER_PEER_CA_FILE": CA_CERT_PATH, + "CONSUMER_SERVER_CERT_FILE": TLS_CERT_PATH, + "CONSUMER_SERVER_CERT_KEY_FILE": TLS_KEY_PATH, + "DOCKER_DEFAULT_PLATFORM": "linux/amd64", + "FLOW_BUILDS_ROOT": FLOW_BUILDS_ROOT, + }, + links='https://reactor.flow.localhost:9000/debug/pprof', resource_deps=['etcd'], readiness_probe=probe( initial_delay_secs=5, - http_get=http_get_action(port=9000, path='/debug/ready') - ) + http_get=http_get_action(port=9000, path='/debug/ready', scheme='https') + ), ) -local_resource('agent', serve_cmd='%s/flow/.build/package/bin/agent \ +local_resource( + 'agent', + serve_cmd='%s/flow/.build/package/bin/agent \ --connector-network supabase_network_flow \ --allow-local \ --allow-origin http://localhost:3000 \ --api-port 8675 \ - --builds-root %s \ --serve-handlers \ - --bin-dir %s/flow/.build/package/bin' % (REPO_BASE, FLOW_BUILDS_ROOT, REPO_BASE), + ' % (REPO_BASE), + serve_env={ + "BIN_DIR": '%s/flow/.build/package/bin' % REPO_BASE, + "BUILDS_ROOT": FLOW_BUILDS_ROOT, + "DATABASE_URL": DATABASE_URL, + "RUST_LOG": "info", + "SSL_CERT_FILE": CA_CERT_PATH, + }, resource_deps=['reactor', 'gazette'] ) -local_resource('create-data-plane-local-cluster', - cmd='sleep 5 && curl -v \ +local_resource( + 'create-data-plane-local-cluster', + cmd='sleep 2 && curl -v \ -X POST \ -H "content-type: application/json" \ -H "authorization: bearer %s" \ @@ -100,16 +174,17 @@ local_resource('create-data-plane-local-cluster', "name":"local-cluster",\ "category": {\ "manual": {\ - "brokerAddress": "http://localhost:8080",\ - "reactorAddress": "http://localhost:9000",\ + "brokerAddress": "https://gazette.flow.localhost:8080",\ + "reactorAddress": "https://reactor.flow.localhost:9000",\ "hmacKeys": ["c3VwZXJzZWNyZXQ="]\ }\ }\ - }\' http://localhost:8675/admin/create-data-plane' % SYSTEM_USER_TOKEN, + }\' http://agent.flow.localhost:8675/admin/create-data-plane' % SYSTEM_USER_TOKEN, resource_deps=['agent'] ) -local_resource('update-l2-reporting', +local_resource( + 'update-l2-reporting', cmd='curl -v \ -X POST \ -H "content-type: application/json" \ @@ -117,16 +192,18 @@ local_resource('update-l2-reporting', --data-binary \'{ \ "defaultDataPlane":"ops/dp/public/local-cluster",\ "dryRun":false\ - }\' http://localhost:8675/admin/update-l2-reporting' % SYSTEM_USER_TOKEN, + }\' http://agent.flow.localhost:8675/admin/update-l2-reporting' % SYSTEM_USER_TOKEN, resource_deps=['create-data-plane-local-cluster'] ) -local_resource('local-ops-view', +local_resource( + 'local-ops-view', cmd='./local/ops-publication.sh ops-catalog/local-view.bundle.json | psql "%s"' % DATABASE_URL, resource_deps=['update-l2-reporting'] ) -local_resource('config-encryption', +local_resource( + 'config-encryption', serve_cmd='%s/config-encryption/target/debug/flow-config-encryption --gcp-kms %s' % (REPO_BASE, TEST_KMS_KEY) ) @@ -137,45 +214,8 @@ local_resource( ) local_resource( - 'ui', + 'dashboard', serve_dir='%s/ui' % REPO_BASE, serve_cmd='BROWSER=none npm start', links='http://localhost:3000' -) - -DPG_REPO='%s/data-plane-gateway' % REPO_BASE -DPG_TLS_CERT_PATH='%s/local-tls-cert.pem' % DPG_REPO -DPG_TLS_KEY_PATH='%s/local-tls-private-key.pem' % DPG_REPO - -local_resource('dpg-tls-cert', - dir='%s/data-plane-gateway' % REPO_BASE, - # These incantations create a non-CA self-signed certificate which is - # valid for localhost and its subdomains. rustls is quite fiddly about - # accepting self-signed certificates so all of these are required. - cmd='[ -f %s ] || openssl req -x509 -nodes -days 365 \ - -subj "/ST=QC/O=Estuary/CN=localhost" \ - -addext basicConstraints=critical,CA:FALSE,pathlen:1 \ - -addext "subjectAltName=DNS:localhost,DNS:*.localhost,IP:127.0.0.1" \ - -newkey rsa:2048 -keyout "%s" \ - -out "%s"' % (DPG_TLS_KEY_PATH, DPG_TLS_KEY_PATH, DPG_TLS_CERT_PATH) -) - -local_resource('data-plane-gateway', - dir=DPG_REPO, - serve_dir=DPG_REPO, - cmd='go build .', - serve_cmd='./data-plane-gateway \ - --tls-private-key=%s \ - --tls-certificate=%s \ - --broker-address=localhost:8080 \ - --consumer-address=localhost:9000 \ - --log.level=debug \ - --inference-address=localhost:9090 \ - --control-plane-auth-url=http://localhost:3000' % ( - DPG_TLS_KEY_PATH, - DPG_TLS_CERT_PATH - ), - links='https://localhost:28318/', - resource_deps=['gazette', 'reactor', 'dpg-tls-cert'] -) - +) \ No newline at end of file diff --git a/go.mod b/go.mod index ecbaa8a123..41280f0d0e 100644 --- a/go.mod +++ b/go.mod @@ -12,6 +12,7 @@ require ( github.com/gogo/protobuf v1.3.2 github.com/golang-jwt/jwt/v5 v5.2.1 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 + github.com/hashicorp/golang-lru/v2 v2.0.7 github.com/jessevdk/go-flags v1.5.0 github.com/jgraettinger/gorocksdb v0.0.0-20240221161858-8f4873ee26e0 github.com/mattn/go-sqlite3 v2.0.3+incompatible @@ -22,7 +23,7 @@ require ( github.com/stretchr/testify v1.9.0 go.etcd.io/etcd/api/v3 v3.5.0 go.etcd.io/etcd/client/v3 v3.5.0 - go.gazette.dev/core v0.99.1-0.20240908182145-d0c9d56a6796 + go.gazette.dev/core v0.99.1-0.20240920050446-fc9749109d92 golang.org/x/net v0.26.0 google.golang.org/api v0.126.0 google.golang.org/grpc v1.65.0 @@ -49,6 +50,9 @@ require ( github.com/davecgh/go-spew v1.1.1 // indirect github.com/dustin/go-humanize v1.0.0 // indirect github.com/dustinkirkland/golang-petname v0.0.0-20191129215211-8e5a1ed0cff0 // indirect + github.com/ghodss/yaml v1.0.0 // indirect + github.com/gogo/gateway v1.1.0 // indirect + github.com/golang/glog v1.2.1 // indirect github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/golang/protobuf v1.5.4 // indirect github.com/golang/snappy v0.0.4 // indirect @@ -57,6 +61,7 @@ require ( github.com/googleapis/enterprise-certificate-proxy v0.2.3 // indirect github.com/googleapis/gax-go/v2 v2.11.0 // indirect github.com/gorilla/schema v1.4.1 // indirect + github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect github.com/hashicorp/golang-lru v0.5.4 // indirect github.com/jgraettinger/cockroach-encoding v1.1.0 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect diff --git a/go.sum b/go.sum index f2f4285c71..7d8a7d1f0f 100644 --- a/go.sum +++ b/go.sum @@ -130,6 +130,7 @@ github.com/fatih/color v1.7.0 h1:DkWD4oS2D8LGGgTQ6IvwJJXSL5Vp2ffcQg58nFV38Ys= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/form3tech-oss/jwt-go v3.2.2+incompatible h1:TcekIExNqud5crz4xD2pavyTgWiPvpYe4Xau31I0PRk= github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= +github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= @@ -142,12 +143,17 @@ github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= +github.com/gogo/gateway v1.1.0 h1:u0SuhL9+Il+UbjM9VIE3ntfRujKbvVpFvNB4HbjeVQ0= +github.com/gogo/gateway v1.1.0/go.mod h1:S7rR8FRQyG3QFESeSv4l2WnsyzlCLG0CzBbUUo/mbic= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17wHk= github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/glog v1.2.1 h1:OptwRhECazUx5ix5TTWC3EZhsZEHWcYWY4FQHTIubm4= +github.com/golang/glog v1.2.1/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= @@ -224,11 +230,15 @@ github.com/gorilla/schema v1.4.1 h1:jUg5hUjCSDZpNGLuXQOgIWGdlgrIdYvgQ0wZtdK1M3E= github.com/gorilla/schema v1.4.1/go.mod h1:Dg5SSm5PV60mhF2NFaTV1xuYYj8tV8NOPRo4FggUMnM= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= +github.com/grpc-ecosystem/grpc-gateway v1.8.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= +github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= +github.com/hashicorp/golang-lru/v2 v2.0.7 h1:a+bsQ5rvGLjzHuww6tVxozPZFVghXaHOwFs4luLUK2k= +github.com/hashicorp/golang-lru/v2 v2.0.7/go.mod h1:QeFd9opnmA6QUJc5vARoKUSoFhyfM2/ZepoAG6RGpeM= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.5.0 h1:1jKYvbxEjfUl0fmqTCOfonvskHHXMjBySTLW4y9LFvc= @@ -329,6 +339,7 @@ github.com/prometheus/procfs v0.7.3 h1:4jVXhlkAyzOScmCkXBTOLRLTz8EeU+eyjrwB/EPq0 github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= +github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= @@ -372,8 +383,8 @@ go.etcd.io/etcd/client/pkg/v3 v3.5.0 h1:2aQv6F436YnN7I4VbI8PPYrBhu+SmrTaADcf8Mi/ go.etcd.io/etcd/client/pkg/v3 v3.5.0/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= go.etcd.io/etcd/client/v3 v3.5.0 h1:62Eh0XOro+rDwkrypAGDfgmNh5Joq+z+W9HZdlXMzek= go.etcd.io/etcd/client/v3 v3.5.0/go.mod h1:AIKXXVX/DQXtfTEqBryiLTUXwON+GuvO6Z7lLS/oTh0= -go.gazette.dev/core v0.99.1-0.20240908182145-d0c9d56a6796 h1:GIKRhZsXYToRso4HXrFI6RipUs8We/gksc/hUgN830A= -go.gazette.dev/core v0.99.1-0.20240908182145-d0c9d56a6796/go.mod h1:ztgm7s+VPfspBo3NSC7sedoZlASlKIa4Etmf4sYI8PY= +go.gazette.dev/core v0.99.1-0.20240920050446-fc9749109d92 h1:QDNKYcKpBDuF5Dan1++wAn5+zELjf4nPpWSJybJ/cig= +go.gazette.dev/core v0.99.1-0.20240920050446-fc9749109d92/go.mod h1:QR31EBrUMzThz/oDYxJHYwmlFSOjgvuNE14lHl/ViX4= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= @@ -443,6 +454,7 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -505,6 +517,7 @@ golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190130150945-aca44879d564/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -686,6 +699,7 @@ google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157/go. google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157 h1:Zy9XzmMEflZ/MAaA7vNcoebnRAld7FsPW1EeBB7V0m8= google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.19.1/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= @@ -725,6 +739,8 @@ gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= +gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= diff --git a/go/flowctl-go/cmd-temp-data-plane.go b/go/flowctl-go/cmd-temp-data-plane.go index 219ff972b8..16aeccbebc 100644 --- a/go/flowctl-go/cmd-temp-data-plane.go +++ b/go/flowctl-go/cmd-temp-data-plane.go @@ -188,6 +188,7 @@ func (cmd cmdTempDataPlane) consumerCmd(ctx context.Context, tempdir, buildsRoot "consumer", "--broker.address", gazetteAddr, "--broker.cache.size", "128", + "--consumer.allow-origin", "http://localhost:3000", "--consumer.limit", "1024", "--consumer.max-hot-standbys", "0", "--consumer.port", port, @@ -195,6 +196,8 @@ func (cmd cmdTempDataPlane) consumerCmd(ctx context.Context, tempdir, buildsRoot "--consumer.watch-delay", "0ms", // Speed test execution. "--etcd.address", etcdAddr, "--flow.builds-root", buildsRoot, + "--flow.control-api", "http://agent.flow.localhost:8675", + "--flow.dashboard", "http://dashboard.flow.localhost:3000", "--flow.test-apis", "--log.format", cmd.Log.Format, "--log.level", cmd.Log.Level,