Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[storage] Sink progress WIP #31152

Draft
wants to merge 14 commits into
base: main
Choose a base branch
from
4 changes: 3 additions & 1 deletion src/adapter/src/catalog/transact.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1057,8 +1057,10 @@ impl Catalog {
CatalogItem::ContinualTask(ct) => {
storage_collections_to_create.insert(ct.global_id());
}
CatalogItem::Sink(sink) => {
storage_collections_to_create.insert(sink.global_id());
}
CatalogItem::Log(_)
| CatalogItem::Sink(_)
| CatalogItem::View(_)
| CatalogItem::Index(_)
| CatalogItem::Type(_)
Expand Down
40 changes: 38 additions & 2 deletions src/adapter/src/coord.rs
Original file line number Diff line number Diff line change
Expand Up @@ -146,12 +146,13 @@ use mz_sql::session::vars::SystemVars;
use mz_sql_parser::ast::display::AstDisplay;
use mz_sql_parser::ast::ExplainStage;
use mz_storage_client::client::TableData;
use mz_storage_client::controller::{CollectionDescription, DataSource};
use mz_storage_client::controller::{CollectionDescription, DataSource, ExportDescription};
use mz_storage_types::connections::inline::{IntoInlineConnection, ReferencedConnection};
use mz_storage_types::connections::Connection as StorageConnection;
use mz_storage_types::connections::ConnectionContext;
use mz_storage_types::read_holds::ReadHold;
use mz_storage_types::sinks::S3SinkFormat;
use mz_storage_types::sinks::{S3SinkFormat, StorageSinkDesc};
use mz_storage_types::sources::kafka::KAFKA_PROGRESS_DESC;
use mz_storage_types::sources::Timeline;
use mz_timestamp_oracle::postgres_oracle::{
PostgresTimestampOracle, PostgresTimestampOracleConfig,
Expand Down Expand Up @@ -2101,6 +2102,11 @@ impl Coordinator {
self.create_storage_export(sink.global_id(), sink)
.await
.unwrap_or_terminate("cannot fail to create exports");
policies_to_set
.entry(CompactionWindow::Default)
.or_insert_with(Default::default)
.storage_ids
.insert(sink.global_id());
}
CatalogItem::Connection(catalog_connection) => {
if let ConnectionDetails::AwsPrivatelink(conn) = &catalog_connection.details {
Expand Down Expand Up @@ -2754,6 +2760,36 @@ impl Coordinator {
collections.push((ct.global_id(), collection_desc));
}
}
CatalogItem::Sink(sink) => {
let collection_desc = CollectionDescription {
// TODO(sinks): make generic once we have more than one sink type.
desc: KAFKA_PROGRESS_DESC.clone(),
data_source: DataSource::Sink {
desc: ExportDescription {
sink: StorageSinkDesc {
from: sink.from,
from_desc: KAFKA_PROGRESS_DESC.clone(),
connection: sink
.connection
.clone()
.into_inline_connection(self.catalog().state()),
partition_strategy: sink.partition_strategy.clone(),
envelope: sink.envelope,
as_of: Antichain::from_elem(Timestamp::minimum()),
with_snapshot: sink.with_snapshot,
version: sink.version,
from_storage_metadata: (),
to_storage_metadata: (),
},
instance_id: sink.cluster_id,
},
},
since: None,
status_collection_id: None,
timeline: None,
};
collections.push((sink.global_id, collection_desc));
}
_ => (),
}
}
Expand Down
64 changes: 39 additions & 25 deletions src/adapter/src/coord/ddl.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,16 @@ use std::collections::{BTreeMap, BTreeSet};
use std::sync::Arc;
use std::time::Duration;

use crate::active_compute_sink::{ActiveComputeSink, ActiveComputeSinkRetireReason};
use crate::catalog::{DropObjectInfo, Op, ReplicaCreateDropReason, TransactionResult};
use crate::coord::appends::BuiltinTableAppendNotify;
use crate::coord::timeline::{TimelineContext, TimelineState};
use crate::coord::{Coordinator, ReplicaMetadata};
use crate::session::{Session, Transaction, TransactionOps};
use crate::statement_logging::StatementEndedExecutionReason;
use crate::telemetry::{EventDetails, SegmentClientExt};
use crate::util::ResultExt;
use crate::{catalog, flags, AdapterError, TimestampProvider};
use fail::fail_point;
use futures::Future;
use maplit::{btreemap, btreeset};
Expand Down Expand Up @@ -46,25 +56,15 @@ use mz_sql::session::vars::{
MAX_REPLICAS_PER_CLUSTER, MAX_ROLES, MAX_SCHEMAS_PER_DATABASE, MAX_SECRETS, MAX_SINKS,
MAX_SOURCES, MAX_TABLES,
};
use mz_storage_client::controller::ExportDescription;
use mz_storage_client::controller::{CollectionDescription, DataSource, ExportDescription};
use mz_storage_types::connections::inline::IntoInlineConnection;
use mz_storage_types::connections::PostgresConnection;
use mz_storage_types::read_policy::ReadPolicy;
use mz_storage_types::sources::kafka::KAFKA_PROGRESS_DESC;
use mz_storage_types::sources::GenericSourceConnection;
use serde_json::json;
use tracing::{event, info_span, warn, Instrument, Level};

use crate::active_compute_sink::{ActiveComputeSink, ActiveComputeSinkRetireReason};
use crate::catalog::{DropObjectInfo, Op, ReplicaCreateDropReason, TransactionResult};
use crate::coord::appends::BuiltinTableAppendNotify;
use crate::coord::timeline::{TimelineContext, TimelineState};
use crate::coord::{Coordinator, ReplicaMetadata};
use crate::session::{Session, Transaction, TransactionOps};
use crate::statement_logging::StatementEndedExecutionReason;
use crate::telemetry::{EventDetails, SegmentClientExt};
use crate::util::ResultExt;
use crate::{catalog, flags, AdapterError, TimestampProvider};

impl Coordinator {
/// Same as [`Self::catalog_transact_conn`] but takes a [`Session`].
#[instrument(name = "coord::catalog_transact")]
Expand Down Expand Up @@ -1056,9 +1056,10 @@ impl Coordinator {
}

pub(crate) fn drop_storage_sinks(&mut self, sink_gids: Vec<GlobalId>) {
let storage_metadata = self.catalog.state().storage_metadata();
self.controller
.storage
.drop_sinks(sink_gids)
.drop_sinks(storage_metadata, sink_gids)
.unwrap_or_terminate("cannot fail to drop sinks");
}

Expand Down Expand Up @@ -1286,9 +1287,6 @@ impl Coordinator {
id: GlobalId,
sink: &Sink,
) -> Result<(), AdapterError> {
// Validate `sink.from` is in fact a storage collection
self.controller.storage.check_exists(sink.from)?;

// The AsOf is used to determine at what time to snapshot reading from
// the persist collection. This is primarily relevant when we do _not_
// want to include the snapshot in the sink.
Expand All @@ -1308,6 +1306,7 @@ impl Coordinator {
// TODO: Maybe in the future, pass those holds on to storage, to hold on
// to them and downgrade when possible?
let read_holds = self.acquire_read_holds(&id_bundle);

let as_of = self.least_valid_read(&read_holds);

let storage_sink_from_entry = self.catalog().get_entry_by_global_id(&sink.from);
Expand All @@ -1330,25 +1329,40 @@ impl Coordinator {
with_snapshot: sink.with_snapshot,
version: sink.version,
from_storage_metadata: (),
to_storage_metadata: (),
};

let res = self
.controller
.storage
.create_exports(vec![(
id,
ExportDescription {
let collection_desc = CollectionDescription {
// TODO(sinks): make generic once we have more than one sink type.
desc: KAFKA_PROGRESS_DESC.clone(),
data_source: DataSource::Sink {
desc: ExportDescription {
sink: storage_sink_desc,
instance_id: sink.cluster_id,
},
)])
.await;
},
since: None,
status_collection_id: None,
timeline: None,
};
let collections = vec![(id, collection_desc)];

// Create the collections.
let storage_metadata = self.catalog.state().storage_metadata();
self.controller
.storage
.create_collections(storage_metadata, None, collections)
.await
.unwrap_or_terminate("cannot fail to create collections");

// Validate `sink.from` is in fact a storage collection
self.controller.storage.check_exists(sink.from)?;

// Drop read holds after the export has been created, at which point
// storage will have put in its own read holds.
drop(read_holds);

Ok(res?)
Ok(())
}

/// Validate all resource limits in a catalog transaction and return an error if that limit is
Expand Down
31 changes: 20 additions & 11 deletions src/adapter/src/coord/sequencer/inner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,15 +21,22 @@ use futures::{future, Future};
use itertools::Itertools;
use maplit::btreeset;
use mz_adapter_types::compaction::CompactionWindow;
use mz_adapter_types::connection::ConnectionId;
use mz_catalog::memory::objects::{
CatalogItem, Cluster, Connection, DataSourceDesc, Sink, Source, Table, TableDataSource, Type,
};
use mz_cloud_resources::VpcEndpointConfig;
use mz_controller_types::ReplicaId;
use mz_expr::{
CollectionPlan, MapFilterProject, OptimizedMirRelationExpr, ResultSpec, RowSetFinishing,
};
use mz_ore::cast::CastFrom;
use mz_ore::collections::{CollectionExt, HashSet};
use mz_ore::task::{self, spawn, JoinHandle};
use mz_ore::tracing::OpenTelemetryContext;
use mz_ore::vec::VecExt;
use mz_ore::{assert_none, instrument};
use mz_persist_client::stats::SnapshotPartStats;
use mz_repr::adt::jsonb::Jsonb;
use mz_repr::adt::mz_acl_item::{MzAclItem, PrivilegeMap};
use mz_repr::explain::json::json_string;
Expand All @@ -39,6 +46,7 @@ use mz_repr::{
CatalogItemId, Datum, Diff, GlobalId, IntoRowIterator, RelationVersion,
RelationVersionSelector, Row, RowArena, RowIterator, Timestamp,
};
use mz_sql::ast::AlterSourceAddSubsourceOption;
use mz_sql::ast::{CreateSubsourceStatement, MySqlConfigOptionName, UnresolvedItemName};
use mz_sql::catalog::{
CatalogCluster, CatalogClusterReplica, CatalogDatabase, CatalogError,
Expand All @@ -52,17 +60,7 @@ use mz_sql::names::{
use mz_sql::plan::{ConnectionDetails, NetworkPolicyRule, StatementContext};
use mz_sql::pure::{generate_subsource_statements, PurifiedSourceExport};
use mz_storage_types::sinks::StorageSinkDesc;
use smallvec::SmallVec;
use timely::progress::Timestamp as TimelyTimestamp;
// Import `plan` module, but only import select elements to avoid merge conflicts on use statements.
use mz_adapter_types::connection::ConnectionId;
use mz_catalog::memory::objects::{
CatalogItem, Cluster, Connection, DataSourceDesc, Sink, Source, Table, TableDataSource, Type,
};
use mz_ore::cast::CastFrom;
use mz_ore::{assert_none, instrument};
use mz_persist_client::stats::SnapshotPartStats;
use mz_sql::ast::AlterSourceAddSubsourceOption;
use mz_sql::plan::{
AlterConnectionAction, AlterConnectionPlan, CreateSourcePlanBundle, ExplainSinkSchemaPlan,
Explainee, ExplaineeStatement, MutationKind, Params, Plan, PlannedAlterRoleOption,
Expand Down Expand Up @@ -90,7 +88,9 @@ use mz_storage_types::AlterCompatible;
use mz_transform::dataflow::DataflowMetainfo;
use mz_transform::notice::{OptimizerNoticeApi, OptimizerNoticeKind, RawOptimizerNotice};
use mz_transform::EmptyStatisticsOracle;
use smallvec::SmallVec;
use timely::progress::Antichain;
use timely::progress::Timestamp as TimelyTimestamp;
use tokio::sync::{oneshot, watch};
use tracing::{warn, Instrument, Span};

Expand Down Expand Up @@ -1349,6 +1349,9 @@ impl Coordinator {
.await
.unwrap_or_terminate("cannot fail to create exports");

self.initialize_storage_read_policies([item_id].into(), CompactionWindow::Default)
.await;

ctx.retire(Ok(ExecuteResponse::CreatedSink))
}

Expand Down Expand Up @@ -3527,7 +3530,12 @@ impl Coordinator {
.expect("sink known to exist")
.write_frontier;
let as_of = ctx.read_hold.least_valid_read();
assert!(write_frontier.iter().all(|t| as_of.less_than(t)));
assert!(
write_frontier.iter().all(|t| as_of.less_than(t)),
"{:?} should be strictly less than {:?}",
&*as_of,
&**write_frontier
);

let catalog_sink = Sink {
create_sql: sink.create_sql,
Expand Down Expand Up @@ -3576,6 +3584,7 @@ impl Coordinator {
version: sink.version,
partition_strategy: sink.partition_strategy,
from_storage_metadata: (),
to_storage_metadata: (),
};

self.controller
Expand Down
2 changes: 1 addition & 1 deletion src/catalog/src/memory/objects.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1614,9 +1614,9 @@ impl CatalogItem {
CatalogItem::Table(_)
| CatalogItem::Source(_)
| CatalogItem::MaterializedView(_)
| CatalogItem::Sink(_)
| CatalogItem::ContinualTask(_) => true,
CatalogItem::Log(_)
| CatalogItem::Sink(_)
| CatalogItem::View(_)
| CatalogItem::Index(_)
| CatalogItem::Type(_)
Expand Down
Loading