From e91678f69ca397fd647b9bb6153e727935f57ddf Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Tue, 28 Jan 2025 16:04:14 -0500 Subject: [PATCH 01/12] catalog: Key migration shard by binary version (#31210) The builtin migration shard is a persist shard used during version upgrades. It uses the environment's deploy generation as part of the key for the values. The assumption was that two environments with the same deploy generation would always have the same binary version. This assumption would allow all migration steps of two environments with the same deploy generation to be idempotent. This assumption was not correct. Two environments with different binary version can use the same deploy generation as long as one environment never fully completed a deployment. This is especially bad because the migration shard is written to and read from in read-only mode, before a deployment is complete. This commit updates the key of the builtin migration shard to explicitly use the binary version of environmentd so that the migration steps are idempotent. Fixes #MaterializeInc/database-issues/issues/8917 --- src/adapter/src/catalog/migrate.rs | 17 ++++++ .../catalog/open/builtin_item_migration.rs | 58 +++++++++++-------- src/adapter/src/coord.rs | 1 - src/catalog/src/config.rs | 1 - src/catalog/src/durable/transaction.rs | 7 +++ 5 files changed, 58 insertions(+), 26 deletions(-) diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index f3ab840ef4e23..93d2decf6b264 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -208,6 +208,23 @@ pub(crate) fn durable_migrate( Some(EXPR_CACHE_MIGRATION_DONE), )?; } + + // Migrate the builtin migration shard to a new shard. We're updating the keys to use the explicit + // binary version instead of the deploy generation. + const BUILTIN_MIGRATION_SHARD_MIGRATION_KEY: &str = "migration_shard_migration"; + const BUILTIN_MIGRATION_SHARD_MIGRATION_DONE: u64 = 1; + if tx.get_config(BUILTIN_MIGRATION_SHARD_MIGRATION_KEY.to_string()) + != Some(BUILTIN_MIGRATION_SHARD_MIGRATION_DONE) + { + if let Some(shard_id) = tx.get_builtin_migration_shard() { + tx.mark_shards_as_finalized(btreeset! {shard_id}); + tx.set_builtin_migration_shard(ShardId::new())?; + } + tx.set_config( + BUILTIN_MIGRATION_SHARD_MIGRATION_KEY.to_string(), + Some(BUILTIN_MIGRATION_SHARD_MIGRATION_DONE), + )?; + } Ok(()) } diff --git a/src/adapter/src/catalog/open/builtin_item_migration.rs b/src/adapter/src/catalog/open/builtin_item_migration.rs index a88882b8634a3..077ffb1da93aa 100644 --- a/src/adapter/src/catalog/open/builtin_item_migration.rs +++ b/src/adapter/src/catalog/open/builtin_item_migration.rs @@ -68,7 +68,6 @@ pub(crate) async fn migrate_builtin_items( } BuiltinItemMigrationConfig::ZeroDownTime { persist_client, - deploy_generation, read_only, } => { migrate_builtin_items_0dt( @@ -77,7 +76,6 @@ pub(crate) async fn migrate_builtin_items( local_expr_cache, persist_client, migrated_builtins, - deploy_generation, read_only, ) .await @@ -134,11 +132,11 @@ async fn migrate_builtin_items_legacy( /// /// 1. Each environment has a dedicated persist shard, called the migration shard, that allows /// environments to durably write down metadata while in read-only mode. The shard is a -/// mapping of `(GlobalId, deploy_generation)` to `ShardId`. -/// 2. Collect the `GlobalId` of all migrated tables for the current deploy generation. +/// mapping of `(GlobalId, build_version)` to `ShardId`. +/// 2. Collect the `GlobalId` of all migrated tables for the current build version. /// 3. Read in the current contents of the migration shard. /// 4. Collect all the `ShardId`s from the migration shard that are not at the current -/// `deploy_generation` or are not in the set of migrated tables. +/// `build_version` or are not in the set of migrated tables. /// a. If they ARE NOT mapped to a `GlobalId` in the storage metadata then they are shards /// from an incomplete migration. Finalize them and remove them from the migration shard. /// Note: care must be taken to not remove the shard from the migration shard until we are @@ -146,10 +144,10 @@ async fn migrate_builtin_items_legacy( /// b. If they ARE mapped to a `GlobalId` in the storage metadata then they are shards from a /// complete migration. Remove them from the migration shard. /// 5. Collect all the `GlobalId`s of tables that are migrated, but not in the migration shard -/// for the current deploy generation. Generate new `ShardId`s and add them to the migration +/// for the current build version. Generate new `ShardId`s and add them to the migration /// shard. /// 6. At this point the migration shard should only logically contain a mapping of migrated -/// table `GlobalId`s to new `ShardId`s for the current deploy generation. For each of these +/// table `GlobalId`s to new `ShardId`s for the current build version. For each of these /// `GlobalId`s such that the `ShardId` isn't already in the storage metadata: /// a. Remove the current `GlobalId` to `ShardId` mapping from the storage metadata. /// b. Finalize the removed `ShardId`s. @@ -177,7 +175,6 @@ async fn migrate_builtin_items_0dt( local_expr_cache: &mut LocalExpressionCache, persist_client: PersistClient, migrated_builtins: Vec, - deploy_generation: u64, read_only: bool, ) -> Result { assert_eq!( @@ -186,6 +183,8 @@ async fn migrate_builtin_items_0dt( "txn must be in savepoint mode when read_only is true, and in writable mode when read_only is false" ); + let build_version = state.config.build_info.semver_version(); + // 0. Update durably stored fingerprints. let id_fingerprint_map: BTreeMap<_, _> = BUILTINS::iter(&state.config().builtins_cfg) .map(|builtin| { @@ -237,7 +236,9 @@ async fn migrate_builtin_items_0dt( .expect("builtin migration shard should exist for opened catalogs"); let diagnostics = Diagnostics { shard_name: "builtin_migration".to_string(), - handle_purpose: format!("builtin table migration shard for org {organization_id:?} generation {deploy_generation:?}"), + handle_purpose: format!( + "builtin table migration shard for org {organization_id:?} version {build_version:?}" + ), }; let mut since_handle: SinceHandle = persist_client .open_critical_since( @@ -348,16 +349,16 @@ async fn migrate_builtin_items_0dt( txn.get_collection_metadata() }; for (table_key, shard_id) in global_id_shards.clone() { - if table_key.deploy_generation > deploy_generation { + if table_key.build_version > build_version { halt!( - "saw deploy generation {}, which is greater than current deploy generation {}", - table_key.deploy_generation, - deploy_generation + "saw build version {}, which is greater than current build version {}", + table_key.build_version, + build_version ); } if !migrated_storage_collections.contains(&table_key.global_id) - || table_key.deploy_generation < deploy_generation + || table_key.build_version < build_version { global_id_shards.remove(&table_key); if storage_collection_metadata.get(&GlobalId::System(table_key.global_id)) @@ -370,7 +371,7 @@ async fn migrate_builtin_items_0dt( } } - // 5. Add migrated tables to migration shard for current generation. + // 5. Add migrated tables to migration shard for current build version. let mut global_id_shards: BTreeMap<_, _> = global_id_shards .into_iter() .map(|(table_key, shard_id)| (table_key.global_id, shard_id)) @@ -381,7 +382,7 @@ async fn migrate_builtin_items_0dt( global_id_shards.insert(global_id, shard_id); let table_key = TableKey { global_id, - deploy_generation, + build_version: build_version.clone(), }; migrated_shard_updates.push(((table_key, shard_id), upper, 1)); } @@ -541,15 +542,15 @@ mod persist_schema { use mz_persist_types::stats::NoneStats; use mz_persist_types::Codec; - #[derive(Debug, Clone, Default, Eq, Ord, PartialEq, PartialOrd)] + #[derive(Debug, Clone, Eq, Ord, PartialEq, PartialOrd)] pub(super) struct TableKey { pub(super) global_id: u64, - pub(super) deploy_generation: u64, + pub(super) build_version: semver::Version, } impl std::fmt::Display for TableKey { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{}-{}", self.global_id, self.deploy_generation) + write!(f, "{}-{}", self.global_id, self.build_version) } } @@ -557,19 +558,19 @@ mod persist_schema { type Err = String; fn from_str(s: &str) -> Result { - let parts: Vec<_> = s.split('-').collect(); - let &[global_id, deploy_generation] = parts.as_slice() else { + let parts: Vec<_> = s.splitn(2, '-').collect(); + let &[global_id, build_version] = parts.as_slice() else { return Err(format!("invalid TableKey '{s}'")); }; let global_id = global_id .parse() .map_err(|e: ParseIntError| e.to_string())?; - let deploy_generation = deploy_generation + let build_version = build_version .parse() - .map_err(|e: ParseIntError| e.to_string())?; + .map_err(|e: semver::Error| e.to_string())?; Ok(TableKey { global_id, - deploy_generation, + build_version, }) } } @@ -588,6 +589,15 @@ mod persist_schema { } } + impl Default for TableKey { + fn default() -> Self { + Self { + global_id: Default::default(), + build_version: semver::Version::new(0, 0, 0), + } + } + } + impl Codec for TableKey { type Storage = (); type Schema = TableKeySchema; diff --git a/src/adapter/src/coord.rs b/src/adapter/src/coord.rs index 2f26cff656e02..0bd74636268a3 100644 --- a/src/adapter/src/coord.rs +++ b/src/adapter/src/coord.rs @@ -3986,7 +3986,6 @@ pub fn serve( let builtin_item_migration_config = if enable_0dt_deployment { BuiltinItemMigrationConfig::ZeroDownTime { persist_client: persist_client.clone(), - deploy_generation: controller_config.deploy_generation, read_only: read_only_controllers, } } else { diff --git a/src/catalog/src/config.rs b/src/catalog/src/config.rs index b088ad52060c9..f4a0acb81ae10 100644 --- a/src/catalog/src/config.rs +++ b/src/catalog/src/config.rs @@ -98,7 +98,6 @@ pub enum BuiltinItemMigrationConfig { Legacy, ZeroDownTime { persist_client: PersistClient, - deploy_generation: u64, read_only: bool, }, } diff --git a/src/catalog/src/durable/transaction.rs b/src/catalog/src/durable/transaction.rs index 45fc9ec2e0b81..47b4f465af9f1 100644 --- a/src/catalog/src/durable/transaction.rs +++ b/src/catalog/src/durable/transaction.rs @@ -1831,6 +1831,13 @@ impl<'a> Transaction<'a> { .map(|shard_id| shard_id.parse().expect("valid ShardId")) } + pub fn set_builtin_migration_shard(&mut self, shard_id: ShardId) -> Result<(), CatalogError> { + self.set_setting( + BUILTIN_MIGRATION_SHARD_KEY.to_string(), + Some(shard_id.to_string()), + ) + } + pub fn get_expression_cache_shard(&self) -> Option { self.get_setting(EXPRESSION_CACHE_SHARD_KEY.to_string()) .map(|shard_id| shard_id.parse().expect("valid ShardId")) From 15d12a28fda2449109bac04a303bbb5107026f9c Mon Sep 17 00:00:00 2001 From: Materialize Date: Wed, 29 Jan 2025 00:00:00 +0000 Subject: [PATCH 02/12] LICENSE: update change date --- LICENSE | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/LICENSE b/LICENSE index 9b59d683809b3..23abb8278dfbf 100644 --- a/LICENSE +++ b/LICENSE @@ -13,7 +13,7 @@ Business Source License 1.1 Licensor: Materialize, Inc. -Licensed Work: Materialize Version 20250128 +Licensed Work: Materialize Version 20250129 The Licensed Work is © 2025 Materialize, Inc. Additional Use Grant: Within a single installation of Materialize, you @@ -32,7 +32,7 @@ Additional Use Grant: Within a single installation of Materialize, you whose definitions are controlled by such third parties. -Change Date: January 28, 2029 +Change Date: January 29, 2029 Change License: Apache License, Version 2.0 From fcc0efd55256936664c8876d09ae15ed5bc05b35 Mon Sep 17 00:00:00 2001 From: Dennis Felsing Date: Tue, 28 Jan 2025 06:06:18 +0000 Subject: [PATCH 03/12] GCP terraform + helm-chart test --- bin/ci-builder | 1 + ci/builder/Dockerfile | 14 +- ci/nightly/pipeline.template.yml | 19 +- ci/plugins/mzcompose/hooks/pre-exit | 2 +- .../lint-main/checks/check-mzcompose-files.sh | 1 + test/terraform/gcp/main.tf | 81 +++ test/terraform/mzcompose.py | 540 ++++++++++++++++-- 7 files changed, 591 insertions(+), 67 deletions(-) create mode 100644 test/terraform/gcp/main.tf diff --git a/bin/ci-builder b/bin/ci-builder index c5c225ea17836..b6edaac6abc71 100755 --- a/bin/ci-builder +++ b/bin/ci-builder @@ -188,6 +188,7 @@ case "$cmd" in --env COMMON_ANCESTOR_OVERRIDE --env CONFLUENT_CLOUD_DEVEX_KAFKA_PASSWORD --env CONFLUENT_CLOUD_DEVEX_KAFKA_USERNAME + --env GCP_SERVICE_ACCOUNT_JSON --env GITHUB_TOKEN --env GPG_KEY --env LAUNCHDARKLY_API_TOKEN diff --git a/ci/builder/Dockerfile b/ci/builder/Dockerfile index c939b5c1f91c5..50c7a0bbed20b 100644 --- a/ci/builder/Dockerfile +++ b/ci/builder/Dockerfile @@ -326,7 +326,17 @@ RUN curl -fsSL https://github.com/deb-s3/deb-s3/releases/download/0.11.3/deb-s3- RUN curl -fsSL "https://awscli.amazonaws.com/awscli-exe-linux-$ARCH_GCC-2.17.2.zip" > awscli.zip \ && unzip awscli.zip \ && ./aws/install \ - && rm -rf aws + && rm -rf aws awscli.zip + +# Install the gcloud CLI. + +RUN arch_gcloud=$(echo "$ARCH_GCC" | sed -e "s/aarch64/arm/" -e "s/amd64/x86_64/") \ + && curl -fsSL "https://storage.googleapis.com/cloud-sdk-release/google-cloud-cli-507.0.0-linux-$arch_gcloud.tar.gz" > gcloud.tar.gz \ + && tar -xzf gcloud.tar.gz -C /opt \ + && rm gcloud.tar.gz \ + && CLOUDSDK_CORE_DISABLE_PROMPTS=1 /opt/google-cloud-sdk/install.sh --path-update false \ + && /opt/google-cloud-sdk/bin/gcloud config set disable_usage_reporting false \ + && /opt/google-cloud-sdk/bin/gcloud components install gke-gcloud-auth-plugin # Install docs site dependencies. These are towards the end for the same reason # as the Python dependencies. These are only supported on x86_64 at the moment. @@ -410,7 +420,7 @@ ENV TARGET_CC=$CC ENV TARGET_CXX=$CXX ENV TARGET_CXXSTDLIB=static=stdc++ ENV TARGET_RANLIB=$RANLIB -ENV PATH=/opt/x-tools/$ARCH_GCC-unknown-linux-gnu/bin:$PATH +ENV PATH=/opt/google-cloud-sdk/bin:/opt/x-tools/$ARCH_GCC-unknown-linux-gnu/bin:$PATH ENV CARGO_TARGET_X86_64_UNKNOWN_LINUX_GNU_LINKER=x86_64-unknown-linux-gnu-cc ENV CARGO_TARGET_AARCH64_UNKNOWN_LINUX_GNU_LINKER=aarch64-unknown-linux-gnu-cc ENV CARGO_TARGET_DIR=/mnt/build diff --git a/ci/nightly/pipeline.template.yml b/ci/nightly/pipeline.template.yml index 01d43fc235446..4a4af37729861 100644 --- a/ci/nightly/pipeline.template.yml +++ b/ci/nightly/pipeline.template.yml @@ -1216,15 +1216,32 @@ steps: depends_on: build-aarch64 timeout_in_minutes: 1200 concurrency: 1 - concurrency_group: 'terraform' + concurrency_group: 'terraform-aws' agents: queue: linux-aarch64-small plugins: - ./ci/plugins/scratch-aws-access: ~ - ./ci/plugins/mzcompose: composition: terraform + run: aws branches: "main v*.* lts-v*" + - id: terraform-gcp + label: "Terraform + Helm Chart E2E on GCP" + artifact_paths: [test/terraform/aws/terraform.tfstate] + depends_on: build-aarch64 + timeout_in_minutes: 1200 + concurrency: 1 + concurrency_group: 'terraform-gcp' + agents: + queue: linux-aarch64-small + plugins: + - ./ci/plugins/scratch-aws-access: ~ + - ./ci/plugins/mzcompose: + composition: terraform + run: gcp + #branches: "main v*.* lts-v*" + - group: "Output consistency" key: output-consistency steps: diff --git a/ci/plugins/mzcompose/hooks/pre-exit b/ci/plugins/mzcompose/hooks/pre-exit index 5153e85197d18..37f8a29cb3317 100755 --- a/ci/plugins/mzcompose/hooks/pre-exit +++ b/ci/plugins/mzcompose/hooks/pre-exit @@ -85,7 +85,7 @@ timeout 300 buildkite-agent artifact upload "$artifacts_str" || true bin/ci-builder run stable bin/ci-annotate-errors --test-cmd="$(cat test_cmd)" --test-desc="$(cat test_desc)" "${artifacts[@]}" > ci-annotate-errors.log || CI_ANNOTATE_ERRORS_RESULT=$? buildkite-agent artifact upload "ci-annotate-errors.log" -if [ ! -s services.log ] && [ "$BUILDKITE_LABEL" != "Maelstrom coverage of persist" ] && [ "$BUILDKITE_LABEL" != "Long single-node Maelstrom coverage of persist" ] && [ "$BUILDKITE_LABEL" != "Maelstrom coverage of txn-wal" ] && [ "$BUILDKITE_LABEL" != "Mz E2E Test" ] && [ "$BUILDKITE_LABEL" != "Output consistency (version for DFR)" ] && [ "$BUILDKITE_LABEL" != "Output consistency (version for CTF)" ] && [ "$BUILDKITE_LABEL" != "QA Canary Environment Base Load" ] && [ "$BUILDKITE_LABEL" != "Parallel Benchmark against QA Canary Environment" ] && [ "$BUILDKITE_LABEL" != "Parallel Benchmark against QA Benchmarking Staging Environment" ] && [ "$BUILDKITE_LABEL" != "Terraform + Helm Chart E2E on AWS" ]; then +if [ ! -s services.log ] && [ "$BUILDKITE_LABEL" != "Maelstrom coverage of persist" ] && [ "$BUILDKITE_LABEL" != "Long single-node Maelstrom coverage of persist" ] && [ "$BUILDKITE_LABEL" != "Maelstrom coverage of txn-wal" ] && [ "$BUILDKITE_LABEL" != "Mz E2E Test" ] && [ "$BUILDKITE_LABEL" != "Output consistency (version for DFR)" ] && [ "$BUILDKITE_LABEL" != "Output consistency (version for CTF)" ] && [ "$BUILDKITE_LABEL" != "QA Canary Environment Base Load" ] && [ "$BUILDKITE_LABEL" != "Parallel Benchmark against QA Canary Environment" ] && [ "$BUILDKITE_LABEL" != "Parallel Benchmark against QA Benchmarking Staging Environment" ] && [ "$BUILDKITE_LABEL" != "Terraform + Helm Chart E2E on AWS" ] && [ "$BUILDKITE_LABEL" != "Terraform + Helm Chart E2E on GCP" ]; then echo "+++ services.log is empty, failing" exit 1 fi diff --git a/ci/test/lint-main/checks/check-mzcompose-files.sh b/ci/test/lint-main/checks/check-mzcompose-files.sh index 0031f92c3dc3d..ed16e1ac22f96 100755 --- a/ci/test/lint-main/checks/check-mzcompose-files.sh +++ b/ci/test/lint-main/checks/check-mzcompose-files.sh @@ -45,6 +45,7 @@ check_default_workflow_references_others() { -not -wholename "./test/canary-environment/mzcompose.py" `# Only run manually` \ -not -wholename "./test/ssh-connection/mzcompose.py" `# Handled differently` \ -not -wholename "./test/scalability/mzcompose.py" `# Other workflows are for manual usage` \ + -not -wholename "./test/terraform/mzcompose.py" `# Handled differently` \ ) for file in "${MZCOMPOSE_TEST_FILES[@]}"; do diff --git a/test/terraform/gcp/main.tf b/test/terraform/gcp/main.tf new file mode 100644 index 0000000000000..5b6e628686591 --- /dev/null +++ b/test/terraform/gcp/main.tf @@ -0,0 +1,81 @@ +# Copyright Materialize, Inc. and contributors. All rights reserved. +# +# Use of this software is governed by the Business Source License +# included in the LICENSE file at the root of this repository. +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0. + +terraform { + required_version = ">= 1.0" + + required_providers { + google = { + source = "hashicorp/google" + version = ">= 6.0" + } + } +} + +provider "google" { + project = var.project_id + region = var.region +} + +module "materialize" { + source = "github.com/MaterializeInc/terraform-google-materialize?ref=v0.1.1" + + project_id = var.project_id + region = var.region + prefix = "tf-gcp-test" + + database_config = { + tier = "db-custom-2-4096" + version = "POSTGRES_15" + password = var.database_password + } + + labels = { + environment = "simple" + example = "true" + } + + install_materialize_operator = false +} + +variable "project_id" { + description = "GCP Project ID" + type = string + default = "materialize-ci" +} + +variable "region" { + description = "GCP Region" + type = string + default = "us-east1" +} + +variable "database_password" { + description = "Password for Cloud SQL database user" + default = "your-strong-password" + type = string + sensitive = true +} + +output "gke_cluster" { + description = "GKE cluster details" + value = module.materialize.gke_cluster + sensitive = true +} + +output "service_accounts" { + description = "Service account details" + value = module.materialize.service_accounts +} + +output "connection_strings" { + description = "Connection strings for metadata and persistence backends" + value = module.materialize.connection_strings + sensitive = true +} diff --git a/test/terraform/mzcompose.py b/test/terraform/mzcompose.py index 68bafb8c438ca..3c06dfc22c650 100644 --- a/test/terraform/mzcompose.py +++ b/test/terraform/mzcompose.py @@ -11,6 +11,7 @@ """ import argparse +import json import os import signal import subprocess @@ -34,6 +35,69 @@ Testdrive(), # Overridden below ] +COMPATIBLE_TESTDRIVE_FILES = [ + "array.td", + "cancel-subscribe.td", + "char-varchar-distinct.td", + "char-varchar-joins.td", + "char-varchar-multibyte.td", + "constants.td", + "coordinator-multiplicities.td", + "create-views.td", + "date_func.td", + "decimal-distinct.td", + "decimal-join.td", + "decimal-order.td", + "decimal-overflow.td", + "decimal-sum.td", + "decimal-zero.td", + "delete-using.td", + "drop.td", + "duplicate-table-names.td", + "failpoints.td", + "fetch-tail-as-of.td", + "fetch-tail-large-diff.td", + "fetch-tail-limit-timeout.td", + "fetch-tail-timestamp-zero.td", + "fetch-timeout.td", + "float_sum.td", + "get-started.td", + "github-11563.td", + "github-1947.td", + "github-3281.td", + "github-5502.td", + "github-5774.td", + "github-5873.td", + "github-5983.td", + "github-5984.td", + "github-6335.td", + "github-6744.td", + "github-6950.td", + "github-7171.td", + "github-7191.td", + "github-795.td", + "joins.td", + "jsonb.td", + "list.td", + "logging.td", + "map.td", + "multijoins.td", + "numeric-sum.td", + "numeric.td", + "oid.td", + "orms.td", + "pg-catalog.td", + "runtime-errors.td", + "search_path.td", + "self-test.td", + "string.td", + "subquery-scalar-errors.td", + "system-functions.td", + "test-skip-if.td", + "type_char_quoted.td", + "version.td", +] + def run_ignore_error( args: Sequence[Path | str], @@ -46,7 +110,7 @@ def run_ignore_error( pass -def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: +def workflow_aws(c: Composition, parser: WorkflowArgumentParser) -> None: """To run locally use `aws sso login` first.""" parser.add_argument( "--setup", @@ -68,68 +132,7 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: parser.add_argument( "files", nargs="*", - default=[ - "array.td", - "cancel-subscribe.td", - "char-varchar-distinct.td", - "char-varchar-joins.td", - "char-varchar-multibyte.td", - "constants.td", - "coordinator-multiplicities.td", - "create-views.td", - "date_func.td", - "decimal-distinct.td", - "decimal-join.td", - "decimal-order.td", - "decimal-overflow.td", - "decimal-sum.td", - "decimal-zero.td", - "delete-using.td", - "drop.td", - "duplicate-table-names.td", - "failpoints.td", - "fetch-tail-as-of.td", - "fetch-tail-large-diff.td", - "fetch-tail-limit-timeout.td", - "fetch-tail-timestamp-zero.td", - "fetch-timeout.td", - "float_sum.td", - "get-started.td", - "github-11563.td", - "github-1947.td", - "github-3281.td", - "github-5502.td", - "github-5774.td", - "github-5873.td", - "github-5983.td", - "github-5984.td", - "github-6335.td", - "github-6744.td", - "github-6950.td", - "github-7171.td", - "github-7191.td", - "github-795.td", - "joins.td", - "jsonb.td", - "list.td", - "logging.td", - "map.td", - "multijoins.td", - "numeric-sum.td", - "numeric.td", - "oid.td", - "orms.td", - "pg-catalog.td", - "runtime-errors.td", - "search_path.td", - "self-test.td", - "string.td", - "subquery-scalar-errors.td", - "system-functions.td", - "test-skip-if.td", - "type_char_quoted.td", - "version.td", - ], + default=COMPATIBLE_TESTDRIVE_FILES, help="run against the specified files", ) @@ -534,3 +537,414 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: ) run_ignore_error(["kubectl", "delete", "namespace", "materialize"]) spawn.runv(["terraform", "destroy", "-auto-approve"], cwd=path) + + +def workflow_gcp(c: Composition, parser: WorkflowArgumentParser) -> None: + """To run locally use `aws sso login` first.""" + parser.add_argument( + "--setup", + default=True, + action=argparse.BooleanOptionalAction, + help="Run setup steps", + ) + parser.add_argument( + "--cleanup", + default=True, + action=argparse.BooleanOptionalAction, + help="Destroy the region at the end of the workflow.", + ) + parser.add_argument( + "--tag", + type=str, + help="Custom version tag to use", + ) + parser.add_argument( + "files", + nargs="*", + default=COMPATIBLE_TESTDRIVE_FILES, + help="run against the specified files", + ) + + args = parser.parse_args() + + tag = args.tag or f"v{ci_util.get_mz_version()}--pr.g{git.rev_parse('HEAD')}" + materialize_environment = None + environmentd_port_forward_process = None + balancerd_port_forward_process = None + + path = MZ_ROOT / "test" / "terraform" / "gcp" + + gcp_service_account_json = os.getenv("GCP_SERVICE_ACCOUNT_JSON") + assert ( + gcp_service_account_json + ), "GCP_SERVICE_ACCOUNT_JSON environment variable has to be set" + gcloud_creds_path = path / "gcp.json" + with open(gcloud_creds_path, "w") as f: + f.write(gcp_service_account_json) + os.environ["GOOGLE_APPLICATION_CREDENTIALS"] = str(gcloud_creds_path) + + try: + spawn.runv(["gcloud", "config", "set", "project", "materialize-ci"]) + + spawn.runv( + [ + "gcloud", + "auth", + "activate-service-account", + f"--key-file={gcloud_creds_path}", + ], + ) + + if args.setup: + print("--- Setup") + spawn.runv(["terraform", "init"], cwd=path) + spawn.runv(["terraform", "validate"], cwd=path) + spawn.runv(["terraform", "plan"], cwd=path) + spawn.runv(["terraform", "apply", "-auto-approve"], cwd=path) + + json.loads( + spawn.capture( + ["terraform", "output", "-json", "service_accounts"], cwd=path + ).strip() + ) + gke_cluster = json.loads( + spawn.capture( + ["terraform", "output", "-json", "gke_cluster"], cwd=path + ).strip() + ) + connection_strings = json.loads( + spawn.capture( + ["terraform", "output", "-json", "connection_strings"], cwd=path + ).strip() + ) + + spawn.runv( + [ + "gcloud", + "container", + "clusters", + "get-credentials", + gke_cluster["name"], + "--region", + gke_cluster["location"], + "--project", + "materialize-ci", + ] + ) + + spawn.runv(["kubectl", "get", "nodes"]) + + if args.setup: + public_ip_address = spawn.capture( + ["curl", "http://checkip.amazonaws.com"] + ).strip() + + materialize_values = { + "operator": { + "image": {"tag": tag}, + "cloudProvider": { + "type": "gcp", + "region": "us-east1", + "providers": { + "gcp": { + "enabled": True, + } + }, + }, + }, + "rbac": {"enabled": False}, + "networkPolicies": { + "enabled": True, + "egress": {"enabled": True, "cidrs": ["0.0.0.0/0"]}, + "ingress": {"enabled": True, "cidrs": [f"{public_ip_address}/24"]}, + "internal": {"enabled": True}, + }, + } + + spawn.runv( + [ + "helm", + "install", + "materialize-operator", + "misc/helm-charts/operator", + "--namespace", + "materialize", + "--create-namespace", + "-f", + "-", + ], + cwd=MZ_ROOT, + stdin=yaml.dump(materialize_values).encode(), + ) + for i in range(60): + try: + spawn.runv( + ["kubectl", "get", "pods", "-n", "materialize"], + cwd=path, + ) + status = spawn.capture( + [ + "kubectl", + "get", + "pods", + "-n", + "materialize", + "-o", + "jsonpath={.items[0].status.phase}", + ], + cwd=path, + ) + if status == "Running": + break + except subprocess.CalledProcessError: + time.sleep(1) + else: + raise ValueError("Never completed") + + spawn.runv(["kubectl", "create", "namespace", "materialize-environment"]) + + materialize_backend_secret = { + "apiVersion": "v1", + "kind": "Secret", + "metadata": { + "name": "materialize-backend", + "namespace": "materialize-environment", + }, + "stringData": { + "metadata_backend_url": connection_strings["metadata_backend_url"], + "persist_backend_url": connection_strings["persist_backend_url"], + }, + } + + spawn.runv( + ["kubectl", "apply", "-f", "-"], + cwd=path, + stdin=yaml.dump(materialize_backend_secret).encode(), + ) + + materialize_environment = { + "apiVersion": "materialize.cloud/v1alpha1", + "kind": "Materialize", + "metadata": { + "name": "12345678-1234-1234-1234-123456789012", + "namespace": "materialize-environment", + }, + "spec": { + "environmentdImageRef": f"materialize/environmentd:{tag}", + "environmentdResourceRequirements": { + "limits": {"memory": "4Gi"}, + "requests": {"cpu": "2", "memory": "4Gi"}, + }, + "balancerdResourceRequirements": { + "limits": {"memory": "256Mi"}, + "requests": {"cpu": "100m", "memory": "256Mi"}, + }, + "backendSecretName": "materialize-backend", + }, + } + + spawn.runv( + ["kubectl", "apply", "-f", "-"], + cwd=path, + stdin=yaml.dump(materialize_environment).encode(), + ) + for i in range(60): + try: + spawn.runv( + [ + "kubectl", + "get", + "materializes", + "-n", + "materialize-environment", + ], + cwd=path, + ) + break + except subprocess.CalledProcessError: + time.sleep(1) + else: + raise ValueError("Never completed") + for i in range(180): + try: + spawn.runv( + ["kubectl", "get", "pods", "-n", "materialize-environment"], + cwd=path, + ) + status = spawn.capture( + [ + "kubectl", + "get", + "pods", + "-l", + "app=environmentd", + "-n", + "materialize-environment", + "-o", + "jsonpath={.items[0].status.phase}", + ], + cwd=path, + ) + if status == "Running": + break + except subprocess.CalledProcessError: + time.sleep(1) + else: + raise ValueError("Never completed") + + # Can take a while for balancerd to come up + for i in range(240): + try: + status = spawn.capture( + [ + "kubectl", + "get", + "pods", + "-l", + "app=balancerd", + "-n", + "materialize-environment", + "-o", + "jsonpath={.items[0].status.phase}", + ], + cwd=path, + ) + if status == "Running": + break + except subprocess.CalledProcessError: + time.sleep(1) + else: + raise ValueError("Never completed") + + print("--- Running tests") + environmentd_name = spawn.capture( + [ + "kubectl", + "get", + "pods", + "-l", + "app=environmentd", + "-n", + "materialize-environment", + "-o", + "jsonpath={.items[*].metadata.name}", + ], + cwd=path, + ) + + balancerd_name = spawn.capture( + [ + "kubectl", + "get", + "pods", + "-l", + "app=balancerd", + "-n", + "materialize-environment", + "-o", + "jsonpath={.items[*].metadata.name}", + ], + cwd=path, + ) + # error: arguments in resource/name form must have a single resource and name + print(f"Got balancerd name: {balancerd_name}") + + environmentd_port_forward_process = subprocess.Popen( + [ + "kubectl", + "port-forward", + f"pod/{environmentd_name}", + "-n", + "materialize-environment", + "6877:6877", + "6878:6878", + ], + preexec_fn=os.setpgrp, + ) + balancerd_port_forward_process = subprocess.Popen( + [ + "kubectl", + "port-forward", + f"pod/{balancerd_name}", + "-n", + "materialize-environment", + "6875:6875", + "6876:6876", + ], + preexec_fn=os.setpgrp, + ) + time.sleep(10) + + with c.override( + Testdrive( + materialize_url="postgres://materialize@127.0.0.1:6875/materialize", + materialize_url_internal="postgres://mz_system:materialize@127.0.0.1:6877/materialize", + materialize_use_https=True, + no_consistency_checks=True, + network_mode="host", + volume_workdir="../testdrive:/workdir", + # For full testdrive support we'll need: + # kafka_url=... + # schema_registry_url=... + # aws_endpoint=... + ) + ): + c.up("testdrive", persistent=True) + c.testdrive( + dedent( + """ + > SELECT 1 + 1 + """ + ) + ) + + with psycopg.connect( + "postgres://materialize@127.0.0.1:6875/materialize" + ) as conn: + with conn.cursor() as cur: + cur.execute("SELECT 1") + results = cur.fetchall() + assert results == [(1,)], results + cur.execute("SELECT mz_version()") + version = cur.fetchall()[0][0] + assert version.startswith(tag.split("--")[0] + " ") + with open( + MZ_ROOT / "misc" / "helm-charts" / "operator" / "Chart.yaml" + ) as f: + content = yaml.load(f, Loader=yaml.Loader) + helm_chart_version = content["version"] + assert version.endswith(f", helm chart: {helm_chart_version})") + + c.run_testdrive_files(*args.files) + finally: + if environmentd_port_forward_process: + os.killpg(os.getpgid(environmentd_port_forward_process.pid), signal.SIGTERM) + if balancerd_port_forward_process: + os.killpg(os.getpgid(balancerd_port_forward_process.pid), signal.SIGTERM) + + if args.cleanup: + print("--- Cleaning up") + if materialize_environment: + run_ignore_error( + ["kubectl", "delete", "-f", "-"], + cwd=path, + stdin=yaml.dump(materialize_environment).encode(), + ) + run_ignore_error( + [ + "kubectl", + "delete", + "materialize.materialize.cloud/12345678-1234-1234-1234-123456789012", + "-n" "materialize-environment", + ] + ) + run_ignore_error( + ["kubectl", "delete", "namespace", "materialize-environment"] + ) + run_ignore_error( + ["helm", "uninstall", "materialize-operator"], + cwd=path, + ) + run_ignore_error(["kubectl", "delete", "namespace", "materialize"]) + spawn.runv(["terraform", "destroy", "-auto-approve"], cwd=path) From e17ed4c5d7760377eec55e989ae67cbb005e28a9 Mon Sep 17 00:00:00 2001 From: Dennis Felsing Date: Wed, 29 Jan 2025 10:33:26 +0000 Subject: [PATCH 04/12] canary load: Retry frontegg auth Context: https://materializeinc.slack.com/archives/CL68GT3AT/p1738136577471549 --- test/canary-load/mzcompose.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/canary-load/mzcompose.py b/test/canary-load/mzcompose.py index d5bda2320e3f4..00f14e543c7ac 100644 --- a/test/canary-load/mzcompose.py +++ b/test/canary-load/mzcompose.py @@ -158,7 +158,7 @@ def fetch_token(user_name: str, password: str) -> str: password=password, host="admin.cloud.materialize.com/frontegg", scheme="https", - max_tries=1, + max_tries=10, ) From 1e1eb50330b178951aea6489ee9cbe01b9c76bee Mon Sep 17 00:00:00 2001 From: Dennis Felsing Date: Wed, 29 Jan 2025 10:54:50 +0000 Subject: [PATCH 05/12] ci: Clean up potentially leftover state in case job was cancelled Happened in https://buildkite.com/materialize/nightly/builds/10992#0194ae51-f610-4440-84f7-84fb30f17361 The usual cleanup logic can't be run even though it's in a `finally` block in our Python test code. The `pre-exit` hook is still being run. --- ci/plugins/mzcompose/hooks/pre-exit | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/ci/plugins/mzcompose/hooks/pre-exit b/ci/plugins/mzcompose/hooks/pre-exit index 37f8a29cb3317..25daab73409a6 100755 --- a/ci/plugins/mzcompose/hooks/pre-exit +++ b/ci/plugins/mzcompose/hooks/pre-exit @@ -145,3 +145,7 @@ ci_unimportant_heading ":docker: Cleaning up after mzcompose" run kill || true run rm --force -v run down --volumes + +ci_unimportant_heading "terraform: Destroying leftover state in case job was cancelled or timed out..." +bin/ci-builder run stable terraform -chdir=test/terraform/aws destroy || true +bin/ci-builder run stable terraform -chdir=test/terraform/gcp destroy || true From 7eae3b35415dab1d56ff1b8160ffd4dafc8ccd33 Mon Sep 17 00:00:00 2001 From: Dennis Felsing Date: Wed, 29 Jan 2025 12:20:30 +0000 Subject: [PATCH 06/12] ci: Enable branch filter for GCP terraform+helm chart test Works now, doesn't have to run on PRs --- ci/nightly/pipeline.template.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/nightly/pipeline.template.yml b/ci/nightly/pipeline.template.yml index aaa6db84fe216..d6a9bae57793d 100644 --- a/ci/nightly/pipeline.template.yml +++ b/ci/nightly/pipeline.template.yml @@ -1252,7 +1252,7 @@ steps: - ./ci/plugins/mzcompose: composition: terraform run: gcp - #branches: "main v*.* lts-v*" + branches: "main v*.* lts-v*" - group: "Output consistency" key: output-consistency From 2ed27fedf89b9b747544fd2dbacef31711682452 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Wed, 29 Jan 2025 15:30:00 +0100 Subject: [PATCH 07/12] Update to latest Timely (#31158) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Point Materialize at latest Timely. We need to incorporate some changes around reachability logging, which is now typed, and event iterators that return cow'ed data. Some of the complexity stems from the fact that event links are single-writer, so we need separate event links for each reachability log variant. Signed-off-by: Moritz Hoffmann ### Motivation ### Tips for reviewer ### Checklist - [ ] This PR has adequate test coverage / QA involvement has been duly considered. ([trigger-ci for additional test/nightly runs](https://trigger-ci.dev.materialize.com/)) - [ ] This PR has an associated up-to-date [design doc](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/design/README.md), is a design doc ([template](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/design/00000000_template.md)), or is sufficiently small to not require a design. - [ ] If this PR evolves [an existing `$T ⇔ Proto$T` mapping](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/command-and-response-binary-encoding.md) (possibly in a backwards-incompatible way), then it is tagged with a `T-proto` label. - [ ] If this PR will require changes to cloud orchestration or tests, there is a companion cloud PR to account for those changes that is tagged with the release-blocker label ([example](https://github.com/MaterializeInc/cloud/pull/5021)). - [ ] If this PR includes major [user-facing behavior changes](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/guide-changes.md#what-changes-require-a-release-note), I have pinged the relevant PM to schedule a changelog post. --------- Signed-off-by: Moritz Hoffmann --- Cargo.lock | 12 +- src/adapter-types/Cargo.toml | 2 +- src/adapter/Cargo.toml | 4 +- src/catalog/Cargo.toml | 4 +- src/catalog/src/builtin.rs | 22 +++- src/cluster/Cargo.toml | 4 +- src/compute-client/Cargo.toml | 4 +- src/compute-client/src/logging.rs | 12 +- src/compute-types/Cargo.toml | 4 +- src/compute/Cargo.toml | 6 +- src/compute/src/logging.rs | 13 ++- src/compute/src/logging/compute.rs | 6 +- src/compute/src/logging/differential.rs | 25 +++-- src/compute/src/logging/initialize.rs | 104 +++++++++++------- src/compute/src/logging/reachability.rs | 23 ++-- src/compute/src/logging/timely.rs | 12 +- src/controller/Cargo.toml | 2 +- src/durable-cache/Cargo.toml | 4 +- src/environmentd/Cargo.toml | 2 +- src/expr/Cargo.toml | 2 +- src/interchange/Cargo.toml | 4 +- src/persist-cli/Cargo.toml | 4 +- src/persist-client/Cargo.toml | 4 +- src/persist-types/Cargo.toml | 2 +- src/persist/Cargo.toml | 4 +- src/repr/Cargo.toml | 4 +- src/service/Cargo.toml | 2 +- src/storage-client/Cargo.toml | 4 +- src/storage-controller/Cargo.toml | 4 +- src/storage-operators/Cargo.toml | 4 +- src/storage-types/Cargo.toml | 4 +- src/storage/Cargo.toml | 4 +- src/timely-util/Cargo.toml | 4 +- src/timely-util/src/replay.rs | 23 ++-- src/transform/Cargo.toml | 2 +- src/txn-wal/Cargo.toml | 4 +- test/sqllogictest/cluster.slt | 11 +- .../mz_catalog_server_index_accounting.slt | 5 +- .../testdrive-old-kafka-src-syntax/indexes.td | 2 +- test/testdrive/indexes.td | 2 +- test/testdrive/logging.td | 6 +- 41 files changed, 207 insertions(+), 163 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 21a1db322eb3f..6797f59041cc5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2369,9 +2369,9 @@ dependencies = [ [[package]] name = "differential-dataflow" -version = "0.13.3" +version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86f47ab549a056e3959ce2036d6b20ba369a75e2a0605ccf64256e153626b352" +checksum = "75b93af605b7f82fbf6e671a5bb0f940b385e57254a0af59ce6dfb98b8c4b302" dependencies = [ "columnar", "fnv", @@ -2381,9 +2381,9 @@ dependencies = [ [[package]] name = "differential-dogs3" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8ece5ac3c62f1cb6e21560b53102963aa91513d3f72911d99b6620f2d3ac6b1" +checksum = "17ba29145a1df1bdc3da1142eeb991f0866620c79ce9d85e83a3837f29112ba0" dependencies = [ "differential-dataflow", "serde", @@ -10406,9 +10406,9 @@ dependencies = [ [[package]] name = "timely" -version = "0.16.1" +version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ace21eb2a22c1b80b0b9b5be0627eb0f95a128a5751e866aefc90a85e3007d3" +checksum = "0506b6506ef04c371ea6151942df60e309e7f5e710458a6b533e364ee0b3cab3" dependencies = [ "bincode", "byteorder", diff --git a/src/adapter-types/Cargo.toml b/src/adapter-types/Cargo.toml index 5383238d65749..d1cdc0ae56507 100644 --- a/src/adapter-types/Cargo.toml +++ b/src/adapter-types/Cargo.toml @@ -15,7 +15,7 @@ mz-ore = { path = "../ore" } mz-repr = { path = "../repr" } mz-storage-types = { path = "../storage-types" } serde = "1.0.152" -timely = "0.16.0" +timely = "0.17.0" workspace-hack = { version = "0.0.0", path = "../workspace-hack", optional = true } [package.metadata.cargo-udeps.ignore] diff --git a/src/adapter/Cargo.toml b/src/adapter/Cargo.toml index a4298b2fe1703..92c489f39a4e5 100644 --- a/src/adapter/Cargo.toml +++ b/src/adapter/Cargo.toml @@ -18,7 +18,7 @@ bytesize = "1.1.0" chrono = { version = "0.4.35", default-features = false, features = ["std"] } dec = "0.4.8" derivative = "2.2.0" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" enum-kinds = "0.5.1" fail = { version = "0.5.1", features = ["failpoints"] } futures = "0.3.25" @@ -78,7 +78,7 @@ serde_plain = "1.0.1" sha2 = "0.10.6" smallvec = { version = "1.10.0", features = ["union"] } static_assertions = "1.1" -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", features = ["rt", "time"] } tokio-postgres = { version = "0.7.8" } tracing = "0.1.37" diff --git a/src/catalog/Cargo.toml b/src/catalog/Cargo.toml index 2751f63aec9c3..6d1e030684df6 100644 --- a/src/catalog/Cargo.toml +++ b/src/catalog/Cargo.toml @@ -18,7 +18,7 @@ bytesize = "1.1.0" chrono = { version = "0.4.35", default-features = false, features = ["std"] } clap = { version = "4.5.23", features = ["derive"] } derivative = "2.2.0" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" futures = "0.3.25" ipnet = "2.5.0" itertools = "0.12.1" @@ -60,7 +60,7 @@ serde_plain = "1.0.1" static_assertions = "1.1" sha2 = "0.10.6" thiserror = "1.0.37" -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0" } tracing = "0.1.37" uuid = "1.2.2" diff --git a/src/catalog/src/builtin.rs b/src/catalog/src/builtin.rs index f14e5c808b7a9..692a60a394c06 100644 --- a/src/catalog/src/builtin.rs +++ b/src/catalog/src/builtin.rs @@ -5567,15 +5567,25 @@ pub static MZ_DATAFLOW_OPERATOR_REACHABILITY_PER_WORKER: LazyLock = oid: oid::VIEW_MZ_DATAFLOW_OPERATOR_REACHABILITY_PER_WORKER_OID, column_defs: None, sql: "SELECT - address, + addr2.id, + reachability.worker_id, port, - worker_id, update_type, time, pg_catalog.count(*) as count FROM - mz_introspection.mz_dataflow_operator_reachability_raw -GROUP BY address, port, worker_id, update_type, time", + mz_introspection.mz_dataflow_operator_reachability_raw reachability, + mz_introspection.mz_dataflow_addresses_per_worker addr1, + mz_introspection.mz_dataflow_addresses_per_worker addr2 +WHERE + CASE + WHEN source = 0 THEN addr2.address = addr1.address + ELSE addr2.address = addr1.address || reachability.source + END + AND addr1.id = reachability.id + AND addr1.worker_id = reachability.worker_id + AND addr2.worker_id = reachability.worker_id +GROUP BY addr2.id, reachability.worker_id, port, update_type, time", access: vec![PUBLIC_SELECT], }); @@ -5587,13 +5597,13 @@ pub static MZ_DATAFLOW_OPERATOR_REACHABILITY: LazyLock = column_defs: None, sql: " SELECT - address, + id, port, update_type, time, pg_catalog.sum(count) as count FROM mz_introspection.mz_dataflow_operator_reachability_per_worker -GROUP BY address, port, update_type, time", +GROUP BY id, port, update_type, time", access: vec![PUBLIC_SELECT], }); diff --git a/src/cluster/Cargo.toml b/src/cluster/Cargo.toml index d40e4100b2481..43ca3a040c8c2 100644 --- a/src/cluster/Cargo.toml +++ b/src/cluster/Cargo.toml @@ -13,7 +13,7 @@ workspace = true anyhow = "1.0.66" async-trait = "0.1.83" crossbeam-channel = "0.5.8" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" futures = "0.3.25" mz-cluster-client = { path = "../cluster-client" } mz-ore = { path = "../ore", features = ["async", "process", "tracing_"] } @@ -21,7 +21,7 @@ mz-persist-client = { path = "../persist-client" } mz-service = { path = "../service" } mz-txn-wal = { path = "../txn-wal" } regex = "1.7.0" -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", features = ["fs", "rt", "sync", "net"] } tracing = "0.1.37" workspace-hack = { version = "0.0.0", path = "../workspace-hack", optional = true } diff --git a/src/compute-client/Cargo.toml b/src/compute-client/Cargo.toml index dd09ba95f42ee..c7ed7aa902e90 100644 --- a/src/compute-client/Cargo.toml +++ b/src/compute-client/Cargo.toml @@ -15,7 +15,7 @@ async-trait = "0.1.83" bytesize = "1.1.0" crossbeam-channel = "0.5.8" derivative = "2.2.0" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" futures = "0.3.25" http = "1.1.0" mz-build-info = { path = "../build-info" } @@ -43,7 +43,7 @@ prost = { version = "0.13.2", features = ["no-recursion-limit"] } serde = { version = "1.0.152", features = ["derive"] } serde_json = "1.0.125" thiserror = "1.0.37" -timely = "0.16.0" +timely = "0.17.0" tokio = "1.38.0" tokio-stream = "0.1.11" tonic = "0.12.1" diff --git a/src/compute-client/src/logging.rs b/src/compute-client/src/logging.rs index c6f56ae1a8982..c0a617d8af4bb 100644 --- a/src/compute-client/src/logging.rs +++ b/src/compute-client/src/logging.rs @@ -448,16 +448,10 @@ impl LogVariant { .finish(), LogVariant::Timely(TimelyLog::Reachability) => RelationDesc::builder() - .with_column( - "address", - ScalarType::List { - element_type: Box::new(ScalarType::UInt64), - custom_id: None, - } - .nullable(false), - ) - .with_column("port", ScalarType::UInt64.nullable(false)) + .with_column("id", ScalarType::UInt64.nullable(false)) .with_column("worker_id", ScalarType::UInt64.nullable(false)) + .with_column("source", ScalarType::UInt64.nullable(false)) + .with_column("port", ScalarType::UInt64.nullable(false)) .with_column("update_type", ScalarType::String.nullable(false)) .with_column("time", ScalarType::MzTimestamp.nullable(true)) .finish(), diff --git a/src/compute-types/Cargo.toml b/src/compute-types/Cargo.toml index a82db2a67b378..4e368867b8edd 100644 --- a/src/compute-types/Cargo.toml +++ b/src/compute-types/Cargo.toml @@ -12,7 +12,7 @@ workspace = true [dependencies] columnar = "0.2.2" columnation = "0.1.0" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" itertools = "0.12.1" mz-dyncfg = { path = "../dyncfg" } mz-expr = { path = "../expr" } @@ -24,7 +24,7 @@ proptest = { version = "1.6.0", default-features = false, features = ["std"] } proptest-derive = { version = "0.5.1", features = ["boxed_union"] } prost = { version = "0.13.2", features = ["no-recursion-limit"] } serde = { version = "1.0.152", features = ["derive"] } -timely = "0.16.0" +timely = "0.17.0" tracing = "0.1.37" workspace-hack = { version = "0.0.0", path = "../workspace-hack", optional = true } diff --git a/src/compute/Cargo.toml b/src/compute/Cargo.toml index 3f9ba4d140a3d..e972dbb2f1782 100644 --- a/src/compute/Cargo.toml +++ b/src/compute/Cargo.toml @@ -16,8 +16,8 @@ bytesize = "1.1.0" columnar = "0.2.2" crossbeam-channel = "0.5.8" dec = { version = "0.4.8", features = ["serde"] } -differential-dataflow = "0.13.3" -differential-dogs3 = "0.1.3" +differential-dataflow = "0.13.4" +differential-dogs3 = "0.1.4" futures = "0.3.25" itertools = "0.12.1" lgalloc = "0.4" @@ -39,7 +39,7 @@ prometheus = { version = "0.13.3", default-features = false } scopeguard = "1.1.0" serde = { version = "1.0.152", features = ["derive"] } smallvec = { version = "1.10.0", features = ["serde", "union"] } -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", features = ["fs", "rt", "sync", "net"] } tracing = "0.1.37" uuid = { version = "1.7.0", features = ["serde", "v4"] } diff --git a/src/compute/src/logging.rs b/src/compute/src/logging.rs index 23a9a18f02f6f..ede369d705c49 100644 --- a/src/compute/src/logging.rs +++ b/src/compute/src/logging.rs @@ -115,18 +115,23 @@ where /// /// This is just a bundle-type intended to make passing around its contents in the logging /// initialization code more convenient. +/// +/// The `N` type parameter specifies the number of links to create for the event queue. We need +/// separate links for queues that feed from multiple loggers because the `EventLink` type is not +/// multi-producer safe (it is a linked-list, and multiple writers would blindly append, replacing +/// existing new data, and cutting off other writers). #[derive(Clone)] -struct EventQueue { - link: Rc>, +struct EventQueue { + links: [Rc>; N], activator: RcActivator, } -impl EventQueue { +impl EventQueue { fn new(name: &str) -> Self { let activator_name = format!("{name}_activator"); let activate_after = 128; Self { - link: Rc::new(EventLink::new()), + links: [(); N].map(|_| Rc::new(EventLink::new())), activator: RcActivator::new(activator_name, activate_after), } } diff --git a/src/compute/src/logging/compute.rs b/src/compute/src/logging/compute.rs index 049f952b2f50a..8c0610d3b055a 100644 --- a/src/compute/src/logging/compute.rs +++ b/src/compute/src/logging/compute.rs @@ -308,16 +308,16 @@ pub(super) fn construct( worker.dataflow_named("Dataflow: compute logging", move |scope| { let enable_logging = config.enable_logging; - let (logs, token) = Some(event_queue.link).mz_replay::<_, ProvidedBuilder<_>, _>( + let (logs, token) = event_queue.links.mz_replay::<_, ProvidedBuilder<_>, _>( scope, "compute logs", config.interval, event_queue.activator, - move |mut session, data| { + move |mut session, mut data| { // If logging is disabled, we still need to install the indexes, but we can leave them // empty. We do so by immediately filtering all logs events. if enable_logging { - session.give_container(&mut data.clone()) + session.give_container(data.to_mut()) } }, ); diff --git a/src/compute/src/logging/differential.rs b/src/compute/src/logging/differential.rs index 42ae6e508bdec..7e3affdc5be04 100644 --- a/src/compute/src/logging/differential.rs +++ b/src/compute/src/logging/differential.rs @@ -20,15 +20,15 @@ use differential_dataflow::logging::{ }; use mz_ore::cast::CastFrom; use mz_repr::{Datum, Diff, Timestamp}; -use mz_timely_util::containers::{columnar_exchange, Col2ValBatcher, ColumnBuilder}; +use mz_timely_util::containers::{ + columnar_exchange, Col2ValBatcher, ColumnBuilder, ProvidedBuilder, +}; use mz_timely_util::replay::MzReplay; use timely::communication::Allocate; -use timely::container::CapacityContainerBuilder; use timely::dataflow::channels::pact::{ExchangeCore, Pipeline}; use timely::dataflow::channels::pushers::buffer::Session; use timely::dataflow::channels::pushers::{Counter, Tee}; use timely::dataflow::operators::generic::builder_rc::OperatorBuilder; -use timely::dataflow::operators::Filter; use timely::dataflow::Stream; use crate::extensions::arrange::MzArrangeCore; @@ -57,21 +57,22 @@ pub(super) fn construct( let dataflow_index = worker.next_dataflow_index(); worker.dataflow_named("Dataflow: differential logging", move |scope| { - let (mut logs, token) = Some(event_queue.link) - .mz_replay::<_, CapacityContainerBuilder<_>, _>( + let enable_logging = config.enable_logging; + let (logs, token) = event_queue.links + .mz_replay::<_, ProvidedBuilder<_>, _>( scope, "differential logs", config.interval, event_queue.activator, - |mut session, data| session.give_iterator(data.iter()), + move |mut session, mut data|{ + // If logging is disabled, we still need to install the indexes, but we can leave them + // empty. We do so by immediately filtering all logs events. + if enable_logging { + session.give_container(data.to_mut()) + } + } ); - // If logging is disabled, we still need to install the indexes, but we can leave them - // empty. We do so by immediately filtering all logs events. - if !config.enable_logging { - logs = logs.filter(|_| false); - } - // Build a demux operator that splits the replayed event stream up into the separate // logging streams. let mut demux = diff --git a/src/compute/src/logging/initialize.rs b/src/compute/src/logging/initialize.rs index 9c4a51fc7adf0..3b9d7f65b9899 100644 --- a/src/compute/src/logging/initialize.rs +++ b/src/compute/src/logging/initialize.rs @@ -21,8 +21,8 @@ use mz_timely_util::containers::{Column, ColumnBuilder}; use mz_timely_util::operator::CollectionExt; use timely::communication::Allocate; use timely::container::{ContainerBuilder, PushInto}; -use timely::logging::{ProgressEventTimestamp, TimelyEvent, TimelyEventBuilder}; -use timely::logging_core::Logger; +use timely::logging::{TimelyEvent, TimelyEventBuilder}; +use timely::logging_core::{Logger, Registry}; use timely::order::Product; use timely::progress::reachability::logging::{TrackerEvent, TrackerEventBuilder}; @@ -81,10 +81,7 @@ pub fn initialize( (logger, traces) } -pub(super) type ReachabilityEvent = ( - Vec, - Vec<(usize, usize, bool, Option, Diff)>, -); +pub(super) type ReachabilityEvent = (usize, Vec<(usize, usize, bool, Timestamp, Diff)>); struct LoggingContext<'a, A: Allocate> { worker: &'a mut timely::worker::Worker, @@ -93,7 +90,7 @@ struct LoggingContext<'a, A: Allocate> { now: Instant, start_offset: Duration, t_event_queue: EventQueue>, - r_event_queue: EventQueue>, + r_event_queue: EventQueue, 3>, d_event_queue: EventQueue>, c_event_queue: EventQueue>, shared_state: Rc>, @@ -149,15 +146,35 @@ impl LoggingContext<'_, A> { .collect() } + /// Construct a new reachability logger for timestamp type `T`. + /// + /// Inserts a logger with the name `timely/reachability/{type_name::()}`, following + /// Timely naming convention. + fn register_reachability_logger( + &self, + registry: &mut Registry, + index: usize, + ) { + let logger = self.reachability_logger::(index); + let type_name = std::any::type_name::(); + registry.insert_logger(&format!("timely/reachability/{type_name}"), logger); + } + + /// Register all loggers with the timely worker. + /// + /// Registers the timely, differential, compute, and reachability loggers. fn register_loggers(&self) { let t_logger = self.simple_logger::(self.t_event_queue.clone()); - let r_logger = self.reachability_logger(); let d_logger = self.simple_logger::(self.d_event_queue.clone()); let c_logger = self.simple_logger::(self.c_event_queue.clone()); let mut register = self.worker.log_register(); register.insert_logger("timely", t_logger); - register.insert_logger("timely/reachability", r_logger); + // Note that each reachability logger has a unique index, this is crucial to avoid dropping + // data because the event link structure is not multi-producer safe. + self.register_reachability_logger::(&mut register, 0); + self.register_reachability_logger::>>(&mut register, 1); + self.register_reachability_logger::<(Timestamp, Subtime)>(&mut register, 2); register.insert_logger("differential/arrange", d_logger); register.insert_logger("materialize/compute", c_logger.clone()); @@ -168,7 +185,9 @@ impl LoggingContext<'_, A> { &self, event_queue: EventQueue, ) -> Logger { - let mut logger = BatchLogger::new(event_queue.link, self.interval_ms); + let [link] = event_queue.links; + let mut logger = BatchLogger::new(link, self.interval_ms); + let activator = event_queue.activator.clone(); Logger::new( self.now, self.start_offset, @@ -176,18 +195,23 @@ impl LoggingContext<'_, A> { if let Some(data) = data.take() { logger.publish_batch(data); } else if logger.report_progress(*time) { - event_queue.activator.activate(); + activator.activate(); } }, ) } - fn reachability_logger(&self) -> Logger { - let event_queue = self.r_event_queue.clone(); - - let mut logger = BatchLogger::new(event_queue.link, self.interval_ms); + /// Construct a reachability logger for timestamp type `T`. The index must + /// refer to a unique link in the reachability event queue. + fn reachability_logger(&self, index: usize) -> Logger> + where + T: ExtractTimestamp, + { + let link = Rc::clone(&self.r_event_queue.links[index]); + let mut logger = BatchLogger::new(link, self.interval_ms); let mut massaged = Vec::new(); let mut builder = ColumnBuilder::default(); + let activator = self.r_event_queue.activator.clone(); let action = move |batch_time: &Duration, data: &mut Option>| { if let Some(data) = data { @@ -197,25 +221,23 @@ impl LoggingContext<'_, A> { TrackerEvent::SourceUpdate(update) => { massaged.extend(update.updates.iter().map( |(node, port, time, diff)| { - let ts = try_downcast_timestamp(time); let is_source = true; - (*node, *port, is_source, ts, *diff) + (*node, *port, is_source, T::extract(time), *diff) }, )); - builder.push_into((time, (&update.tracker_id, &massaged))); + builder.push_into((time, (update.tracker_id, &massaged))); massaged.clear(); } TrackerEvent::TargetUpdate(update) => { massaged.extend(update.updates.iter().map( |(node, port, time, diff)| { - let ts = try_downcast_timestamp(time); let is_source = false; - (*node, *port, is_source, ts, *diff) + (*node, *port, is_source, time.extract(), *diff) }, )); - builder.push_into((time, (&update.tracker_id, &massaged))); + builder.push_into((time, (update.tracker_id, &massaged))); massaged.clear(); } } @@ -230,7 +252,7 @@ impl LoggingContext<'_, A> { } if logger.report_progress(*batch_time) { - event_queue.activator.activate(); + activator.activate(); } } }; @@ -239,20 +261,26 @@ impl LoggingContext<'_, A> { } } -/// Extracts a `Timestamp` from a `dyn ProgressEventTimestamp`. -/// -/// For nested timestamps, only extracts the outermost one. The rest of the timestamps are -/// ignored for now. -#[inline] -fn try_downcast_timestamp(time: &dyn ProgressEventTimestamp) -> Option { - let time_any = time.as_any(); - time_any - .downcast_ref::() - .copied() - .or_else(|| { - time_any - .downcast_ref::>>() - .map(|t| t.outer) - }) - .or_else(|| time_any.downcast_ref::<(Timestamp, Subtime)>().map(|t| t.0)) +/// Helper trait to extract a timestamp from various types of timestamp used in rendering. +trait ExtractTimestamp: Clone + 'static { + /// Extracts the timestamp from the type. + fn extract(&self) -> Timestamp; +} + +impl ExtractTimestamp for Timestamp { + fn extract(&self) -> Timestamp { + *self + } +} + +impl ExtractTimestamp for Product> { + fn extract(&self) -> Timestamp { + self.outer + } +} + +impl ExtractTimestamp for (Timestamp, Subtime) { + fn extract(&self) -> Timestamp { + self.0 + } } diff --git a/src/compute/src/logging/reachability.rs b/src/compute/src/logging/reachability.rs index 079ea5972e332..463a30f21430b 100644 --- a/src/compute/src/logging/reachability.rs +++ b/src/compute/src/logging/reachability.rs @@ -39,7 +39,7 @@ use crate::typedefs::RowRowSpine; pub(super) fn construct( worker: &mut timely::worker::Worker, config: &LoggingConfig, - event_queue: EventQueue>, + event_queue: EventQueue, 3>, ) -> BTreeMap { let interval_ms = std::cmp::max(1, config.interval.as_millis()); let worker_index = worker.index(); @@ -48,10 +48,10 @@ pub(super) fn construct( // A dataflow for multiple log-derived arrangements. let traces = worker.dataflow_named("Dataflow: timely reachability logging", move |scope| { let enable_logging = config.enable_logging; - type UpdatesKey = (bool, Vec, usize, usize, Option); + type UpdatesKey = (bool, usize, usize, usize, Timestamp); type CB = ColumnBuilder<((UpdatesKey, ()), Timestamp, Diff)>; - let (updates, token) = Some(event_queue.link).mz_replay::<_, CB, _>( + let (updates, token) = event_queue.links.mz_replay::<_, CB, _>( scope, "reachability logs", config.interval, @@ -62,11 +62,11 @@ pub(super) fn construct( if !enable_logging { return; } - for (time, (addr, massaged)) in data.iter() { + for (time, (operator_id, massaged)) in data.iter() { let time_ms = ((time.as_millis() / interval_ms) + 1) * interval_ms; let time_ms: Timestamp = time_ms.try_into().expect("must fit"); for (source, port, update_type, ts, diff) in massaged.into_iter() { - let datum = (update_type, addr, source, port, ts); + let datum = (update_type, operator_id, source, port, ts); session.give(((datum, ()), time_ms, diff)); } } @@ -76,22 +76,17 @@ pub(super) fn construct( // Restrict results by those logs that are meant to be active. let logs_active = [LogVariant::Timely(TimelyLog::Reachability)]; - let mut addr_row = Row::default(); let updates = consolidate_and_pack::<_, Col2ValBatcher, ColumnBuilder<_>, _, _>( &updates, TimelyLog::Reachability, move |((datum, ()), time, diff), packer, session| { - let (update_type, addr, source, port, ts) = datum; + let (update_type, operator_id, source, port, ts) = datum; let update_type = if *update_type { "source" } else { "target" }; - addr_row.packer().push_list( - IntoIterator::into_iter(addr) - .chain(std::iter::once(source)) - .map(|id| Datum::UInt64(u64::cast_from(*id))), - ); let data = packer.pack_slice(&[ - addr_row.iter().next().unwrap(), - Datum::UInt64(u64::cast_from(*port)), + Datum::UInt64(u64::cast_from(*operator_id)), Datum::UInt64(u64::cast_from(worker_index)), + Datum::UInt64(u64::cast_from(*source)), + Datum::UInt64(u64::cast_from(*port)), Datum::String(update_type), Datum::from(*ts), ]); diff --git a/src/compute/src/logging/timely.rs b/src/compute/src/logging/timely.rs index f98f493af8fda..d8e0776e85a43 100644 --- a/src/compute/src/logging/timely.rs +++ b/src/compute/src/logging/timely.rs @@ -18,11 +18,12 @@ use differential_dataflow::consolidation::ConsolidatingContainerBuilder; use mz_compute_client::logging::LoggingConfig; use mz_ore::cast::CastFrom; use mz_repr::{Datum, Diff, Timestamp}; -use mz_timely_util::containers::{columnar_exchange, Col2ValBatcher, ColumnBuilder}; +use mz_timely_util::containers::{ + columnar_exchange, Col2ValBatcher, ColumnBuilder, ProvidedBuilder, +}; use mz_timely_util::replay::MzReplay; use timely::communication::Allocate; use timely::container::columnation::{Columnation, CopyRegion}; -use timely::container::CapacityContainerBuilder; use timely::dataflow::channels::pact::{ExchangeCore, Pipeline}; use timely::dataflow::channels::pushers::buffer::Session; use timely::dataflow::channels::pushers::{Counter, Tee}; @@ -61,20 +62,19 @@ pub(super) fn construct( worker.dataflow_named("Dataflow: timely logging", move |scope| { let enable_logging = config.enable_logging; let (logs, token) = - Some(event_queue.link).mz_replay::<_, CapacityContainerBuilder<_>, _>( + event_queue.links.mz_replay::<_, ProvidedBuilder<_>, _>( scope, "timely logs", config.interval, event_queue.activator, - move |mut session, data| { + move |mut session, mut data| { // If logging is disabled, we still need to install the indexes, but we can leave them // empty. We do so by immediately filtering all logs events. if enable_logging { - session.give_iterator(data.iter()) + session.give_container(data.to_mut()) } }, ); - let logs = logs.container::>(); // Build a demux operator that splits the replayed event stream up into the separate // logging streams. diff --git a/src/controller/Cargo.toml b/src/controller/Cargo.toml index b31042d9424e7..d73475b4d1d47 100644 --- a/src/controller/Cargo.toml +++ b/src/controller/Cargo.toml @@ -32,7 +32,7 @@ mz-txn-wal = { path = "../txn-wal" } regex = "1.7.0" serde = { version = "1.0.152", features = ["derive"] } serde_json = "1.0.125" -timely = "0.16.0" +timely = "0.17.0" tokio = "1.38.0" tracing = "0.1.37" uuid = { version = "1.7.0" } diff --git a/src/durable-cache/Cargo.toml b/src/durable-cache/Cargo.toml index 7904408060a62..336e15e10234c 100644 --- a/src/durable-cache/Cargo.toml +++ b/src/durable-cache/Cargo.toml @@ -12,7 +12,7 @@ workspace = true [dependencies] async-trait = "0.1.83" bytes = { version = "1.3.0" } -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" futures = "0.3.25" itertools = { version = "0.12.1" } mz-ore = { path = "../ore", features = ["process"] } @@ -23,7 +23,7 @@ mz-timely-util = { path = "../timely-util" } prometheus = { version = "0.13.3", default-features = false } prost = { version = "0.13.1", features = ["no-recursion-limit"] } serde = { version = "1.0.152", features = ["derive", "rc"] } -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", default-features = false, features = ["rt", "rt-multi-thread"] } tracing = "0.1.37" uuid = { version = "1.7.0", features = ["v4"] } diff --git a/src/environmentd/Cargo.toml b/src/environmentd/Cargo.toml index 0e1225c397095..7aa66f877e1a6 100644 --- a/src/environmentd/Cargo.toml +++ b/src/environmentd/Cargo.toml @@ -145,7 +145,7 @@ reqwest = { version = "0.11.13", features = ["blocking"] } serde_json = "1.0.125" serde_urlencoded = "0.7.1" similar-asserts = "1.4" -timely = "0.16.0" +timely = "0.17.0" tokio-postgres = { version = "0.7.8", features = ["with-chrono-0_4"] } [build-dependencies] diff --git a/src/expr/Cargo.toml b/src/expr/Cargo.toml index 4bcbea5da90de..48de6ee02ab19 100644 --- a/src/expr/Cargo.toml +++ b/src/expr/Cargo.toml @@ -59,7 +59,7 @@ serde_json = "1.0.125" sha1 = "0.10.5" sha2 = "0.10.6" subtle = "2.4.1" -timely = "0.16.0" +timely = "0.17.0" tracing = "0.1.37" uncased = "0.9.7" uuid = { version = "1.7.0", features = ["v5"] } diff --git a/src/interchange/Cargo.toml b/src/interchange/Cargo.toml index f2adb9ac94d18..f9b9580f10e55 100644 --- a/src/interchange/Cargo.toml +++ b/src/interchange/Cargo.toml @@ -20,7 +20,7 @@ byteorder = "1.4.3" bytes = "1.3.0" chrono = { version = "0.4.35", default-features = false, features = ["std"] } clap = { version = "4.5.23", features = ["derive"] } -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" itertools = "0.12.1" maplit = "1.0.2" mz-avro = { path = "../avro", features = ["snappy"] } @@ -33,7 +33,7 @@ prost = { version = "0.13.2", features = ["no-recursion-limit"] } prost-reflect = "0.14.5" seahash = "4" serde_json = "1.0.125" -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", features = ["macros", "net", "rt", "rt-multi-thread", "time"] } tracing = "0.1.37" uuid = { version = "1.7.0", features = ["serde"] } diff --git a/src/persist-cli/Cargo.toml b/src/persist-cli/Cargo.toml index 648b829feb944..6e73b01894abd 100644 --- a/src/persist-cli/Cargo.toml +++ b/src/persist-cli/Cargo.toml @@ -23,7 +23,7 @@ async-trait = "0.1.83" axum = "0.7.5" bytes = { version = "1.3.0", features = ["serde"] } clap = { version = "4.5.23", features = ["derive", "env"] } -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" futures = "0.3.25" humantime = "2.1.0" mz-http-util = { path = "../http-util" } @@ -40,7 +40,7 @@ num_enum = "0.5.7" prometheus = { version = "0.13.3", default-features = false } serde = { version = "1.0.152", features = ["derive", "rc"] } serde_json = "1.0.125" -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", default-features = false, features = ["macros", "sync", "rt", "rt-multi-thread", "time"] } tracing = "0.1.37" url = "2.3.1" diff --git a/src/persist-client/Cargo.toml b/src/persist-client/Cargo.toml index 66574c15c148c..eaf6e557c503b 100644 --- a/src/persist-client/Cargo.toml +++ b/src/persist-client/Cargo.toml @@ -35,7 +35,7 @@ async-stream = "0.3.3" async-trait = "0.1.83" bytes = { version = "1.3.0", features = ["serde"] } clap = { version = "4.5.23", features = ["derive"] } -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" futures = "0.3.25" futures-util = "0.3" h2 = "0.3.13" @@ -59,7 +59,7 @@ sentry-tracing = "0.29.1" semver = { version = "1.0.16", features = ["serde"] } serde = { version = "1.0.152", features = ["derive", "rc"] } serde_json = "1.0.125" -timely = "0.16.0" +timely = "0.17.0" thiserror = "1.0.37" tokio = { version = "1.38.0", default-features = false, features = ["macros", "sync", "rt", "rt-multi-thread", "time"] } tokio-metrics = "0.3.0" diff --git a/src/persist-types/Cargo.toml b/src/persist-types/Cargo.toml index 026d084ae9869..fb268f82042b8 100644 --- a/src/persist-types/Cargo.toml +++ b/src/persist-types/Cargo.toml @@ -26,7 +26,7 @@ proptest-derive = { version = "0.5.1", features = ["boxed_union"] } prost = { version = "0.13.2", features = ["no-recursion-limit"] } serde = { version = "1.0.152", features = ["derive"] } serde_json = { version = "1.0.125" } -timely = "0.16.0" +timely = "0.17.0" tracing = "0.1.37" uuid = { version = "1.7.0", features = ["v4"] } workspace-hack = { version = "0.0.0", path = "../workspace-hack", optional = true } diff --git a/src/persist/Cargo.toml b/src/persist/Cargo.toml index 68aca52c1ef49..39e8d8d4e20d3 100644 --- a/src/persist/Cargo.toml +++ b/src/persist/Cargo.toml @@ -37,7 +37,7 @@ azure_core = "0.21.0" base64 = "0.13.1" bytes = "1.3.0" deadpool-postgres = "0.10.3" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" fail = { version = "0.5.1", features = ["failpoints"] } futures-util = "0.3.25" md-5 = "0.10.5" @@ -58,7 +58,7 @@ proptest-derive = { version = "0.5.1", features = ["boxed_union"] } prost = { version = "0.13.2", features = ["no-recursion-limit"] } rand = { version = "0.8.5", features = ["small_rng"] } serde = { version = "1.0.152", features = ["derive"] } -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", default-features = false, features = ["fs", "macros", "sync", "rt", "rt-multi-thread"] } tokio-postgres = { version = "0.7.8" } tracing = "0.1.37" diff --git a/src/repr/Cargo.toml b/src/repr/Cargo.toml index 4a4f4ff354640..440808c72e692 100644 --- a/src/repr/Cargo.toml +++ b/src/repr/Cargo.toml @@ -36,7 +36,7 @@ columnation = "0.1.0" chrono = { version = "0.4.35", default-features = false, features = ["serde", "std"] } compact_bytes = "0.1.2" dec = "0.4.8" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" enum-kinds = "0.5.1" flatcontainer = "0.5.0" hex = "0.4.3" @@ -68,7 +68,7 @@ serde_json = { version = "1.0.125", features = ["arbitrary_precision", "preserve smallvec = { version = "1.10.0", features = ["serde", "union"] } static_assertions = "1.1" strsim = "0.11.1" -timely = "0.16.0" +timely = "0.17.0" tokio-postgres = { version = "0.7.8" } tracing-core = "0.1.30" url = { version = "2.3.1", features = ["serde"] } diff --git a/src/service/Cargo.toml b/src/service/Cargo.toml index d35313ead9677..095502c93f3d8 100644 --- a/src/service/Cargo.toml +++ b/src/service/Cargo.toml @@ -35,7 +35,7 @@ prost = { version = "0.13.2", features = ["no-recursion-limit"] } semver = "1.0.16" serde = { version = "1.0.152", features = ["derive"] } sysinfo = "0.27.2" -timely = "0.16.0" +timely = "0.17.0" tokio = "1.38.0" tokio-stream = "0.1.11" tonic = "0.12.1" diff --git a/src/storage-client/Cargo.toml b/src/storage-client/Cargo.toml index ce9e33156ffbb..25d7a30777e3a 100644 --- a/src/storage-client/Cargo.toml +++ b/src/storage-client/Cargo.toml @@ -13,7 +13,7 @@ workspace = true anyhow = "1.0.66" async-trait = "0.1.83" chrono = { version = "0.4.35", default-features = false, features = ["std"] } -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" futures = "0.3.25" http = "1.1.0" itertools = { version = "0.12.1" } @@ -46,7 +46,7 @@ serde = { version = "1.0.152", features = ["derive"] } serde_json = { version = "1.0.125" } smallvec = { version = "1.10.0", features = ["serde", "union"] } static_assertions = "1.1" -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", features = [ "fs", "rt", diff --git a/src/storage-controller/Cargo.toml b/src/storage-controller/Cargo.toml index 31a898e4838b9..73a5e83ae0e2a 100644 --- a/src/storage-controller/Cargo.toml +++ b/src/storage-controller/Cargo.toml @@ -15,7 +15,7 @@ async-trait = "0.1.83" bytes = "1.3.0" chrono = { version = "0.4.35", default-features = false, features = ["std"] } derivative = "2.2.0" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" futures = "0.3.25" itertools = { version = "0.12.1" } mz-build-info = { path = "../build-info" } @@ -38,7 +38,7 @@ proptest = { version = "1.6.0", default-features = false, features = ["std"] } prost = { version = "0.13.2", features = ["no-recursion-limit"] } serde = { version = "1.0.152", features = ["derive"] } serde_json = { version = "1.0.125" } -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", features = ["fs", "rt", "sync", "test-util", "time"] } tokio-postgres = { version = "0.7.8", features = ["serde"] } tokio-stream = "0.1.11" diff --git a/src/storage-operators/Cargo.toml b/src/storage-operators/Cargo.toml index 1bf2cd8c0b811..9dcc08ab9c819 100644 --- a/src/storage-operators/Cargo.toml +++ b/src/storage-operators/Cargo.toml @@ -17,7 +17,7 @@ async-stream = "0.3.3" aws-types = "1.1.1" bytes = "1.3.0" bytesize = "1.1.0" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" csv-async = { version = "1.3.0", features = ["tokio"] } futures = "0.3.25" http = "1.1.0" @@ -41,7 +41,7 @@ reqwest = { version = "0.11.13", features = ["stream"] } sentry = { version = "0.29.1" } serde = { version = "1.0.152", features = ["derive"] } smallvec = { version = "1.10.0", features = ["union"] } -timely = "0.16.0" +timely = "0.17.0" thiserror = "1.0.37" tokio = { version = "1.38.0", features = ["fs", "rt", "sync", "test-util", "time"] } tokio-stream = "0.1.11" diff --git a/src/storage-types/Cargo.toml b/src/storage-types/Cargo.toml index d04a4341742af..ab09f4d1be935 100644 --- a/src/storage-types/Cargo.toml +++ b/src/storage-types/Cargo.toml @@ -25,7 +25,7 @@ bytes = "1.3.0" columnation = "0.1.0" dec = "0.4.8" derivative = "2.2.0" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" hex = "0.4.3" http = "1.1.0" itertools = { version = "0.12.1" } @@ -62,7 +62,7 @@ regex = "1.7.0" serde = { version = "1.0.152", features = ["derive"] } serde_json = { version = "1.0.125", features = ["preserve_order"] } thiserror = "1.0.37" -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", features = ["fs", "rt", "sync", "test-util", "time"] } tokio-postgres = { version = "0.7.8", features = ["serde"] } tracing = "0.1.37" diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index a586bd888889a..0bd2af8dacca5 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -26,7 +26,7 @@ columnation = "0.1.0" crossbeam-channel = "0.5.8" csv-core = { version = "0.1.10" } dec = "0.4.8" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" fail = { version = "0.5.1", features = ["failpoints"] } futures = "0.3.25" indexmap = { version = "2.0.0", default-features = false, features = ["std"] } @@ -77,7 +77,7 @@ serde = { version = "1.0.152", features = ["derive"] } serde_json = { version = "1.0.125" } serde_bytes = { version = "0.11.14" } sha2 = "0.10.6" -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", features = ["fs", "rt", "sync", "test-util"] } tokio-postgres = { version = "0.7.8", features = ["serde"] } tokio-stream = "0.1.11" diff --git a/src/timely-util/Cargo.toml b/src/timely-util/Cargo.toml index 9d895e52c755b..88901d8931402 100644 --- a/src/timely-util/Cargo.toml +++ b/src/timely-util/Cargo.toml @@ -15,7 +15,7 @@ bincode = "1.3.3" bytemuck = "1.21.0" columnar = "0.2.2" columnation = "0.1.0" -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" either = "1" futures-util = "0.3.25" lgalloc = "0.4" @@ -23,7 +23,7 @@ mz-ore = { path = "../ore", features = ["async", "process", "tracing_", "test"] num-traits = "0.2" proptest = { version = "1.6.0", default-features = false, features = ["std"] } serde = { version = "1.0.152", features = ["derive"] } -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", features = ["macros", "rt-multi-thread", "time"] } tracing = "0.1.37" uuid = { version = "1.7.0", features = ["serde", "v4"] } diff --git a/src/timely-util/src/replay.rs b/src/timely-util/src/replay.rs index d7cb296d1f568..f0fe73ff4ba4e 100644 --- a/src/timely-util/src/replay.rs +++ b/src/timely-util/src/replay.rs @@ -13,6 +13,7 @@ //! provides the protocol and semantics of the [MzReplay] operator. use std::any::Any; +use std::borrow::Cow; use std::rc::Rc; use std::time::{Duration, Instant}; use timely::container::ContainerBuilder; @@ -56,14 +57,14 @@ where ) -> (StreamCore, Rc) where CB: ContainerBuilder, - L: FnMut(Session>>, &C) + L: FnMut(Session>>, Cow) + 'static; } impl MzReplay for I where T: Timestamp, - C: Container, + C: Container + Clone, I: IntoIterator, I::Item: EventIterator + 'static, A: ActivatorTrait + 'static, @@ -78,7 +79,7 @@ where ) -> (StreamCore, Rc) where for<'a> CB: ContainerBuilder, - L: FnMut(Session>>, &C) + L: FnMut(Session>>, Cow) + 'static, { let name = format!("Replay {}", name); @@ -135,13 +136,21 @@ where if weak_token.upgrade().is_some() { for event_stream in event_streams.iter_mut() { while let Some(event) = event_stream.next() { - match &event { - Event::Progress(vec) => { + use Cow::*; + match event { + Owned(Event::Progress(vec)) => { + progress.internals[0].extend(vec.iter().cloned()); + progress_sofar.extend(vec.into_iter()); + } + Owned(Event::Messages(time, data)) => { + logic(output.session_with_builder(&time), Owned(data)); + } + Borrowed(Event::Progress(vec)) => { progress.internals[0].extend(vec.iter().cloned()); progress_sofar.extend(vec.iter().cloned()); } - Event::Messages(time, data) => { - logic(output.session_with_builder(time), data); + Borrowed(Event::Messages(time, data)) => { + logic(output.session_with_builder(time), Borrowed(data)); } } } diff --git a/src/transform/Cargo.toml b/src/transform/Cargo.toml index 980a526f9e0ba..2718a2b83b3c8 100644 --- a/src/transform/Cargo.toml +++ b/src/transform/Cargo.toml @@ -10,7 +10,7 @@ publish = false workspace = true [dependencies] -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" enum-kinds = "0.5.1" itertools = "0.12.1" mz-compute-types = { path = "../compute-types" } diff --git a/src/txn-wal/Cargo.toml b/src/txn-wal/Cargo.toml index e1fce89097ebe..6e7e2bdff4a10 100644 --- a/src/txn-wal/Cargo.toml +++ b/src/txn-wal/Cargo.toml @@ -12,7 +12,7 @@ workspace = true [dependencies] async-trait = "0.1.83" bytes = { version = "1.3.0" } -differential-dataflow = "0.13.3" +differential-dataflow = "0.13.4" futures = "0.3.25" itertools = { version = "0.12.1" } mz-ore = { path = "../ore", features = ["process"] } @@ -23,7 +23,7 @@ mz-timely-util = { path = "../timely-util" } prometheus = { version = "0.13.3", default-features = false } prost = { version = "0.13.2", features = ["no-recursion-limit"] } serde = { version = "1.0.152", features = ["derive", "rc"] } -timely = "0.16.0" +timely = "0.17.0" tokio = { version = "1.38.0", default-features = false, features = ["rt", "rt-multi-thread"] } tracing = "0.1.37" uuid = { version = "1.7.0", features = ["v4"] } diff --git a/test/sqllogictest/cluster.slt b/test/sqllogictest/cluster.slt index c0976f74e90e8..ca0cb75c4e692 100644 --- a/test/sqllogictest/cluster.slt +++ b/test/sqllogictest/cluster.slt @@ -233,11 +233,12 @@ bar mz_dataflow_addresses_per_worker mz_dataflow_addresses_per_worker_u7_prima bar mz_dataflow_addresses_per_worker mz_dataflow_addresses_per_worker_u7_primary_idx 2 worker_id NULL false bar mz_dataflow_channels_per_worker mz_dataflow_channels_per_worker_u7_primary_idx 1 id NULL false bar mz_dataflow_channels_per_worker mz_dataflow_channels_per_worker_u7_primary_idx 2 worker_id NULL false -bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 1 address NULL false -bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 2 port NULL false -bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 3 worker_id NULL false -bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 4 update_type NULL false -bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 5 time NULL true +bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 1 id NULL false +bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 2 worker_id NULL false +bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 3 source NULL false +bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 4 port NULL false +bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 5 update_type NULL false +bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 6 time NULL true bar mz_dataflow_operators_per_worker mz_dataflow_operators_per_worker_u7_primary_idx 1 id NULL false bar mz_dataflow_operators_per_worker mz_dataflow_operators_per_worker_u7_primary_idx 2 worker_id NULL false bar mz_dataflow_shutdown_durations_histogram_raw mz_dataflow_shutdown_durations_histogram_raw_u7_primary_idx 1 worker_id NULL false diff --git a/test/sqllogictest/mz_catalog_server_index_accounting.slt b/test/sqllogictest/mz_catalog_server_index_accounting.slt index 5202fc259d718..74bc88616e793 100644 --- a/test/sqllogictest/mz_catalog_server_index_accounting.slt +++ b/test/sqllogictest/mz_catalog_server_index_accounting.slt @@ -64,7 +64,7 @@ mz_continual_tasks_ind CREATE␠INDEX␠"mz_continual_tasks_ind"␠IN␠CLUSTER mz_databases_ind CREATE␠INDEX␠"mz_databases_ind"␠IN␠CLUSTER␠[s2]␠ON␠[s457␠AS␠"mz_catalog"."mz_databases"]␠("name") mz_dataflow_addresses_per_worker_s2_primary_idx CREATE␠INDEX␠"mz_dataflow_addresses_per_worker_s2_primary_idx"␠IN␠CLUSTER␠[s2]␠ON␠"mz_introspection"."mz_dataflow_addresses_per_worker"␠("id",␠"worker_id") mz_dataflow_channels_per_worker_s2_primary_idx CREATE␠INDEX␠"mz_dataflow_channels_per_worker_s2_primary_idx"␠IN␠CLUSTER␠[s2]␠ON␠"mz_introspection"."mz_dataflow_channels_per_worker"␠("id",␠"worker_id") -mz_dataflow_operator_reachability_raw_s2_primary_idx CREATE␠INDEX␠"mz_dataflow_operator_reachability_raw_s2_primary_idx"␠IN␠CLUSTER␠[s2]␠ON␠"mz_introspection"."mz_dataflow_operator_reachability_raw"␠("address",␠"port",␠"worker_id",␠"update_type",␠"time") +mz_dataflow_operator_reachability_raw_s2_primary_idx CREATE␠INDEX␠"mz_dataflow_operator_reachability_raw_s2_primary_idx"␠IN␠CLUSTER␠[s2]␠ON␠"mz_introspection"."mz_dataflow_operator_reachability_raw"␠("id",␠"worker_id",␠"source",␠"port",␠"update_type",␠"time") mz_dataflow_operators_per_worker_s2_primary_idx CREATE␠INDEX␠"mz_dataflow_operators_per_worker_s2_primary_idx"␠IN␠CLUSTER␠[s2]␠ON␠"mz_introspection"."mz_dataflow_operators_per_worker"␠("id",␠"worker_id") mz_dataflow_shutdown_durations_histogram_raw_s2_primary_idx CREATE␠INDEX␠"mz_dataflow_shutdown_durations_histogram_raw_s2_primary_idx"␠IN␠CLUSTER␠[s2]␠ON␠"mz_introspection"."mz_dataflow_shutdown_durations_histogram_raw"␠("worker_id",␠"duration_ns") mz_frontiers_ind CREATE␠INDEX␠"mz_frontiers_ind"␠IN␠CLUSTER␠[s2]␠ON␠[s696␠AS␠"mz_internal"."mz_frontiers"]␠("object_id") @@ -369,8 +369,9 @@ mz_dataflow_channels_per_worker id mz_dataflow_channels_per_worker to_index mz_dataflow_channels_per_worker to_port mz_dataflow_channels_per_worker worker_id -mz_dataflow_operator_reachability_raw address +mz_dataflow_operator_reachability_raw id mz_dataflow_operator_reachability_raw port +mz_dataflow_operator_reachability_raw source mz_dataflow_operator_reachability_raw time mz_dataflow_operator_reachability_raw update_type mz_dataflow_operator_reachability_raw worker_id diff --git a/test/testdrive-old-kafka-src-syntax/indexes.td b/test/testdrive-old-kafka-src-syntax/indexes.td index bcd037d2b552b..afe535df1b2a9 100644 --- a/test/testdrive-old-kafka-src-syntax/indexes.td +++ b/test/testdrive-old-kafka-src-syntax/indexes.td @@ -318,7 +318,7 @@ mz_continual_tasks_ind mz_continual_tasks mz_databases_ind mz_databases mz_catalog_server {name} "" mz_dataflow_addresses_per_worker_s2_primary_idx mz_dataflow_addresses_per_worker mz_catalog_server {id,worker_id} "" mz_dataflow_channels_per_worker_s2_primary_idx mz_dataflow_channels_per_worker mz_catalog_server {id,worker_id} "" -mz_dataflow_operator_reachability_raw_s2_primary_idx mz_dataflow_operator_reachability_raw mz_catalog_server {address,port,worker_id,update_type,time} "" +mz_dataflow_operator_reachability_raw_s2_primary_idx mz_dataflow_operator_reachability_raw mz_catalog_server {id,worker_id,source,port,update_type,time} "" mz_dataflow_operators_per_worker_s2_primary_idx mz_dataflow_operators_per_worker mz_catalog_server {id,worker_id} "" mz_dataflow_shutdown_durations_histogram_raw_s2_primary_idx mz_dataflow_shutdown_durations_histogram_raw mz_catalog_server {worker_id,duration_ns} "" mz_frontiers_ind mz_frontiers mz_catalog_server {object_id} "" diff --git a/test/testdrive/indexes.td b/test/testdrive/indexes.td index 45eb7f7eb6959..a2c3af8e7d24f 100644 --- a/test/testdrive/indexes.td +++ b/test/testdrive/indexes.td @@ -335,7 +335,7 @@ mz_continual_tasks_ind mz_continual_tasks mz_databases_ind mz_databases mz_catalog_server {name} "" mz_dataflow_addresses_per_worker_s2_primary_idx mz_dataflow_addresses_per_worker mz_catalog_server {id,worker_id} "" mz_dataflow_channels_per_worker_s2_primary_idx mz_dataflow_channels_per_worker mz_catalog_server {id,worker_id} "" -mz_dataflow_operator_reachability_raw_s2_primary_idx mz_dataflow_operator_reachability_raw mz_catalog_server {address,port,worker_id,update_type,time} "" +mz_dataflow_operator_reachability_raw_s2_primary_idx mz_dataflow_operator_reachability_raw mz_catalog_server {id,worker_id,source,port,update_type,time} "" mz_dataflow_operators_per_worker_s2_primary_idx mz_dataflow_operators_per_worker mz_catalog_server {id,worker_id} "" mz_dataflow_shutdown_durations_histogram_raw_s2_primary_idx mz_dataflow_shutdown_durations_histogram_raw mz_catalog_server {worker_id,duration_ns} "" mz_frontiers_ind mz_frontiers mz_catalog_server {object_id} "" diff --git a/test/testdrive/logging.td b/test/testdrive/logging.td index 7512451095722..b3d9f2949b213 100644 --- a/test/testdrive/logging.td +++ b/test/testdrive/logging.td @@ -156,9 +156,9 @@ SID batch_received 7 bigint ORDER BY position id name position type -------------------------------------- -SID address 1 list -SID port 2 uint8 -SID worker_id 3 uint8 +SID id 1 uint8 +SID worker_id 2 uint8 +SID port 3 uint8 SID update_type 4 text SID time 5 mz_timestamp SID count 6 bigint From d356bf2cf0693ae3efb5ae10d387dd1dad5e1ec4 Mon Sep 17 00:00:00 2001 From: Jan Teske Date: Wed, 29 Jan 2025 15:35:15 +0100 Subject: [PATCH 08/12] *: update cmake to 0.15.3 To pull in https://github.com/rust-lang/cmake-rs/pull/229, which caused a major regression in Mz build times. --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 21a1db322eb3f..3f2d3e3346df6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1741,9 +1741,9 @@ checksum = "f46ad14479a25103f283c0f10005961cf086d8dc42205bb44c46ac563475dca6" [[package]] name = "cmake" -version = "0.1.50" +version = "0.1.53" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a31c789563b815f77f4250caee12365734369f942439b7defd71e18a48197130" +checksum = "e24a03c8b52922d68a1589ad61032f2c1aa5a8158d2aa0d93c6e9534944bbad6" dependencies = [ "cc", ] From 8e5ad9d8e0059428477a932cd1f1cb6cd9a3e823 Mon Sep 17 00:00:00 2001 From: Joseph Koshakow Date: Wed, 29 Jan 2025 10:17:15 -0500 Subject: [PATCH 09/12] adapter: Remove legacy builtin item migrations (#31218) Previously, the adapter maintained two different implementations of the builtin item migration framework. One that was used before 0dt and one that was used with 0dt. The 0dt implementation still works when 0dt is turned off, but we wanted to keep the legacy implementation around while the 0dt implementation was still new. The 0dt implementation has been around long enough in production to prove that it works. On the other hand, the legacy implementation has not been tested in production in a long time. This commit removes the legacy implementation and always uses the new implementation. This should reduce the maintenance burden of builtin item migrations. In the future, the builtin item migration framework should be completely removed and replaced with `ALTER TABLE`. --- src/adapter/src/catalog.rs | 135 +- src/adapter/src/catalog/open.rs | 1117 +---------------- .../catalog/open/builtin_item_migration.rs | 66 +- src/adapter/src/coord.rs | 10 +- src/catalog-debug/src/main.rs | 11 +- src/catalog/src/config.rs | 9 +- src/compute-client/src/as_of_selection.rs | 1 - src/storage-client/src/storage_collections.rs | 17 +- 8 files changed, 54 insertions(+), 1312 deletions(-) diff --git a/src/adapter/src/catalog.rs b/src/adapter/src/catalog.rs index 9aa37bc400501..e317ad29f5893 100644 --- a/src/adapter/src/catalog.rs +++ b/src/adapter/src/catalog.rs @@ -85,9 +85,7 @@ use uuid::Uuid; // DO NOT add any more imports from `crate` outside of `crate::catalog`. pub use crate::catalog::builtin_table_updates::BuiltinTableUpdate; -pub use crate::catalog::open::{ - BuiltinMigrationMetadata, InitializeStateResult, OpenCatalogResult, -}; +pub use crate::catalog::open::{InitializeStateResult, OpenCatalogResult}; pub use crate::catalog::state::CatalogState; pub use crate::catalog::transact::{ DropObjectInfo, Op, ReplicaCreateDropReason, TransactionResult, @@ -671,9 +669,9 @@ impl Catalog { // debugging/testing. let previous_ts = now().into(); let replica_size = &bootstrap_args.default_cluster_replica_size; + let read_only = false; let OpenCatalogResult { catalog, - storage_collections_to_drop: _, migrated_storage_collections_0dt: _, new_builtin_collections: _, builtin_table_updates: _, @@ -687,7 +685,7 @@ impl Catalog { all_features: false, build_info: &DUMMY_BUILD_INFO, environment_id: environment_id.unwrap_or(EnvironmentId::for_tests()), - read_only: false, + read_only, now, boot_ts: previous_ts, skip_migrations: true, @@ -705,7 +703,10 @@ impl Catalog { aws_privatelink_availability_zones: None, http_host_name: None, connection_context: ConnectionContext::for_tests(secrets_reader), - builtin_item_migration_config: BuiltinItemMigrationConfig::Legacy, + builtin_item_migration_config: BuiltinItemMigrationConfig { + persist_client: persist_client.clone(), + read_only, + }, persist_client, enable_expression_cache_override, enable_0dt_deployment: true, @@ -2267,10 +2268,7 @@ mod tests { use tokio_postgres::NoTls; use uuid::Uuid; - use mz_catalog::builtin::{ - Builtin, BuiltinType, UnsafeBuiltinTableFingerprintWhitespace, BUILTINS, - UNSAFE_DO_NOT_CALL_THIS_IN_PRODUCTION_BUILTIN_TABLE_FINGERPRINT_WHITESPACE, - }; + use mz_catalog::builtin::{Builtin, BuiltinType, BUILTINS}; use mz_catalog::durable::{test_bootstrap_args, CatalogError, DurableCatalogError, FenceError}; use mz_catalog::SYSTEM_CONN_ID; use mz_controller_types::{ClusterId, ReplicaId}; @@ -2285,9 +2283,7 @@ mod tests { CatalogItemId, Datum, GlobalId, RelationType, RelationVersionSelector, RowArena, ScalarType, Timestamp, }; - use mz_sql::catalog::{ - BuiltinsConfig, CatalogDatabase, CatalogSchema, CatalogType, SessionCatalog, - }; + use mz_sql::catalog::{BuiltinsConfig, CatalogSchema, CatalogType, SessionCatalog}; use mz_sql::func::{Func, FuncImpl, Operation, OP_IMPLS}; use mz_sql::names::{ self, DatabaseId, ItemQualifiers, ObjectId, PartialItemName, QualifiedItemName, @@ -3507,119 +3503,6 @@ mod tests { .await } - #[mz_ore::test(tokio::test)] - #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_client_method` on OS `linux` - async fn test_builtin_migrations() { - let persist_client = PersistClient::new_for_tests().await; - let bootstrap_args = test_bootstrap_args(); - let organization_id = Uuid::new_v4(); - let mv_name = "mv"; - let (mz_tables_id, mv_id) = { - let mut catalog = Catalog::open_debug_catalog( - persist_client.clone(), - organization_id.clone(), - &bootstrap_args, - ) - .await - .expect("unable to open debug catalog"); - - // Create a materialized view over `mz_tables`. - let database_id = DatabaseId::User(1); - let database = catalog.get_database(&database_id); - let database_name = database.name(); - let schemas = database.schemas(); - let schema = schemas.first().expect("must have at least one schema"); - let schema_spec = schema.id().clone(); - let schema_name = &schema.name().schema; - let database_spec = ResolvedDatabaseSpecifier::Id(database_id); - let id_ts = catalog.storage().await.current_upper().await; - let (mv_id, mv_gid) = catalog - .allocate_user_id(id_ts) - .await - .expect("unable to allocate id"); - let mv = catalog - .state() - .deserialize_item( - mv_gid, - &format!("CREATE MATERIALIZED VIEW {database_name}.{schema_name}.{mv_name} AS SELECT name FROM mz_tables"), - &BTreeMap::new(), - &mut LocalExpressionCache::Closed, - None, - ) - .expect("unable to deserialize item"); - let commit_ts = catalog.current_upper().await; - catalog - .transact( - None, - commit_ts, - None, - vec![Op::CreateItem { - id: mv_id, - name: QualifiedItemName { - qualifiers: ItemQualifiers { - database_spec, - schema_spec, - }, - item: mv_name.to_string(), - }, - item: mv, - owner_id: MZ_SYSTEM_ROLE_ID, - }], - ) - .await - .expect("unable to transact"); - - let mz_tables_id = catalog - .entries() - .find(|entry| &entry.name.item == "mz_tables" && entry.is_table()) - .expect("mz_tables doesn't exist") - .id(); - let check_mv_id = catalog - .entries() - .find(|entry| &entry.name.item == mv_name && entry.is_materialized_view()) - .unwrap_or_else(|| panic!("{mv_name} doesn't exist")) - .id(); - assert_eq!(check_mv_id, mv_id); - catalog.expire().await; - (mz_tables_id, mv_id) - }; - // Forcibly migrate all tables. - { - let mut guard = - UNSAFE_DO_NOT_CALL_THIS_IN_PRODUCTION_BUILTIN_TABLE_FINGERPRINT_WHITESPACE - .lock() - .expect("lock poisoned"); - *guard = Some(( - UnsafeBuiltinTableFingerprintWhitespace::All, - "\n".to_string(), - )); - } - { - let catalog = - Catalog::open_debug_catalog(persist_client, organization_id, &bootstrap_args) - .await - .expect("unable to open debug catalog"); - - let new_mz_tables_id = catalog - .entries() - .find(|entry| &entry.name.item == "mz_tables" && entry.is_table()) - .expect("mz_tables doesn't exist") - .id(); - // Assert that the table was migrated and got a new ID. - assert_ne!(new_mz_tables_id, mz_tables_id); - - let new_mv_id = catalog - .entries() - .find(|entry| &entry.name.item == mv_name && entry.is_materialized_view()) - .unwrap_or_else(|| panic!("{mv_name} doesn't exist")) - .id(); - // Assert that the materialized view was migrated and got a new ID. - assert_ne!(new_mv_id, mv_id); - - catalog.expire().await; - } - } - #[mz_ore::test(tokio::test)] #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_client_method` on OS `linux` async fn test_multi_subscriber_catalog() { diff --git a/src/adapter/src/catalog/open.rs b/src/adapter/src/catalog/open.rs index 1b66b119edd7a..ac200eba0a54f 100644 --- a/src/adapter/src/catalog/open.rs +++ b/src/adapter/src/catalog/open.rs @@ -17,12 +17,10 @@ use std::time::{Duration, Instant}; use futures::future::{BoxFuture, FutureExt}; use itertools::{Either, Itertools}; -use mz_adapter_types::compaction::CompactionWindow; use mz_adapter_types::dyncfgs::{ENABLE_CONTINUAL_TASK_BUILTINS, ENABLE_EXPRESSION_CACHE}; use mz_catalog::builtin::{ - Builtin, BuiltinTable, Fingerprint, BUILTINS, BUILTIN_CLUSTERS, BUILTIN_CLUSTER_REPLICAS, - BUILTIN_PREFIXES, BUILTIN_ROLES, MZ_STORAGE_USAGE_BY_SHARD_DESCRIPTION, - RUNTIME_ALTERABLE_FINGERPRINT_SENTINEL, + Builtin, Fingerprint, BUILTINS, BUILTIN_CLUSTERS, BUILTIN_CLUSTER_REPLICAS, BUILTIN_PREFIXES, + BUILTIN_ROLES, MZ_STORAGE_USAGE_BY_SHARD_DESCRIPTION, RUNTIME_ALTERABLE_FINGERPRINT_SENTINEL, }; use mz_catalog::config::{ClusterReplicaSizeMap, StateConfig}; use mz_catalog::durable::objects::{ @@ -34,35 +32,30 @@ use mz_catalog::expr_cache::{ }; use mz_catalog::memory::error::{Error, ErrorKind}; use mz_catalog::memory::objects::{ - BootstrapStateUpdateKind, CatalogEntry, CatalogItem, CommentsMap, DefaultPrivileges, - StateUpdate, + BootstrapStateUpdateKind, CommentsMap, DefaultPrivileges, StateUpdate, }; use mz_catalog::SYSTEM_CONN_ID; -use mz_compute_client::logging::LogVariant; use mz_controller::clusters::{ReplicaAllocation, ReplicaLogging}; use mz_controller_types::ClusterId; use mz_ore::cast::usize_to_u64; -use mz_ore::collections::{CollectionExt, HashSet}; +use mz_ore::collections::HashSet; use mz_ore::now::to_datetime; use mz_ore::{instrument, soft_assert_no_log}; use mz_repr::adt::mz_acl_item::PrivilegeMap; use mz_repr::namespaces::is_unstable_schema; -use mz_repr::role_id::RoleId; -use mz_repr::{CatalogItemId, Diff, GlobalId, RelationVersion, Timestamp}; +use mz_repr::{CatalogItemId, Diff, GlobalId, Timestamp}; use mz_sql::catalog::{ - BuiltinsConfig, CatalogError as SqlCatalogError, CatalogItem as SqlCatalogItem, - CatalogItemType, RoleMembership, RoleVars, + BuiltinsConfig, CatalogError as SqlCatalogError, CatalogItemType, RoleMembership, RoleVars, }; use mz_sql::func::OP_IMPLS; -use mz_sql::names::SchemaId; use mz_sql::rbac; use mz_sql::session::user::{MZ_SYSTEM_ROLE_ID, SYSTEM_USER}; use mz_sql::session::vars::{SessionVars, SystemVars, VarError, VarInput}; -use mz_sql_parser::ast::display::AstDisplay; use mz_storage_client::storage_collections::StorageCollections; use timely::Container; -use tracing::{error, info, warn, Instrument}; +use tracing::{info, warn, Instrument}; use uuid::Uuid; + // DO NOT add any more imports from `crate` outside of `crate::catalog`. use crate::catalog::open::builtin_item_migration::{ migrate_builtin_items, BuiltinItemMigrationResult, @@ -73,113 +66,9 @@ use crate::catalog::{ }; use crate::AdapterError; -#[derive(Debug)] -pub struct BuiltinMigrationMetadata { - /// Used to drop objects on STORAGE nodes. - /// - /// Note: These collections are only known by the storage controller, and not the - /// Catalog, thus we identify them by their [`GlobalId`]. - pub previous_storage_collection_ids: BTreeSet, - // Used to update persisted on disk catalog state - pub migrated_system_object_mappings: BTreeMap, - pub introspection_source_index_updates: - BTreeMap>, - pub user_item_drop_ops: Vec, - pub user_item_create_ops: Vec, -} - -#[derive(Debug)] -pub struct CreateOp { - id: CatalogItemId, - oid: u32, - global_id: GlobalId, - schema_id: SchemaId, - name: String, - owner_id: RoleId, - privileges: PrivilegeMap, - item_rebuilder: CatalogItemRebuilder, -} - -impl BuiltinMigrationMetadata { - fn new() -> BuiltinMigrationMetadata { - BuiltinMigrationMetadata { - previous_storage_collection_ids: BTreeSet::new(), - migrated_system_object_mappings: BTreeMap::new(), - introspection_source_index_updates: BTreeMap::new(), - user_item_drop_ops: Vec::new(), - user_item_create_ops: Vec::new(), - } - } -} - -#[derive(Debug)] -pub enum CatalogItemRebuilder { - SystemSource(CatalogItem), - Object { - sql: String, - is_retained_metrics_object: bool, - custom_logical_compaction_window: Option, - }, -} - -impl CatalogItemRebuilder { - fn new( - entry: &CatalogEntry, - id: CatalogItemId, - ancestor_ids: &BTreeMap, - ) -> Self { - if id.is_system() - && (entry.is_table() || entry.is_introspection_source() || entry.is_source()) - { - Self::SystemSource(entry.item().clone()) - } else { - let create_sql = entry.create_sql().to_string(); - let mut create_stmt = mz_sql::parse::parse(&create_sql) - .expect("invalid create sql persisted to catalog") - .into_element() - .ast; - mz_sql::ast::transform::create_stmt_replace_ids(&mut create_stmt, ancestor_ids); - Self::Object { - sql: create_stmt.to_ast_string_stable(), - is_retained_metrics_object: entry.item().is_retained_metrics_object(), - custom_logical_compaction_window: entry.item().custom_logical_compaction_window(), - } - } - } - - fn build( - self, - global_id: GlobalId, - state: &CatalogState, - versions: &BTreeMap, - ) -> CatalogItem { - match self { - Self::SystemSource(item) => item, - Self::Object { - sql, - is_retained_metrics_object, - custom_logical_compaction_window, - } => state - .parse_item( - global_id, - &sql, - versions, - None, - is_retained_metrics_object, - custom_logical_compaction_window, - &mut LocalExpressionCache::Closed, - None, - ) - .unwrap_or_else(|error| panic!("invalid persisted create sql ({error:?}): {sql}")), - } - } -} - pub struct InitializeStateResult { /// An initialized [`CatalogState`]. pub state: CatalogState, - /// A set of storage collections to drop (only used by legacy migrations). - pub storage_collections_to_drop: BTreeSet, /// A set of new shards that may need to be initialized (only used by 0dt migration). pub migrated_storage_collections_0dt: BTreeSet, /// A set of new builtin items. @@ -199,12 +88,7 @@ pub struct InitializeStateResult { pub struct OpenCatalogResult { /// An opened [`Catalog`]. pub catalog: Catalog, - /// A set of storage collections to drop (only used by legacy migrations). - /// - /// Note: These Collections will not be in the Catalog, and are only known about by - /// the storage controller, which is why we identify them by [`GlobalId`]. - pub storage_collections_to_drop: BTreeSet, - /// A set of new shards that may need to be initialized (only used by 0dt migration). + /// A set of new shards that may need to be initialized. pub migrated_storage_collections_0dt: BTreeSet, /// A set of new builtin items. pub new_builtin_collections: BTreeSet, @@ -540,7 +424,6 @@ impl Catalog { // Migrate builtin items. let BuiltinItemMigrationResult { builtin_table_updates: builtin_table_update, - storage_collections_to_drop, migrated_storage_collections_0dt, cleanup_action, } = migrate_builtin_items( @@ -560,7 +443,6 @@ impl Catalog { Ok(InitializeStateResult { state, - storage_collections_to_drop, migrated_storage_collections_0dt, new_builtin_collections: new_builtin_collections.into_iter().collect(), builtin_table_updates, @@ -588,7 +470,6 @@ impl Catalog { let InitializeStateResult { state, - storage_collections_to_drop, migrated_storage_collections_0dt, new_builtin_collections, mut builtin_table_updates, @@ -640,7 +521,6 @@ impl Catalog { Ok(OpenCatalogResult { catalog, - storage_collections_to_drop, migrated_storage_collections_0dt, new_builtin_collections, builtin_table_updates, @@ -663,7 +543,6 @@ impl Catalog { storage_collections: &Arc< dyn StorageCollections + Send + Sync, >, - storage_collections_to_drop: BTreeSet, ) -> Result<(), mz_catalog::durable::CatalogError> { let collections = self .entries() @@ -679,7 +558,7 @@ impl Catalog { let mut txn = storage.transaction().await?; storage_collections - .initialize_state(&mut txn, collections, storage_collections_to_drop) + .initialize_state(&mut txn, collections) .await .map_err(mz_catalog::durable::DurableCatalogError::from)?; @@ -702,7 +581,6 @@ impl Catalog { config: mz_controller::ControllerConfig, envd_epoch: core::num::NonZeroI64, read_only: bool, - storage_collections_to_drop: BTreeSet, ) -> Result, mz_catalog::durable::CatalogError> { let controller_start = Instant::now(); @@ -726,7 +604,7 @@ impl Catalog { mz_controller::Controller::new(config, envd_epoch, read_only, &read_only_tx).await }; - self.initialize_storage_state(&controller.storage_collections, storage_collections_to_drop) + self.initialize_storage_state(&controller.storage_collections) .await?; info!( @@ -737,277 +615,6 @@ impl Catalog { Ok(controller) } - /// The objects in the catalog form one or more DAGs (directed acyclic graph) via object - /// dependencies. To migrate a builtin object we must drop that object along with all of its - /// descendants, and then recreate that object along with all of its descendants using new - /// [`CatalogItemId`]s. To achieve this we perform a DFS (depth first search) on the catalog - /// items starting with the nodes that correspond to builtin objects that have changed schemas. - /// - /// Objects need to be dropped starting from the leafs of the DAG going up towards the roots, - /// and they need to be recreated starting at the roots of the DAG and going towards the leafs. - fn generate_builtin_migration_metadata( - state: &CatalogState, - txn: &mut Transaction<'_>, - migrated_ids: Vec, - id_fingerprint_map: BTreeMap, - ) -> Result { - // First obtain a topological sorting of all migrated objects and their children. - let mut visited_set = BTreeSet::new(); - let mut sorted_entries = Vec::new(); - for item_id in migrated_ids { - if !visited_set.contains(&item_id) { - let migrated_topological_sort = - Catalog::topological_sort(state, item_id, &mut visited_set); - sorted_entries.extend(migrated_topological_sort); - } - } - sorted_entries.reverse(); - - // Then process all objects in sorted order. - let mut migration_metadata = BuiltinMigrationMetadata::new(); - let mut ancestor_ids = BTreeMap::new(); - let mut migrated_log_ids = BTreeMap::new(); - let log_name_map: BTreeMap<_, _> = BUILTINS::logs() - .map(|log| (log.variant.clone(), log.name)) - .collect(); - for entry in sorted_entries { - let id = entry.id(); - - let (new_item_id, new_global_id) = match id { - CatalogItemId::System(_) => txn.allocate_system_item_ids(1)?.into_element(), - CatalogItemId::IntrospectionSourceIndex(id) => ( - CatalogItemId::IntrospectionSourceIndex(id), - GlobalId::IntrospectionSourceIndex(id), - ), - CatalogItemId::User(_) => txn.allocate_user_item_ids(1)?.into_element(), - _ => unreachable!("can't migrate id: {id}"), - }; - - let name = state.resolve_full_name(entry.name(), None); - info!("migrating {name} from {id} to {new_item_id}"); - - // Generate value to update fingerprint and global ID persisted mapping for system objects. - // Not every system object has a fingerprint, like introspection source indexes. - if let Some(fingerprint) = id_fingerprint_map.get(&id) { - assert!( - id.is_system(), - "id_fingerprint_map should only contain builtin objects" - ); - let schema_name = state - .get_schema( - &entry.name().qualifiers.database_spec, - &entry.name().qualifiers.schema_spec, - entry.conn_id().unwrap_or(&SYSTEM_CONN_ID), - ) - .name - .schema - .as_str(); - migration_metadata.migrated_system_object_mappings.insert( - id, - SystemObjectMapping { - description: SystemObjectDescription { - schema_name: schema_name.to_string(), - object_type: entry.item_type(), - object_name: entry.name().item.clone(), - }, - unique_identifier: SystemObjectUniqueIdentifier { - catalog_id: new_item_id, - global_id: new_global_id, - fingerprint: fingerprint.clone(), - }, - }, - ); - } - - ancestor_ids.insert(id, new_item_id); - - if entry.item().is_storage_collection() { - migration_metadata - .previous_storage_collection_ids - .extend(entry.global_ids()); - } - - // Push drop commands. - match entry.item() { - CatalogItem::Log(log) => { - migrated_log_ids.insert(log.global_id(), log.variant.clone()); - } - CatalogItem::Index(index) => { - if id.is_system() { - if let Some(variant) = migrated_log_ids.get(&index.on) { - migration_metadata - .introspection_source_index_updates - .entry(index.cluster_id) - .or_default() - .push(( - variant.clone(), - log_name_map - .get(variant) - .expect("all variants have a name") - .to_string(), - new_item_id, - new_global_id, - entry.oid(), - )); - } - } - } - CatalogItem::Table(_) - | CatalogItem::Source(_) - | CatalogItem::MaterializedView(_) - | CatalogItem::ContinualTask(_) => { - // Storage objects don't have any external objects to drop. - } - CatalogItem::Sink(_) => { - // Sinks don't have any external objects to drop--however, - // this would change if we add a collections for sinks - // database-issues#5148. - } - CatalogItem::View(_) => { - // Views don't have any external objects to drop. - } - CatalogItem::Type(_) - | CatalogItem::Func(_) - | CatalogItem::Secret(_) - | CatalogItem::Connection(_) => unreachable!( - "impossible to migrate schema for builtin {}", - entry.item().typ() - ), - } - if id.is_user() { - migration_metadata.user_item_drop_ops.push(id); - } - - // Push create commands. - let name = entry.name().clone(); - if id.is_user() { - let schema_id = name.qualifiers.schema_spec.clone().into(); - let item_rebuilder = CatalogItemRebuilder::new(entry, new_item_id, &ancestor_ids); - migration_metadata.user_item_create_ops.push(CreateOp { - id: new_item_id, - oid: entry.oid(), - global_id: new_global_id, - schema_id, - name: name.item.clone(), - owner_id: entry.owner_id().clone(), - privileges: entry.privileges().clone(), - item_rebuilder, - }); - } - } - - // Reverse drop commands. - migration_metadata.user_item_drop_ops.reverse(); - - Ok(migration_metadata) - } - - fn topological_sort<'a, 'b>( - state: &'a CatalogState, - id: CatalogItemId, - visited_set: &'b mut BTreeSet, - ) -> Vec<&'a CatalogEntry> { - let mut sorted_entries = Vec::new(); - visited_set.insert(id); - let entry = state.get_entry(&id); - for dependant in entry.used_by() { - if !visited_set.contains(dependant) { - let child_topological_sort = - Catalog::topological_sort(state, *dependant, visited_set); - sorted_entries.extend(child_topological_sort); - } - } - sorted_entries.push(entry); - sorted_entries - } - - #[mz_ore::instrument] - async fn apply_builtin_migration( - state: &mut CatalogState, - txn: &mut Transaction<'_>, - migration_metadata: &mut BuiltinMigrationMetadata, - ) -> Result>, Error> { - for id in &migration_metadata.user_item_drop_ops { - let entry = state.get_entry(id); - if entry.is_sink() { - let full_name = state.resolve_full_name(entry.name(), None); - error!( - "user sink {full_name} will be recreated as part of a builtin migration which \ - can result in duplicate data being emitted. This is a known issue, \ - https://github.com/MaterializeInc/database-issues/issues/5553. Please inform the \ - customer that their sink may produce duplicate data." - ) - } - } - - let mut builtin_table_updates = Vec::new(); - txn.remove_items(&migration_metadata.user_item_drop_ops.drain(..).collect())?; - txn.update_system_object_mappings(std::mem::take( - &mut migration_metadata.migrated_system_object_mappings, - ))?; - txn.update_introspection_source_index_gids( - std::mem::take(&mut migration_metadata.introspection_source_index_updates) - .into_iter() - .map(|(cluster_id, updates)| { - ( - cluster_id, - updates - .into_iter() - .map(|(_variant, name, item_id, index_id, oid)| { - (name, item_id, index_id, oid) - }), - ) - }), - )?; - let updates = txn.get_and_commit_op_updates(); - let builtin_table_update = state - .apply_updates_for_bootstrap(updates, &mut LocalExpressionCache::Closed) - .await; - builtin_table_updates.extend(builtin_table_update); - for CreateOp { - id, - oid, - global_id, - schema_id, - name, - owner_id, - privileges, - item_rebuilder, - } in migration_metadata.user_item_create_ops.drain(..) - { - // Builtin Items can't be versioned. - let versions = BTreeMap::new(); - let item = item_rebuilder.build(global_id, state, &versions); - let (create_sql, expect_gid, expect_versions) = item.to_serialized(); - assert_eq!( - global_id, expect_gid, - "serializing a CatalogItem changed the GlobalId" - ); - assert_eq!( - versions, expect_versions, - "serializing a CatalogItem changed the Versions" - ); - - txn.insert_item( - id, - oid, - global_id, - schema_id, - &name, - create_sql, - owner_id.clone(), - privileges.all_values_owned().collect(), - versions, - )?; - let updates = txn.get_and_commit_op_updates(); - let builtin_table_update = state - .apply_updates_for_bootstrap(updates, &mut LocalExpressionCache::Closed) - .await; - builtin_table_updates.extend(builtin_table_update); - } - Ok(builtin_table_updates) - } - /// Politely releases all external resources that can only be released in an async context. pub async fn expire(self) { // If no one else holds a reference to storage, then clean up the storage resources. @@ -1570,705 +1177,3 @@ fn get_dyncfg_val_from_defaults_and_remote( } val } - -#[cfg(test)] -mod builtin_migration_tests { - use std::collections::{BTreeMap, BTreeSet}; - - use itertools::Itertools; - use mz_catalog::memory::objects::{ - CatalogItem, Index, MaterializedView, Table, TableDataSource, - }; - use mz_catalog::SYSTEM_CONN_ID; - use mz_controller_types::ClusterId; - use mz_expr::MirRelationExpr; - use mz_ore::id_gen::Gen; - use mz_repr::{ - CatalogItemId, GlobalId, RelationDesc, RelationType, RelationVersion, ScalarType, - VersionedRelationDesc, - }; - use mz_sql::catalog::CatalogDatabase; - use mz_sql::names::{ - DependencyIds, ItemQualifiers, QualifiedItemName, ResolvedDatabaseSpecifier, ResolvedIds, - }; - use mz_sql::session::user::MZ_SYSTEM_ROLE_ID; - use mz_sql::DEFAULT_SCHEMA; - use mz_sql_parser::ast::Expr; - - use crate::catalog::{Catalog, Op, OptimizedMirRelationExpr}; - use crate::session::DEFAULT_DATABASE_NAME; - - enum ItemNamespace { - System, - User, - } - - enum SimplifiedItem { - Table, - MaterializedView { referenced_names: Vec }, - Index { on: String }, - } - - struct SimplifiedCatalogEntry { - name: String, - namespace: ItemNamespace, - item: SimplifiedItem, - } - - impl SimplifiedCatalogEntry { - // A lot of the fields here aren't actually used in the test so we can fill them in with dummy - // values. - fn to_catalog_item( - self, - item_id_mapping: &BTreeMap, - global_id_mapping: &BTreeMap, - global_id_gen: &mut Gen, - ) -> (String, ItemNamespace, CatalogItem, GlobalId) { - let global_id = GlobalId::User(global_id_gen.allocate_id()); - let desc = RelationDesc::builder() - .with_column("a", ScalarType::Int32.nullable(true)) - .with_key(vec![0]) - .finish(); - let item = match self.item { - SimplifiedItem::Table => CatalogItem::Table(Table { - create_sql: Some("CREATE TABLE materialize.public.t (a INT)".to_string()), - desc: VersionedRelationDesc::new(desc), - collections: [(RelationVersion::root(), global_id)].into_iter().collect(), - conn_id: None, - resolved_ids: ResolvedIds::empty(), - custom_logical_compaction_window: None, - is_retained_metrics_object: false, - data_source: TableDataSource::TableWrites { - defaults: vec![Expr::null(); 1], - }, - }), - SimplifiedItem::MaterializedView { referenced_names } => { - let table_list = referenced_names - .iter() - .map(|table| format!("materialize.public.{table}")) - .join(","); - let column_list = referenced_names - .iter() - .enumerate() - .map(|(idx, _)| format!("a{idx}")) - .join(","); - let resolved_ids = - convert_names_to_ids(referenced_names, item_id_mapping, global_id_mapping); - - CatalogItem::MaterializedView(MaterializedView { - global_id, - create_sql: format!( - "CREATE MATERIALIZED VIEW materialize.public.mv ({column_list}) AS SELECT * FROM {table_list}" - ), - raw_expr: mz_sql::plan::HirRelationExpr::constant( - Vec::new(), - RelationType { - column_types: Vec::new(), - keys: Vec::new(), - }, - ).into(), - dependencies: DependencyIds(Default::default()), - optimized_expr: OptimizedMirRelationExpr(MirRelationExpr::Constant { - rows: Ok(Vec::new()), - typ: RelationType { - column_types: Vec::new(), - keys: Vec::new(), - }, - }).into(), - desc: RelationDesc::builder() - .with_column("a", ScalarType::Int32.nullable(true)) - .with_key(vec![0]) - .finish(), - resolved_ids: resolved_ids.into_iter().collect(), - cluster_id: ClusterId::user(1).expect("1 is a valid ID"), - non_null_assertions: vec![], - custom_logical_compaction_window: None, - refresh_schedule: None, - initial_as_of: None, - }) - } - SimplifiedItem::Index { on } => { - let on_item_id = item_id_mapping[&on]; - let on_gid = global_id_mapping[&on]; - CatalogItem::Index(Index { - create_sql: format!("CREATE INDEX idx ON materialize.public.{on} (a)"), - global_id, - on: on_gid, - keys: Default::default(), - conn_id: None, - resolved_ids: [(on_item_id, on_gid)].into_iter().collect(), - cluster_id: ClusterId::user(1).expect("1 is a valid ID"), - custom_logical_compaction_window: None, - is_retained_metrics_object: false, - }) - } - }; - (self.name, self.namespace, item, global_id) - } - } - - struct BuiltinMigrationTestCase { - test_name: &'static str, - initial_state: Vec, - migrated_names: Vec, - expected_previous_storage_collection_names: Vec, - expected_migrated_system_object_mappings: Vec, - expected_user_item_drop_ops: Vec, - expected_user_item_create_ops: Vec, - } - - async fn add_item( - catalog: &mut Catalog, - name: String, - item: CatalogItem, - item_namespace: ItemNamespace, - ) -> CatalogItemId { - let id_ts = catalog.storage().await.current_upper().await; - let (item_id, _) = match item_namespace { - ItemNamespace::User => catalog - .allocate_user_id(id_ts) - .await - .expect("cannot fail to allocate user ids"), - ItemNamespace::System => catalog - .allocate_system_id(id_ts) - .await - .expect("cannot fail to allocate system ids"), - }; - let database_id = catalog - .resolve_database(DEFAULT_DATABASE_NAME) - .expect("failed to resolve default database") - .id(); - let database_spec = ResolvedDatabaseSpecifier::Id(database_id); - let schema_spec = catalog - .resolve_schema_in_database(&database_spec, DEFAULT_SCHEMA, &SYSTEM_CONN_ID) - .expect("failed to resolve default schemas") - .id - .clone(); - - let commit_ts = catalog.storage().await.current_upper().await; - catalog - .transact( - None, - commit_ts, - None, - vec![Op::CreateItem { - id: item_id, - name: QualifiedItemName { - qualifiers: ItemQualifiers { - database_spec, - schema_spec, - }, - item: name, - }, - item, - owner_id: MZ_SYSTEM_ROLE_ID, - }], - ) - .await - .expect("failed to transact"); - - item_id - } - - fn convert_names_to_ids( - name_vec: Vec, - item_id_lookup: &BTreeMap, - global_id_lookup: &BTreeMap, - ) -> BTreeMap { - name_vec - .into_iter() - .map(|name| { - let item_id = item_id_lookup[&name]; - let global_id = global_id_lookup[&name]; - (item_id, global_id) - }) - .collect() - } - - fn convert_ids_to_names>( - ids: I, - name_lookup: &BTreeMap, - ) -> BTreeSet { - ids.into_iter().map(|id| name_lookup[&id].clone()).collect() - } - - fn convert_global_ids_to_names>( - ids: I, - global_id_lookup: &BTreeMap, - ) -> BTreeSet { - ids.into_iter() - .flat_map(|id_a| { - global_id_lookup - .iter() - .filter_map(move |(name, id_b)| (id_a == *id_b).then_some(name)) - }) - .cloned() - .collect() - } - - async fn run_test_case(test_case: BuiltinMigrationTestCase) { - Catalog::with_debug_in_bootstrap(|mut catalog| async move { - let mut item_id_mapping = BTreeMap::new(); - let mut name_mapping = BTreeMap::new(); - - let mut global_id_gen = Gen::::default(); - let mut global_id_mapping = BTreeMap::new(); - - for entry in test_case.initial_state { - let (name, namespace, item, global_id) = - entry.to_catalog_item(&item_id_mapping, &global_id_mapping, &mut global_id_gen); - let item_id = add_item(&mut catalog, name.clone(), item, namespace).await; - - item_id_mapping.insert(name.clone(), item_id); - global_id_mapping.insert(name.clone(), global_id); - name_mapping.insert(item_id, name); - } - - let migrated_ids = test_case - .migrated_names - .into_iter() - .map(|name| item_id_mapping[&name]) - .collect(); - let id_fingerprint_map: BTreeMap = item_id_mapping - .iter() - .filter(|(_name, id)| id.is_system()) - // We don't use the new fingerprint in this test, so we can just hard code it - .map(|(_name, id)| (*id, "".to_string())) - .collect(); - - let migration_metadata = { - // This cloning is a hacky way to appease the borrow checker. It doesn't really - // matter because we never look at catalog again. We could probably rewrite this - // test to not even need a `Catalog` which would significantly speed it up. - let state = catalog.state.clone(); - let mut storage = catalog.storage().await; - let mut txn = storage - .transaction() - .await - .expect("failed to create transaction"); - Catalog::generate_builtin_migration_metadata( - &state, - &mut txn, - migrated_ids, - id_fingerprint_map, - ) - .expect("failed to generate builtin migration metadata") - }; - - assert_eq!( - convert_global_ids_to_names( - migration_metadata - .previous_storage_collection_ids - .into_iter(), - &global_id_mapping - ), - test_case - .expected_previous_storage_collection_names - .into_iter() - .collect(), - "{} test failed with wrong previous collection_names", - test_case.test_name - ); - assert_eq!( - migration_metadata - .migrated_system_object_mappings - .values() - .map(|mapping| mapping.description.object_name.clone()) - .collect::>(), - test_case - .expected_migrated_system_object_mappings - .into_iter() - .collect(), - "{} test failed with wrong migrated system object mappings", - test_case.test_name - ); - assert_eq!( - convert_ids_to_names( - migration_metadata.user_item_drop_ops.into_iter(), - &name_mapping - ), - test_case.expected_user_item_drop_ops.into_iter().collect(), - "{} test failed with wrong user drop ops", - test_case.test_name - ); - assert_eq!( - migration_metadata - .user_item_create_ops - .into_iter() - .map(|create_op| create_op.name) - .collect::>(), - test_case - .expected_user_item_create_ops - .into_iter() - .collect(), - "{} test failed with wrong user create ops", - test_case.test_name - ); - catalog.expire().await; - }) - .await - } - - #[mz_ore::test(tokio::test)] - #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_client_method` on OS `linux` - async fn test_builtin_migration_no_migrations() { - let test_case = BuiltinMigrationTestCase { - test_name: "no_migrations", - initial_state: vec![SimplifiedCatalogEntry { - name: "s1".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::Table, - }], - migrated_names: vec![], - expected_previous_storage_collection_names: vec![], - expected_migrated_system_object_mappings: vec![], - expected_user_item_drop_ops: vec![], - expected_user_item_create_ops: vec![], - }; - run_test_case(test_case).await; - } - - #[mz_ore::test(tokio::test)] - #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_client_method` on OS `linux` - async fn test_builtin_migration_single_migrations() { - let test_case = BuiltinMigrationTestCase { - test_name: "single_migrations", - initial_state: vec![SimplifiedCatalogEntry { - name: "s1".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::Table, - }], - migrated_names: vec!["s1".to_string()], - expected_previous_storage_collection_names: vec!["s1".to_string()], - expected_migrated_system_object_mappings: vec!["s1".to_string()], - expected_user_item_drop_ops: vec![], - expected_user_item_create_ops: vec![], - }; - run_test_case(test_case).await; - } - - #[mz_ore::test(tokio::test)] - #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_client_method` on OS `linux` - async fn test_builtin_migration_child_migrations() { - let test_case = BuiltinMigrationTestCase { - test_name: "child_migrations", - initial_state: vec![ - SimplifiedCatalogEntry { - name: "s1".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::Table, - }, - SimplifiedCatalogEntry { - name: "u1".to_string(), - namespace: ItemNamespace::User, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s1".to_string()], - }, - }, - ], - migrated_names: vec!["s1".to_string()], - expected_previous_storage_collection_names: vec!["u1".to_string(), "s1".to_string()], - expected_migrated_system_object_mappings: vec!["s1".to_string()], - expected_user_item_drop_ops: vec!["u1".to_string()], - expected_user_item_create_ops: vec!["u1".to_string()], - }; - run_test_case(test_case).await; - } - - #[mz_ore::test(tokio::test)] - #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_client_method` on OS `linux` - async fn test_builtin_migration_multi_child_migrations() { - let test_case = BuiltinMigrationTestCase { - test_name: "multi_child_migrations", - initial_state: vec![ - SimplifiedCatalogEntry { - name: "s1".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::Table, - }, - SimplifiedCatalogEntry { - name: "u1".to_string(), - namespace: ItemNamespace::User, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s1".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "u2".to_string(), - namespace: ItemNamespace::User, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s1".to_string()], - }, - }, - ], - migrated_names: vec!["s1".to_string()], - expected_previous_storage_collection_names: vec![ - "u1".to_string(), - "u2".to_string(), - "s1".to_string(), - ], - expected_migrated_system_object_mappings: vec!["s1".to_string()], - expected_user_item_drop_ops: vec!["u1".to_string(), "u2".to_string()], - expected_user_item_create_ops: vec!["u2".to_string(), "u1".to_string()], - }; - run_test_case(test_case).await; - } - - #[mz_ore::test(tokio::test)] - #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_client_method` on OS `linux` - async fn test_builtin_migration_topological_sort() { - let test_case = BuiltinMigrationTestCase { - test_name: "topological_sort", - initial_state: vec![ - SimplifiedCatalogEntry { - name: "s1".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::Table, - }, - SimplifiedCatalogEntry { - name: "s2".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::Table, - }, - SimplifiedCatalogEntry { - name: "u1".to_string(), - namespace: ItemNamespace::User, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s2".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "u2".to_string(), - namespace: ItemNamespace::User, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s1".to_string(), "u1".to_string()], - }, - }, - ], - migrated_names: vec!["s1".to_string(), "s2".to_string()], - expected_previous_storage_collection_names: vec![ - "u2".to_string(), - "u1".to_string(), - "s1".to_string(), - "s2".to_string(), - ], - expected_migrated_system_object_mappings: vec!["s1".to_string(), "s2".to_string()], - expected_user_item_drop_ops: vec!["u2".to_string(), "u1".to_string()], - expected_user_item_create_ops: vec!["u1".to_string(), "u2".to_string()], - }; - run_test_case(test_case).await; - } - - #[mz_ore::test(tokio::test)] - #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_client_method` on OS `linux` - async fn test_builtin_migration_topological_sort_complex() { - let test_case = BuiltinMigrationTestCase { - test_name: "topological_sort_complex", - initial_state: vec![ - SimplifiedCatalogEntry { - name: "s273".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::Table, - }, - SimplifiedCatalogEntry { - name: "s322".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::Table, - }, - SimplifiedCatalogEntry { - name: "s317".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::Table, - }, - SimplifiedCatalogEntry { - name: "s349".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s273".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s421".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s273".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s295".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s273".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s296".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s295".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s320".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s295".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s340".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s295".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s318".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s295".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s323".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s295".to_string(), "s322".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s330".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s318".to_string(), "s317".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s321".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s318".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s315".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s296".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s354".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s296".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s327".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s296".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s339".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s296".to_string()], - }, - }, - SimplifiedCatalogEntry { - name: "s355".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::MaterializedView { - referenced_names: vec!["s315".to_string()], - }, - }, - ], - migrated_names: vec![ - "s273".to_string(), - "s317".to_string(), - "s318".to_string(), - "s320".to_string(), - "s321".to_string(), - "s322".to_string(), - "s323".to_string(), - "s330".to_string(), - "s339".to_string(), - "s340".to_string(), - ], - expected_previous_storage_collection_names: vec![ - "s349".to_string(), - "s421".to_string(), - "s355".to_string(), - "s315".to_string(), - "s354".to_string(), - "s327".to_string(), - "s339".to_string(), - "s296".to_string(), - "s320".to_string(), - "s340".to_string(), - "s330".to_string(), - "s321".to_string(), - "s318".to_string(), - "s323".to_string(), - "s295".to_string(), - "s273".to_string(), - "s317".to_string(), - "s322".to_string(), - ], - expected_migrated_system_object_mappings: vec![ - "s322".to_string(), - "s317".to_string(), - "s273".to_string(), - "s295".to_string(), - "s323".to_string(), - "s318".to_string(), - "s321".to_string(), - "s330".to_string(), - "s340".to_string(), - "s320".to_string(), - "s296".to_string(), - "s339".to_string(), - "s327".to_string(), - "s354".to_string(), - "s315".to_string(), - "s355".to_string(), - "s421".to_string(), - "s349".to_string(), - ], - expected_user_item_drop_ops: vec![], - expected_user_item_create_ops: vec![], - }; - run_test_case(test_case).await; - } - - #[mz_ore::test(tokio::test)] - #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_client_method` on OS `linux` - async fn test_builtin_migration_system_child_migrations() { - let test_case = BuiltinMigrationTestCase { - test_name: "system_child_migrations", - initial_state: vec![ - SimplifiedCatalogEntry { - name: "s1".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::Table, - }, - SimplifiedCatalogEntry { - name: "s2".to_string(), - namespace: ItemNamespace::System, - item: SimplifiedItem::Index { - on: "s1".to_string(), - }, - }, - ], - migrated_names: vec!["s1".to_string()], - expected_previous_storage_collection_names: vec!["s1".to_string()], - expected_migrated_system_object_mappings: vec!["s1".to_string(), "s2".to_string()], - expected_user_item_drop_ops: vec![], - expected_user_item_create_ops: vec![], - }; - run_test_case(test_case).await; - } -} diff --git a/src/adapter/src/catalog/open/builtin_item_migration.rs b/src/adapter/src/catalog/open/builtin_item_migration.rs index 077ffb1da93aa..50273314d7639 100644 --- a/src/adapter/src/catalog/open/builtin_item_migration.rs +++ b/src/adapter/src/catalog/open/builtin_item_migration.rs @@ -40,15 +40,13 @@ use tracing::{debug, error}; use crate::catalog::open::builtin_item_migration::persist_schema::{TableKey, TableKeySchema}; use crate::catalog::state::LocalExpressionCache; -use crate::catalog::{BuiltinTableUpdate, Catalog, CatalogState}; +use crate::catalog::{BuiltinTableUpdate, CatalogState}; /// The results of a builtin item migration. pub(crate) struct BuiltinItemMigrationResult { /// A vec of updates to apply to the builtin tables. pub(crate) builtin_table_updates: Vec>, - /// A set of storage collections to drop (only used by legacy migration). - pub(crate) storage_collections_to_drop: BTreeSet, - /// A set of new shards that may need to be initialized (only used by 0dt migration). + /// A set of new shards that may need to be initialized. pub(crate) migrated_storage_collections_0dt: BTreeSet, /// Some cleanup action to take once the migration has been made durable. pub(crate) cleanup_action: BoxFuture<'static, ()>, @@ -60,59 +58,20 @@ pub(crate) async fn migrate_builtin_items( txn: &mut Transaction<'_>, local_expr_cache: &mut LocalExpressionCache, migrated_builtins: Vec, - config: BuiltinItemMigrationConfig, -) -> Result { - match config { - BuiltinItemMigrationConfig::Legacy => { - migrate_builtin_items_legacy(state, txn, migrated_builtins).await - } - BuiltinItemMigrationConfig::ZeroDownTime { - persist_client, - read_only, - } => { - migrate_builtin_items_0dt( - state, - txn, - local_expr_cache, - persist_client, - migrated_builtins, - read_only, - ) - .await - } - } -} - -/// The legacy method for builtin migrations is to drop all migrated items and all of their -/// dependents and re-create them all with the new schema and new global IDs. -async fn migrate_builtin_items_legacy( - state: &mut CatalogState, - txn: &mut Transaction<'_>, - migrated_builtins: Vec, + BuiltinItemMigrationConfig { + persist_client, + read_only, + }: BuiltinItemMigrationConfig, ) -> Result { - let id_fingerprint_map: BTreeMap<_, _> = BUILTINS::iter(&state.config().builtins_cfg) - .map(|builtin| { - let id = state.resolve_builtin_object(builtin); - let fingerprint = builtin.fingerprint(); - (id, fingerprint) - }) - .collect(); - let mut builtin_migration_metadata = Catalog::generate_builtin_migration_metadata( + migrate_builtin_items_0dt( state, txn, + local_expr_cache, + persist_client, migrated_builtins, - id_fingerprint_map, - )?; - let builtin_table_updates = - Catalog::apply_builtin_migration(state, txn, &mut builtin_migration_metadata).await?; - - let cleanup_action = async {}.boxed(); - Ok(BuiltinItemMigrationResult { - builtin_table_updates, - storage_collections_to_drop: builtin_migration_metadata.previous_storage_collection_ids, - migrated_storage_collections_0dt: BTreeSet::new(), - cleanup_action, - }) + read_only, + ) + .await } /// An implementation of builtin item migrations that is compatible with zero down-time upgrades. @@ -472,7 +431,6 @@ async fn migrate_builtin_items_0dt( Ok(BuiltinItemMigrationResult { builtin_table_updates, - storage_collections_to_drop: BTreeSet::new(), migrated_storage_collections_0dt, cleanup_action, }) diff --git a/src/adapter/src/coord.rs b/src/adapter/src/coord.rs index 0bd74636268a3..a7d19ab15870f 100644 --- a/src/adapter/src/coord.rs +++ b/src/adapter/src/coord.rs @@ -3983,17 +3983,14 @@ pub fn serve( .open(controller_config.persist_location.clone()) .await .context("opening persist client")?; - let builtin_item_migration_config = if enable_0dt_deployment { - BuiltinItemMigrationConfig::ZeroDownTime { + let builtin_item_migration_config = + BuiltinItemMigrationConfig { persist_client: persist_client.clone(), read_only: read_only_controllers, } - } else { - BuiltinItemMigrationConfig::Legacy - }; + ; let OpenCatalogResult { mut catalog, - storage_collections_to_drop, migrated_storage_collections_0dt, new_builtin_collections, builtin_table_updates, @@ -4158,7 +4155,6 @@ pub fn serve( controller_config, controller_envd_epoch, read_only_controllers, - storage_collections_to_drop, ) }) .unwrap_or_terminate("failed to initialize storage_controller"); diff --git a/src/catalog-debug/src/main.rs b/src/catalog-debug/src/main.rs index 0e04188e1ba9b..aaba9b8f7a365 100644 --- a/src/catalog-debug/src/main.rs +++ b/src/catalog-debug/src/main.rs @@ -566,12 +566,12 @@ async fn upgrade_check( .clone(); let boot_ts = now().into(); + let read_only = true; // BOXED FUTURE: As of Nov 2023 the returned Future from this function was 7.5KB. This would // get stored on the stack which is bad for runtime performance, and blow up our stack usage. // Because of that we purposefully move this Future onto the heap (i.e. Box it). let InitializeStateResult { state, - storage_collections_to_drop: _, migrated_storage_collections_0dt: _, new_builtin_collections: _, builtin_table_updates: _, @@ -585,7 +585,7 @@ async fn upgrade_check( all_features: false, build_info: &BUILD_INFO, environment_id: args.environment_id.clone(), - read_only: true, + read_only, now, boot_ts, skip_migrations: false, @@ -610,7 +610,12 @@ async fn upgrade_check( secrets_reader, None, ), - builtin_item_migration_config: BuiltinItemMigrationConfig::Legacy, + builtin_item_migration_config: BuiltinItemMigrationConfig { + // We don't actually want to write anything down, so use an in-memory persist + // client. + persist_client: PersistClient::new_for_tests().await, + read_only, + }, persist_client: persist_client.clone(), enable_expression_cache_override: None, enable_0dt_deployment: true, diff --git a/src/catalog/src/config.rs b/src/catalog/src/config.rs index f4a0acb81ae10..e5324b9176335 100644 --- a/src/catalog/src/config.rs +++ b/src/catalog/src/config.rs @@ -94,12 +94,9 @@ pub struct StateConfig { } #[derive(Debug)] -pub enum BuiltinItemMigrationConfig { - Legacy, - ZeroDownTime { - persist_client: PersistClient, - read_only: bool, - }, +pub struct BuiltinItemMigrationConfig { + pub persist_client: PersistClient, + pub read_only: bool, } #[derive(Debug, Clone, Deserialize, Serialize)] diff --git a/src/compute-client/src/as_of_selection.rs b/src/compute-client/src/as_of_selection.rs index 154db123b74d2..faf132518f351 100644 --- a/src/compute-client/src/as_of_selection.rs +++ b/src/compute-client/src/as_of_selection.rs @@ -812,7 +812,6 @@ mod tests { &self, _txn: &mut (dyn StorageTxn + Send), _init_ids: BTreeSet, - _drop_ids: BTreeSet, ) -> Result<(), StorageError> { unimplemented!() } diff --git a/src/storage-client/src/storage_collections.rs b/src/storage-client/src/storage_collections.rs index 9f0460965fc14..a9115fd7668ad 100644 --- a/src/storage-client/src/storage_collections.rs +++ b/src/storage-client/src/storage_collections.rs @@ -93,14 +93,10 @@ pub trait StorageCollections: Debug { /// We get `init_ids`, which tells us about all collections that currently /// exist, so that we can record durable state for those that _we_ don't /// know yet about. - /// - /// We also get `drop_ids`, which tells us about all collections that we - /// might have known about before and have now been dropped. async fn initialize_state( &self, txn: &mut (dyn StorageTxn + Send), init_ids: BTreeSet, - drop_ids: BTreeSet, ) -> Result<(), StorageError>; /// Update storage configuration with new parameters. @@ -1318,7 +1314,6 @@ where &self, txn: &mut (dyn StorageTxn + Send), init_ids: BTreeSet, - drop_ids: BTreeSet, ) -> Result<(), StorageError> { let metadata = txn.get_collection_metadata(); let existing_metadata: BTreeSet<_> = metadata.into_iter().map(|(id, _)| id).collect(); @@ -1327,12 +1322,16 @@ where let new_collections: BTreeSet = init_ids.difference(&existing_metadata).cloned().collect(); - self.prepare_state(txn, new_collections, drop_ids, BTreeMap::default()) - .await?; + self.prepare_state( + txn, + new_collections, + BTreeSet::default(), + BTreeMap::default(), + ) + .await?; // All shards that belong to collections dropped in the last epoch are - // eligible for finalization. This intentionally includes any built-in - // collections present in `drop_ids`. + // eligible for finalization. // // n.b. this introduces an unlikely race condition: if a collection is // dropped from the catalog, but the dataflow is still running on a From 0753ce30b271415bb52dd2889b75d8dd1eb72e1d Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Wed, 29 Jan 2025 10:19:07 -0500 Subject: [PATCH 10/12] CODEOWNERS: remove self from ownership Given that my last day will be 1/31. --- .github/CODEOWNERS | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 38089d97fe704..30054a8ba3fb7 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -14,7 +14,7 @@ /bin /bin/lint-versions @MaterializeInc/testing /ci @MaterializeInc/testing -/ci/test/lint-deps.toml @danhhz @benesch +/ci/test/lint-deps.toml /doc/user @MaterializeInc/docs /doc/developer/reference/compute @MaterializeInc/cluster /doc/developer/reference/storage @MaterializeInc/cluster @@ -46,7 +46,7 @@ /src/adapter/src/coord/introspection.rs @MaterializeInc/cluster # to track changes to feature flags /src/adapter/src/flags.rs @MaterializeInc/testing -/src/alloc @benesch +/src/alloc /src/audit-log @MaterializeInc/adapter /src/avro @MaterializeInc/cluster /src/avro-derive @MaterializeInc/cluster @@ -65,8 +65,8 @@ /src/controller @MaterializeInc/cluster /src/controller-types @MaterializeInc/cluster /src/durable-cache @MaterializeInc/adapter -/src/dyncfg @danhhz -/src/dyncfgs @danhhz +/src/dyncfg +/src/dyncfgs /src/environmentd @MaterializeInc/adapter /src/expr @MaterializeInc/cluster /src/expr-parser @MaterializeInc/cluster @@ -78,15 +78,15 @@ /src/kafka-util @MaterializeInc/cluster /src/lowertest @MaterializeInc/cluster /src/lowertest-derive @MaterializeInc/cluster -/src/metabase @benesch +/src/metabase /src/mysql-util @MaterializeInc/cluster /src/mz @MaterializeInc/integrations -/src/npm @benesch +/src/npm /src/orchestrator @MaterializeInc/cloud /src/orchestrator-kubernetes @MaterializeInc/cloud /src/orchestrator-process @MaterializeInc/cloud /src/orchestrator-tracing @MaterializeInc/cloud -/src/ore @benesch +/src/ore /src/persist @MaterializeInc/persist /src/persist-cli @MaterializeInc/persist /src/persist-client @MaterializeInc/persist @@ -107,7 +107,7 @@ /src/rocksdb @MaterializeInc/cluster /src/s3-datagen @MaterializeInc/cluster /src/secrets @MaterializeInc/cloud -/src/segment @benesch +/src/segment /src/service @MaterializeInc/cluster /src/sql @MaterializeInc/adapter /src/sql/src/session/vars.rs @@ -132,7 +132,7 @@ /src/testdrive @MaterializeInc/testing /src/timely-util @MaterializeInc/cluster /src/transform @MaterializeInc/cluster -/src/txn-wal @aljoscha @danhhz @jkosh44 +/src/txn-wal @aljoscha /src/walkabout @MaterializeInc/adapter /src/workspace-hack @MaterializeInc/testing /test From fec6eb16c13752055f2d4cdfd4555dee78acaf68 Mon Sep 17 00:00:00 2001 From: Parker Timmerman Date: Wed, 29 Jan 2025 10:56:42 -0500 Subject: [PATCH 11/12] [build] Get the `npm` crate working with Bazel (#31220) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR fixes our flamegraph rendering with our builtin memory and CPU profiling. Previously the vendored JS and CSS artifacts were not getting properly included into our build because the Cargo Build script that is responsible for downloading them gets run in a different sandbox than the action that builds the library. Now the `npm::ensure(...)` call optionally takes an `OUT_DIR` param that it will copy the artifacts into and thus they will get included as part of the Bazel build. I manually built the `materialized` Docker image with Bazel and confirmed the flamegraphs worked! ### Motivation Fix our builtin memory and CPU profiling ### Tips for reviewer ### Checklist - [ ] This PR has adequate test coverage / QA involvement has been duly considered. ([trigger-ci for additional test/nightly runs](https://trigger-ci.dev.materialize.com/)) - [ ] This PR has an associated up-to-date [design doc](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/design/README.md), is a design doc ([template](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/design/00000000_template.md)), or is sufficiently small to not require a design. - [ ] If this PR evolves [an existing `$T ⇔ Proto$T` mapping](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/command-and-response-binary-encoding.md) (possibly in a backwards-incompatible way), then it is tagged with a `T-proto` label. - [ ] If this PR will require changes to cloud orchestration or tests, there is a companion cloud PR to account for those changes that is tagged with the release-blocker label ([example](https://github.com/MaterializeInc/cloud/pull/5021)). - [ ] If this PR includes major [user-facing behavior changes](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/guide-changes.md#what-changes-require-a-release-note), I have pinged the relevant PM to schedule a changelog post. --- src/environmentd/build.rs | 3 ++- src/environmentd/src/http/root.rs | 7 ++++--- src/http-util/src/lib.rs | 16 +++++++++++++--- src/npm/src/lib.rs | 21 +++++++++++++++++---- src/prof-http/build.rs | 3 ++- src/prof-http/src/lib.rs | 7 ++++--- 6 files changed, 42 insertions(+), 15 deletions(-) diff --git a/src/environmentd/build.rs b/src/environmentd/build.rs index cda91dbc28b2b..961fe1962c955 100644 --- a/src/environmentd/build.rs +++ b/src/environmentd/build.rs @@ -16,5 +16,6 @@ fn main() -> Result<(), anyhow::Error> { .file("src/environmentd/sys.c") .compile("environmentd_sys"); - mz_npm::ensure() + let out_dir = std::env::var("OUT_DIR").ok().map(std::path::PathBuf::from); + mz_npm::ensure(out_dir) } diff --git a/src/environmentd/src/http/root.rs b/src/environmentd/src/http/root.rs index 6b29fe3da7204..aa7530c75a12b 100644 --- a/src/environmentd/src/http/root.rs +++ b/src/environmentd/src/http/root.rs @@ -31,7 +31,8 @@ pub async fn handle_home(profiling: bool) -> impl IntoResponse { } mz_http_util::make_handle_static!( - include_dir::include_dir!("$CARGO_MANIFEST_DIR/src/http/static"), - "src/http/static", - "src/http/static-dev" + dir_1: ::include_dir::include_dir!("$CARGO_MANIFEST_DIR/src/http/static"), + dir_2: ::include_dir::include_dir!("$OUT_DIR/src/http/static"), + prod_base_path: "src/http/static", + dev_base_path: "src/http/static-dev", ); diff --git a/src/http-util/src/lib.rs b/src/http-util/src/lib.rs index 6aadc9e644674..18c27324fa5b5 100644 --- a/src/http-util/src/lib.rs +++ b/src/http-util/src/lib.rs @@ -38,17 +38,27 @@ where /// and two strings representing the (crate-local) paths to the production and development /// static files. macro_rules! make_handle_static { - ($static_dir:expr, $prod_base_path:expr, $dev_base_path:expr) => { + ( + dir_1: $dir_1:expr, + $(dir_2: $dir_2:expr,)? + prod_base_path: $prod_base_path:expr, + dev_base_path: $dev_base_path:expr$(,)? + ) => { #[allow(clippy::unused_async)] pub async fn handle_static( path: ::axum::extract::Path, ) -> impl ::axum::response::IntoResponse { #[cfg(not(feature = "dev-web"))] - const STATIC_DIR: ::include_dir::Dir = $static_dir; + const DIR_1: ::include_dir::Dir = $dir_1; + $( + #[cfg(not(feature = "dev-web"))] + const DIR_2: ::include_dir::Dir = $dir_2; + )? + #[cfg(not(feature = "dev-web"))] fn get_static_file(path: &str) -> Option<&'static [u8]> { - STATIC_DIR.get_file(path).map(|f| f.contents()) + DIR_1.get_file(path).or_else(|| DIR_2.get_file(path)).map(|f| f.contents()) } #[cfg(feature = "dev-web")] diff --git a/src/npm/src/lib.rs b/src/npm/src/lib.rs index c3f8eef64f8ed..3639ad8485e26 100644 --- a/src/npm/src/lib.rs +++ b/src/npm/src/lib.rs @@ -196,7 +196,7 @@ impl NpmPackage { } } -pub fn ensure() -> Result<(), anyhow::Error> { +pub fn ensure(out_dir: Option) -> Result<(), anyhow::Error> { println!("ensuring all npm packages are up-to-date..."); let client = reqwest::blocking::Client::new(); @@ -273,9 +273,22 @@ expected: {} for dir in &[CSS_VENDOR, JS_PROD_VENDOR, JS_DEV_VENDOR] { for entry in WalkDir::new(dir) { let entry = entry?; - if entry.file_type().is_file() && !known_paths.contains(entry.path()) { - println!("removing stray vendor file {}", entry.path().display()); - fs::remove_file(entry.path())?; + if entry.file_type().is_file() { + if !known_paths.contains(entry.path()) { + println!("removing stray vendor file {}", entry.path().display()); + fs::remove_file(entry.path())?; + } else if let Some(out_dir) = &out_dir { + let dst_path = out_dir.join(entry.path()); + println!( + "copying path to OUT_DIR, src {}, dst {}", + entry.path().display(), + dst_path.display(), + ); + if let Some(parent) = dst_path.parent() { + fs::create_dir_all(parent)?; + } + fs::copy(entry.path(), dst_path)?; + } } } } diff --git a/src/prof-http/build.rs b/src/prof-http/build.rs index 3d32d79f168f9..782a59286b493 100644 --- a/src/prof-http/build.rs +++ b/src/prof-http/build.rs @@ -8,5 +8,6 @@ // by the Apache License, Version 2.0. fn main() -> Result<(), anyhow::Error> { - mz_npm::ensure() + let out_dir = std::env::var("OUT_DIR").ok().map(std::path::PathBuf::from); + mz_npm::ensure(out_dir) } diff --git a/src/prof-http/src/lib.rs b/src/prof-http/src/lib.rs index 5736f41dc564c..5b389f63d2277 100644 --- a/src/prof-http/src/lib.rs +++ b/src/prof-http/src/lib.rs @@ -42,9 +42,10 @@ static EXECUTABLE: LazyLock = LazyLock::new(|| { }); mz_http_util::make_handle_static!( - include_dir::include_dir!("$CARGO_MANIFEST_DIR/src/http/static"), - "src/http/static", - "src/http/static-dev" + dir_1: ::include_dir::include_dir!("$CARGO_MANIFEST_DIR/src/http/static"), + dir_2: ::include_dir::include_dir!("$OUT_DIR/src/http/static"), + prod_base_path: "src/http/static", + dev_base_path: "src/http/static-dev", ); /// Creates a router that serves the profiling endpoints. From ab43e648f3fa08ec8a47ac40ce0857cbf20a1408 Mon Sep 17 00:00:00 2001 From: Parker Timmerman Date: Wed, 29 Jan 2025 11:06:48 -0500 Subject: [PATCH 12/12] [copy_from] AWS source (#31144) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit _Stacked on top of_: https://github.com/MaterializeInc/materialize/pull/30956 This PR implements a new AwsS3 `OneshotSource` that allows copying in files from S3, e.g. ``` COPY INTO my_table FROM 's3://my-test-bucket' (FORMAT CSV, FILES = ['important.csv']); ``` Along with `FILES = []` we also support a `PATTERN = ` option which allows copying multiple files all at once. ### Motivation Fixes https://github.com/MaterializeInc/database-issues/issues/8860 Fixes https://github.com/MaterializeInc/database-issues/issues/8855 ### Tips for reviewer Review only the final commit, the one titled "start, implementation of an S3 oneshot source" ### Checklist - [x] This PR has adequate test coverage / QA involvement has been duly considered. ([trigger-ci for additional test/nightly runs](https://trigger-ci.dev.materialize.com/)) - [x] This PR has an associated up-to-date [design doc](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/design/README.md), is a design doc ([template](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/design/00000000_template.md)), or is sufficiently small to not require a design. - [x] If this PR evolves [an existing `$T ⇔ Proto$T` mapping](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/command-and-response-binary-encoding.md) (possibly in a backwards-incompatible way), then it is tagged with a `T-proto` label. - [x] If this PR will require changes to cloud orchestration or tests, there is a companion cloud PR to account for those changes that is tagged with the release-blocker label ([example](https://github.com/MaterializeInc/cloud/pull/5021)). - [x] If this PR includes major [user-facing behavior changes](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/guide-changes.md#what-changes-require-a-release-note), I have pinged the relevant PM to schedule a changelog post. --- Cargo.lock | 11 +- src/adapter/src/coord/sequencer.rs | 2 +- .../src/coord/sequencer/inner/copy_from.rs | 81 ++++++- src/aws-util/Cargo.toml | 9 +- src/aws-util/src/s3.rs | 31 ++- src/sql-lexer/src/keywords.txt | 2 + src/sql-parser/src/ast/defs/statement.rs | 5 + src/sql-parser/src/parser.rs | 8 +- src/sql-parser/tests/testdata/copy | 23 +- src/sql/src/plan.rs | 18 +- src/sql/src/plan/statement/dml.rs | 39 ++- src/sql/src/rbac.rs | 1 + src/storage-operators/Cargo.toml | 6 +- src/storage-operators/src/oneshot_source.rs | 138 ++++++++++- .../src/oneshot_source/aws_source.rs | 223 ++++++++++++++++++ .../src/oneshot_source/http_source.rs | 7 +- src/storage-types/src/oneshot_sources.proto | 24 ++ src/storage-types/src/oneshot_sources.rs | 89 ++++++- src/storage/src/render.rs | 5 + 19 files changed, 676 insertions(+), 46 deletions(-) create mode 100644 src/storage-operators/src/oneshot_source/aws_source.rs diff --git a/Cargo.lock b/Cargo.lock index efd8642ce77ae..db04d807601b6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3068,9 +3068,9 @@ checksum = "78cc372d058dcf6d5ecd98510e7fbc9e5aec4d21de70f65fea8fecebcd881bd4" [[package]] name = "glob" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" +checksum = "a8d1add55171497b4705a648c6b583acafb01d58050a51727785f0b2c8e0a2b2" [[package]] name = "globset" @@ -4789,12 +4789,15 @@ dependencies = [ "aws-sdk-s3", "aws-smithy-runtime", "aws-smithy-runtime-api", + "aws-smithy-types", "aws-types", "bytes", "bytesize", + "futures", "http 1.1.0", "hyper-tls 0.5.0", "mz-ore", + "pin-project", "thiserror", "tokio", "uuid", @@ -7137,13 +7140,17 @@ dependencies = [ "arrow", "async-compression", "async-stream", + "aws-smithy-types", "aws-types", "bytes", "bytesize", "csv-async", + "derivative", "differential-dataflow", "futures", + "glob", "http 1.1.0", + "itertools 0.12.1", "mz-arrow-util", "mz-aws-util", "mz-dyncfg", diff --git a/src/adapter/src/coord/sequencer.rs b/src/adapter/src/coord/sequencer.rs index afa6b4d2a90b9..a8cc8b837ce81 100644 --- a/src/adapter/src/coord/sequencer.rs +++ b/src/adapter/src/coord/sequencer.rs @@ -373,7 +373,7 @@ impl Coordinator { session, ); } - CopyFromSource::Url(_) => { + CopyFromSource::Url(_) | CopyFromSource::AwsS3 { .. } => { self.sequence_copy_from(ctx, plan, target_cluster).await; } }, diff --git a/src/adapter/src/coord/sequencer/inner/copy_from.rs b/src/adapter/src/coord/sequencer/inner/copy_from.rs index 80efef8d5f1fa..112c1251b516a 100644 --- a/src/adapter/src/coord/sequencer/inner/copy_from.rs +++ b/src/adapter/src/coord/sequencer/inner/copy_from.rs @@ -7,12 +7,14 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. +use std::str::FromStr; + use mz_adapter_types::connection::ConnectionId; use mz_ore::cast::CastInto; use mz_persist_client::batch::ProtoBatch; use mz_pgcopy::CopyFormatParams; use mz_repr::{CatalogItemId, Datum, RowArena}; -use mz_sql::plan::{self, CopyFromSource, HirScalarExpr}; +use mz_sql::plan::{self, CopyFromFilter, CopyFromSource, HirScalarExpr}; use mz_sql::session::metadata::SessionMetadata; use mz_storage_client::client::TableData; use mz_storage_types::oneshot_sources::OneshotIngestionRequest; @@ -38,16 +40,10 @@ impl Coordinator { source, columns: _, params, + filter, } = plan; - let from_expr = match source { - CopyFromSource::Url(from_expr) => from_expr, - CopyFromSource::Stdin => { - unreachable!("COPY FROM STDIN should be handled elsewhere") - } - }; - - let eval_url = |from: HirScalarExpr| -> Result { + let eval_uri = |from: HirScalarExpr| -> Result { let style = ExprPrepStyle::OneShot { logical_time: EvalTime::NotAvailable, session: ctx.session(), @@ -66,10 +62,8 @@ impl Coordinator { other => coord_bail!("programming error! COPY FROM target cannot be {other}"), }; - Url::parse(eval_string) - .map_err(|err| AdapterError::Unstructured(anyhow::anyhow!("{err}"))) + Ok(eval_string.to_string()) }; - let url = return_if_err!(eval_url(from_expr), ctx); // We check in planning that we're copying into a Table, but be defensive. let Some(dest_table) = self.catalog().get_entry(&id).table() else { @@ -93,9 +87,70 @@ impl Coordinator { } }; + let source = match source { + CopyFromSource::Url(from_expr) => { + let url = return_if_err!(eval_uri(from_expr), ctx); + // TODO(cf2): Structured errors. + let result = Url::parse(&url) + .map_err(|err| AdapterError::Unstructured(anyhow::anyhow!("{err}"))); + let url = return_if_err!(result, ctx); + + mz_storage_types::oneshot_sources::ContentSource::Http { url } + } + CopyFromSource::AwsS3 { + uri, + connection, + connection_id, + } => { + let uri = return_if_err!(eval_uri(uri), ctx); + + // Validate the URI is an S3 URI, with a bucket name. We rely on validating here + // and expect it in clusterd. + // + // TODO(cf2): Structured errors. + let result = http::Uri::from_str(&uri) + .map_err(|err| { + AdapterError::Unstructured(anyhow::anyhow!("expected S3 uri: {err}")) + }) + .and_then(|uri| { + if uri.scheme_str() != Some("s3") { + coord_bail!("only 's3://...' urls are supported as COPY FROM target"); + } + Ok(uri) + }) + .and_then(|uri| { + if uri.host().is_none() { + coord_bail!("missing bucket name from 's3://...' url"); + } + Ok(uri) + }); + let uri = return_if_err!(result, ctx); + + mz_storage_types::oneshot_sources::ContentSource::AwsS3 { + connection, + connection_id, + uri: uri.to_string(), + } + } + CopyFromSource::Stdin => { + unreachable!("COPY FROM STDIN should be handled elsewhere") + } + }; + + let filter = match filter { + None => mz_storage_types::oneshot_sources::ContentFilter::None, + Some(CopyFromFilter::Files(files)) => { + mz_storage_types::oneshot_sources::ContentFilter::Files(files) + } + Some(CopyFromFilter::Pattern(pattern)) => { + mz_storage_types::oneshot_sources::ContentFilter::Pattern(pattern) + } + }; + let request = OneshotIngestionRequest { - source: mz_storage_types::oneshot_sources::ContentSource::Http { url }, + source, format, + filter, }; let target_cluster = match self diff --git a/src/aws-util/Cargo.toml b/src/aws-util/Cargo.toml index 002f20e087766..4fde2216e29f7 100644 --- a/src/aws-util/Cargo.toml +++ b/src/aws-util/Cargo.toml @@ -12,15 +12,20 @@ workspace = true [dependencies] anyhow = "1.0.66" aws-config = { version = "1.2.0", default-features = false } -aws-sdk-s3 = { version = "1.23.0", default-features = false, features = ["rt-tokio"], optional = true } +aws-sdk-s3 = { version = "1.23.0", default-features = false, features = [ + "rt-tokio", +], optional = true } aws-smithy-runtime-api = "1.1.1" aws-smithy-runtime = { version = "1.1.1", features = ["connector-hyper-0-14-x"] } +aws-smithy-types = { version = "1.1.8", features = ["byte-stream-poll-next"] } aws-types = "1.1.1" bytes = "1.3.0" bytesize = "1.1.0" +futures = "0.3.25" http = "1.1.0" hyper-tls = "0.5.0" -mz-ore = { path = "../ore", default-features = false } +mz-ore = { path = "../ore", features = ["async"], default-features = false } +pin-project = "1.0.12" thiserror = "1.0.37" tokio = { version = "1.38.0", default-features = false, features = ["macros"] } uuid = { version = "1.7.0", features = ["v4"] } diff --git a/src/aws-util/src/s3.rs b/src/aws-util/src/s3.rs index 8173c3439dca8..931abc068f7d5 100644 --- a/src/aws-util/src/s3.rs +++ b/src/aws-util/src/s3.rs @@ -8,8 +8,10 @@ // by the Apache License, Version 2.0. use aws_sdk_s3::config::Builder; -use aws_sdk_s3::Client; use aws_types::sdk_config::SdkConfig; +use bytes::Bytes; + +pub use aws_sdk_s3::Client; /// Creates a new client from an [SDK config](aws_types::sdk_config::SdkConfig) /// with Materialize-specific customizations. @@ -46,3 +48,30 @@ pub async fn list_bucket_path( }) .transpose() } + +/// A wrapper around [`ByteStream`] that implements the [`futures::stream::Stream`] trait. +/// +/// [`ByteStream`]: aws_smithy_types::byte_stream::ByteStream +#[pin_project::pin_project] +pub struct ByteStreamAdapter { + #[pin] + inner: aws_smithy_types::byte_stream::ByteStream, +} + +impl ByteStreamAdapter { + pub fn new(bytes: aws_smithy_types::byte_stream::ByteStream) -> Self { + ByteStreamAdapter { inner: bytes } + } +} + +impl futures::stream::Stream for ByteStreamAdapter { + type Item = Result; + + fn poll_next( + self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + let this = self.project(); + aws_smithy_types::byte_stream::ByteStream::poll_next(this.inner, cx) + } +} diff --git a/src/sql-lexer/src/keywords.txt b/src/sql-lexer/src/keywords.txt index d58bf1815a484..14ef39c8b4b1a 100644 --- a/src/sql-lexer/src/keywords.txt +++ b/src/sql-lexer/src/keywords.txt @@ -174,6 +174,7 @@ Features Fetch Fields File +Files Filter First Fixpoint @@ -320,6 +321,7 @@ Partition Partitions Password Path +Pattern Physical Plan Plans diff --git a/src/sql-parser/src/ast/defs/statement.rs b/src/sql-parser/src/ast/defs/statement.rs index 8891a63dd9494..c9665e39822ff 100644 --- a/src/sql-parser/src/ast/defs/statement.rs +++ b/src/sql-parser/src/ast/defs/statement.rs @@ -374,6 +374,8 @@ pub enum CopyOptionName { Header, AwsConnection, MaxFileSize, + Files, + Pattern, } impl AstDisplay for CopyOptionName { @@ -387,6 +389,8 @@ impl AstDisplay for CopyOptionName { CopyOptionName::Header => "HEADER", CopyOptionName::AwsConnection => "AWS CONNECTION", CopyOptionName::MaxFileSize => "MAX FILE SIZE", + CopyOptionName::Files => "FILES", + CopyOptionName::Pattern => "PATTERN", }) } } @@ -407,6 +411,7 @@ impl WithOptionName for CopyOptionName { | CopyOptionName::Header | CopyOptionName::AwsConnection | CopyOptionName::MaxFileSize => false, + CopyOptionName::Files | CopyOptionName::Pattern => true, } } } diff --git a/src/sql-parser/src/parser.rs b/src/sql-parser/src/parser.rs index f55209e99e1b9..cfae801c2a2e3 100644 --- a/src/sql-parser/src/parser.rs +++ b/src/sql-parser/src/parser.rs @@ -6403,9 +6403,9 @@ impl<'a> Parser<'a> { } fn parse_copy_option(&mut self) -> Result, ParserError> { - let name = match self - .expect_one_of_keywords(&[FORMAT, DELIMITER, NULL, ESCAPE, QUOTE, HEADER, AWS, MAX])? - { + let name = match self.expect_one_of_keywords(&[ + FORMAT, DELIMITER, NULL, ESCAPE, QUOTE, HEADER, AWS, MAX, FILES, PATTERN, + ])? { FORMAT => CopyOptionName::Format, DELIMITER => CopyOptionName::Delimiter, NULL => CopyOptionName::Null, @@ -6423,6 +6423,8 @@ impl<'a> Parser<'a> { self.expect_keywords(&[FILE, SIZE])?; CopyOptionName::MaxFileSize } + FILES => CopyOptionName::Files, + PATTERN => CopyOptionName::Pattern, _ => unreachable!(), }; Ok(CopyOption { diff --git a/src/sql-parser/tests/testdata/copy b/src/sql-parser/tests/testdata/copy index 6cc286f74e709..694f8f5326398 100644 --- a/src/sql-parser/tests/testdata/copy +++ b/src/sql-parser/tests/testdata/copy @@ -79,7 +79,7 @@ Copy(CopyStatement { relation: Named { name: Name(UnresolvedItemName([Ident("t") parse-statement COPY t TO STDOUT () ---- -error: Expected one of FORMAT or DELIMITER or NULL or ESCAPE or QUOTE or HEADER or AWS or MAX, found right parenthesis +error: Expected one of FORMAT or DELIMITER or NULL or ESCAPE or QUOTE or HEADER or AWS or MAX or FILES or PATTERN, found right parenthesis COPY t TO STDOUT () ^ @@ -184,3 +184,24 @@ COPY INTO t(a, b) TO '/any/path' error: Expected identifier, found INTO COPY INTO t(a, b) TO '/any/path' ^ + +parse-statement +COPY INTO t1 FROM 'http://spacemonkey.info' WITH (FILES = ['foo.csv', 'bar.csv']); +---- +COPY t1 FROM 'http://spacemonkey.info' WITH (FILES = ('foo.csv', 'bar.csv')) +=> +Copy(CopyStatement { relation: Named { name: Name(UnresolvedItemName([Ident("t1")])), columns: [] }, direction: From, target: Expr(Value(String("http://spacemonkey.info"))), options: [CopyOption { name: Files, value: Some(Sequence([Value(String("foo.csv")), Value(String("bar.csv"))])) }] }) + +parse-statement +COPY INTO t1 FROM 'http://spacemonkey.info' WITH (FILES = ['foo.csv', 'bar.csv'], FORMAT CSV); +---- +COPY t1 FROM 'http://spacemonkey.info' WITH (FILES = ('foo.csv', 'bar.csv'), FORMAT = csv) +=> +Copy(CopyStatement { relation: Named { name: Name(UnresolvedItemName([Ident("t1")])), columns: [] }, direction: From, target: Expr(Value(String("http://spacemonkey.info"))), options: [CopyOption { name: Files, value: Some(Sequence([Value(String("foo.csv")), Value(String("bar.csv"))])) }, CopyOption { name: Format, value: Some(UnresolvedItemName(UnresolvedItemName([Ident("csv")]))) }] }) + +parse-statement +COPY INTO t1 FROM 'http://spacemonkey.info' WITH (FILES = ['foo.csv']); +---- +COPY t1 FROM 'http://spacemonkey.info' WITH (FILES = ('foo.csv')) +=> +Copy(CopyStatement { relation: Named { name: Name(UnresolvedItemName([Ident("t1")])), columns: [] }, direction: From, target: Expr(Value(String("http://spacemonkey.info"))), options: [CopyOption { name: Files, value: Some(Sequence([Value(String("foo.csv"))])) }] }) diff --git a/src/sql/src/plan.rs b/src/sql/src/plan.rs index 54154a2a6883c..35f67bca9f114 100644 --- a/src/sql/src/plan.rs +++ b/src/sql/src/plan.rs @@ -936,16 +936,32 @@ pub struct CopyFromPlan { pub source: CopyFromSource, pub columns: Vec, pub params: CopyFormatParams<'static>, + pub filter: Option, } #[derive(Debug)] pub enum CopyFromSource { /// Copying from a file local to the user, transmitted via pgwire. Stdin, - /// A remote resource, e.g. S3. + /// A remote resource, e.g. HTTP file. /// /// The contained [`HirScalarExpr`] evaluates to the Url for the remote resource. Url(HirScalarExpr), + /// A file in an S3 bucket. + AwsS3 { + /// Expression that evaluates to the file we want to copy. + uri: HirScalarExpr, + /// Details for how we connect to AWS S3. + connection: AwsConnection, + /// ID of the connection object. + connection_id: CatalogItemId, + }, +} + +#[derive(Debug)] +pub enum CopyFromFilter { + Files(Vec), + Pattern(String), } #[derive(Debug, Clone)] diff --git a/src/sql/src/plan/statement/dml.rs b/src/sql/src/plan/statement/dml.rs index e757bfed202b6..41034fdeef264 100644 --- a/src/sql/src/plan/statement/dml.rs +++ b/src/sql/src/plan/statement/dml.rs @@ -54,8 +54,8 @@ use crate::plan::query::{plan_expr, plan_up_to, ExprContext, QueryLifetime}; use crate::plan::scope::Scope; use crate::plan::statement::{ddl, StatementContext, StatementDesc}; use crate::plan::{ - self, side_effecting_func, transform_ast, CopyToPlan, CreateSinkPlan, ExplainPushdownPlan, - ExplainSinkSchemaPlan, ExplainTimestampPlan, + self, side_effecting_func, transform_ast, CopyFromFilter, CopyToPlan, CreateSinkPlan, + ExplainPushdownPlan, ExplainSinkSchemaPlan, ExplainTimestampPlan, }; use crate::plan::{ query, CopyFormat, CopyFromPlan, ExplainPlanPlan, InsertPlan, MutationKind, Params, Plan, @@ -1143,7 +1143,24 @@ fn plan_copy_from( }; let from = plan_expr(ecx, &from_expr)?.type_as(ecx, &ScalarType::String)?; - CopyFromSource::Url(from) + match options.aws_connection { + Some(conn_id) => { + let conn_id = CatalogItemId::from(conn_id); + + // Validate the connection type is one we expect. + let connection = match scx.get_item(&conn_id).connection()? { + mz_storage_types::connections::Connection::Aws(conn) => conn, + _ => sql_bail!("only AWS CONNECTION is supported in COPY ... FROM"), + }; + + CopyFromSource::AwsS3 { + uri: from, + connection, + connection_id: conn_id, + } + } + None => CopyFromSource::Url(from), + } } CopyTarget::Stdout => bail_never_supported!("COPY FROM {} not supported", target), }; @@ -1180,12 +1197,24 @@ fn plan_copy_from( CopyFormat::Parquet => bail_unsupported!("FORMAT PARQUET"), }; + let filter = match (options.files, options.pattern) { + (Some(_), Some(_)) => bail_unsupported!("must specify one of FILES or PATTERN"), + (Some(files), None) => Some(CopyFromFilter::Files(files)), + (None, Some(pattern)) => Some(CopyFromFilter::Pattern(pattern)), + (None, None) => None, + }; + + if filter.is_some() && matches!(source, CopyFromSource::Stdin) { + bail_unsupported!("COPY FROM ... WITH (FILES ...) only supported from a URL") + } + let (id, _, columns) = query::plan_copy_from(scx, table_name, columns)?; Ok(Plan::CopyFrom(CopyFromPlan { id, source, columns, params, + filter, })) } @@ -1206,7 +1235,9 @@ generate_extracted_config!( (Quote, String), (Header, bool), (AwsConnection, with_options::Object), - (MaxFileSize, ByteSize, Default(ByteSize::mb(256))) + (MaxFileSize, ByteSize, Default(ByteSize::mb(256))), + (Files, Vec), + (Pattern, String) ); pub fn plan_copy( diff --git a/src/sql/src/rbac.rs b/src/sql/src/rbac.rs index fabb8603ceb10..bba722dbcbf18 100644 --- a/src/sql/src/rbac.rs +++ b/src/sql/src/rbac.rs @@ -841,6 +841,7 @@ fn generate_rbac_requirements( source: _, columns: _, params: _, + filter: _, }) => RbacRequirements { privileges: vec![ ( diff --git a/src/storage-operators/Cargo.toml b/src/storage-operators/Cargo.toml index 9dcc08ab9c819..b442b4c7b6430 100644 --- a/src/storage-operators/Cargo.toml +++ b/src/storage-operators/Cargo.toml @@ -15,12 +15,16 @@ arrow = { version = "53.3.0", default-features = false } async-compression = { version = "0.4.5", features = ["bzip2", "gzip", "tokio", "xz", "zstd"] } async-stream = "0.3.3" aws-types = "1.1.1" +aws-smithy-types = "1.1.8" bytes = "1.3.0" bytesize = "1.1.0" -differential-dataflow = "0.13.4" csv-async = { version = "1.3.0", features = ["tokio"] } +derivative = "2.2.0" +differential-dataflow = "0.13.3" futures = "0.3.25" +glob = "0.3.2" http = "1.1.0" +itertools = "0.12.1" mz-aws-util = { path = "../aws-util" } mz-arrow-util = { path = "../arrow-util" } mz-dyncfg = { path = "../dyncfg" } diff --git a/src/storage-operators/src/oneshot_source.rs b/src/storage-operators/src/oneshot_source.rs index 17ec3f46bf76c..7755d087b3c2e 100644 --- a/src/storage-operators/src/oneshot_source.rs +++ b/src/storage-operators/src/oneshot_source.rs @@ -75,8 +75,11 @@ use mz_persist_client::cache::PersistClientCache; use mz_persist_client::Diagnostics; use mz_persist_types::codec_impls::UnitSchema; use mz_repr::{Diff, GlobalId, Row, Timestamp}; +use mz_storage_types::connections::ConnectionContext; use mz_storage_types::controller::CollectionMetadata; -use mz_storage_types::oneshot_sources::{ContentFormat, ContentSource, OneshotIngestionRequest}; +use mz_storage_types::oneshot_sources::{ + ContentFilter, ContentFormat, ContentSource, OneshotIngestionRequest, +}; use mz_storage_types::sources::SourceData; use mz_timely_util::builder_async::{ Event as AsyncEvent, OperatorBuilder as AsyncOperatorBuilder, PressOnDropButton, @@ -84,7 +87,7 @@ use mz_timely_util::builder_async::{ use mz_timely_util::pact::Distribute; use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; -use std::collections::LinkedList; +use std::collections::{BTreeSet, LinkedList}; use std::fmt::{Debug, Display}; use std::future::Future; use timely::container::CapacityContainerBuilder; @@ -93,10 +96,13 @@ use timely::dataflow::{Scope, Stream as TimelyStream}; use timely::progress::Antichain; use tracing::info; +use crate::oneshot_source::aws_source::{AwsS3Source, S3Checksum, S3Object}; use crate::oneshot_source::csv::{CsvDecoder, CsvRecord, CsvWorkRequest}; use crate::oneshot_source::http_source::{HttpChecksum, HttpObject, HttpOneshotSource}; pub mod csv; + +pub mod aws_source; pub mod http_source; /// Render a dataflow to do a "oneshot" ingestion. @@ -119,6 +125,7 @@ pub mod http_source; pub fn render( scope: G, persist_clients: Arc, + connection_context: ConnectionContext, collection_id: GlobalId, collection_meta: CollectionMetadata, request: OneshotIngestionRequest, @@ -128,14 +135,28 @@ where G: Scope, F: FnOnce(Result, String>) -> () + 'static, { - let OneshotIngestionRequest { source, format } = request; + let OneshotIngestionRequest { + source, + format, + filter, + } = request; let source = match source { ContentSource::Http { url } => { let source = HttpOneshotSource::new(reqwest::Client::default(), url); SourceKind::Http(source) } + ContentSource::AwsS3 { + connection, + connection_id, + uri, + } => { + let source = AwsS3Source::new(connection, connection_id, connection_context, uri); + SourceKind::AwsS3(source) + } }; + tracing::info!(?source, "created oneshot source"); + let format = match format { ContentFormat::Csv(params) => { let format = CsvDecoder::new(params, &collection_meta.relation_desc); @@ -145,7 +166,7 @@ where // Discover what objects are available to copy. let (objects_stream, discover_token) = - render_discover_objects(scope.clone(), collection_id, source.clone()); + render_discover_objects(scope.clone(), collection_id, source.clone(), filter); // Split the objects into individual units of work. let (work_stream, split_token) = render_split_work( scope.clone(), @@ -194,6 +215,7 @@ pub fn render_discover_objects( scope: G, collection_id: GlobalId, source: S, + filter: ContentFilter, ) -> ( TimelyStream>, PressOnDropButton, @@ -219,14 +241,38 @@ where return; } - info!(%collection_id, %worker_id, "CopyFrom Leader Discover"); + let filter = match ObjectFilter::try_new(filter) { + Ok(filter) => filter, + Err(err) => { + tracing::warn!(?err, "failed to create filter"); + start_handle.give(&start_cap, Err(StorageErrorXKind::generic(err).into())); + return; + } + }; let work = source.list().await.context("list"); match work { - Ok(objects) => objects - .into_iter() - .for_each(|object| start_handle.give(&start_cap, Ok(object))), - Err(err) => start_handle.give(&start_cap, Err(err)), + Ok(objects) => { + let names = objects.iter().map(|(o, _check)| o.name()); + let found: String = itertools::intersperse(names, ", ").collect(); + tracing::info!(%worker_id, %found, "listed objects"); + + let filtered: Vec<_> = objects + .into_iter() + .filter(|(o, _check)| filter.filter::(o)) + .collect(); + let names = filtered.iter().map(|(o, _check)| o.name()); + let returning: String = itertools::intersperse(names, ", ").collect(); + tracing::info!(%worker_id, %returning, "filtered objects"); + + filtered + .into_iter() + .for_each(|object| start_handle.give(&start_cap, Ok(object))) + } + Err(err) => { + tracing::warn!(?err, "failed to list oneshot source"); + start_handle.give(&start_cap, Err(err)) + } } }); @@ -279,6 +325,7 @@ where let format_ = format.clone(); let source_ = source.clone(); let work_requests = mz_ore::task::spawn(|| "split-work", async move { + info!(%worker_id, object = %object.name(), "splitting object"); format_.split_work(source_.clone(), object, checksum).await }) .await @@ -614,7 +661,7 @@ pub trait OneshotSource: Clone + Send { &'s self, object: Self::Object, checksum: Self::Checksum, - range: Option>, + range: Option>, ) -> BoxStream<'s, Result>; } @@ -624,9 +671,10 @@ pub trait OneshotSource: Clone + Send { /// making the trait object safe and it's easier to just wrap it in an enum. Also, this wrapper /// provides a convenient place to add [`StorageErrorXContext::context`] for all of our source /// types. -#[derive(Clone)] +#[derive(Clone, Debug)] pub(crate) enum SourceKind { Http(HttpOneshotSource), + AwsS3(AwsS3Source), } impl OneshotSource for SourceKind { @@ -645,6 +693,16 @@ impl OneshotSource for SourceKind { .collect(); Ok(objects) } + SourceKind::AwsS3(s3) => { + let objects = s3.list().await.context("s3")?; + let objects = objects + .into_iter() + .map(|(object, checksum)| { + (ObjectKind::AwsS3(object), ChecksumKind::AwsS3(checksum)) + }) + .collect(); + Ok(objects) + } } } @@ -652,7 +710,7 @@ impl OneshotSource for SourceKind { &'s self, object: Self::Object, checksum: Self::Checksum, - range: Option>, + range: Option>, ) -> BoxStream<'s, Result> { match (self, object, checksum) { (SourceKind::Http(http), ObjectKind::Http(object), ChecksumKind::Http(checksum)) => { @@ -660,6 +718,13 @@ impl OneshotSource for SourceKind { .map(|result| result.context("http")) .boxed() } + (SourceKind::AwsS3(s3), ObjectKind::AwsS3(object), ChecksumKind::AwsS3(checksum)) => s3 + .get(object, checksum, range) + .map(|result| result.context("aws_s3")) + .boxed(), + (SourceKind::AwsS3(_) | SourceKind::Http(_), _, _) => { + unreachable!("programming error! wrong source, object, and checksum kind"); + } } } } @@ -668,18 +733,21 @@ impl OneshotSource for SourceKind { #[derive(Debug, Clone, Serialize, Deserialize)] pub(crate) enum ObjectKind { Http(HttpObject), + AwsS3(S3Object), } impl OneshotObject for ObjectKind { fn name(&self) -> &str { match self { ObjectKind::Http(object) => object.name(), + ObjectKind::AwsS3(object) => object.name(), } } fn encodings(&self) -> &[Encoding] { match self { ObjectKind::Http(object) => object.encodings(), + ObjectKind::AwsS3(object) => object.encodings(), } } } @@ -688,6 +756,7 @@ impl OneshotObject for ObjectKind { #[derive(Debug, Clone, Serialize, Deserialize)] pub(crate) enum ChecksumKind { Http(HttpChecksum), + AwsS3(S3Checksum), } /// Defines a format that we fetch for a "one time" ingestion. @@ -801,6 +870,37 @@ pub(crate) enum RecordChunkKind { Csv(CsvRecord), } +pub(crate) enum ObjectFilter { + None, + Files(BTreeSet>), + Pattern(glob::Pattern), +} + +impl ObjectFilter { + pub fn try_new(filter: ContentFilter) -> Result { + match filter { + ContentFilter::None => Ok(ObjectFilter::None), + ContentFilter::Files(files) => { + let files = files.into_iter().map(|f| f.into()).collect(); + Ok(ObjectFilter::Files(files)) + } + ContentFilter::Pattern(pattern) => { + let pattern = glob::Pattern::new(&pattern)?; + Ok(ObjectFilter::Pattern(pattern)) + } + } + } + + /// Returns if the object should be included. + pub fn filter(&self, object: &S::Object) -> bool { + match self { + ObjectFilter::None => true, + ObjectFilter::Files(files) => files.contains(object.name()), + ObjectFilter::Pattern(pattern) => pattern.matches(object.name()), + } + } +} + /// Experimental Error Type. /// /// The goal of this type is to combine concepts from both `thiserror` and @@ -828,12 +928,20 @@ pub enum StorageErrorXKind { CsvDecoding(Arc), #[error("reqwest error: {0}")] Reqwest(Arc), + #[error("aws s3 request error: {0}")] + AwsS3Request(String), + #[error("aws s3 bytestream error: {0}")] + AwsS3Bytes(Arc), #[error("invalid reqwest header: {0}")] InvalidHeader(Arc), #[error("failed to decode Row from a record batch: {0}")] InvalidRecordBatch(Arc), #[error("programming error: {0}")] ProgrammingError(Arc), + #[error("failed to get the size of an object")] + MissingSize, + #[error("object is missing the required '{0}' field")] + MissingField(Arc), #[error("something went wrong: {0}")] Generic(String), } @@ -856,6 +964,12 @@ impl From for StorageErrorXKind { } } +impl From for StorageErrorXKind { + fn from(err: aws_smithy_types::byte_stream::error::Error) -> Self { + StorageErrorXKind::AwsS3Request(err.to_string()) + } +} + impl StorageErrorXKind { pub fn with_context(self, context: C) -> StorageErrorX { StorageErrorX { diff --git a/src/storage-operators/src/oneshot_source/aws_source.rs b/src/storage-operators/src/oneshot_source/aws_source.rs new file mode 100644 index 0000000000000..75349fd2e71fe --- /dev/null +++ b/src/storage-operators/src/oneshot_source/aws_source.rs @@ -0,0 +1,223 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0. + +//! AWS S3 [`OneshotSource`]. + +use std::path::Path; +use std::str::FromStr; +use std::sync::Arc; + +use derivative::Derivative; +use futures::stream::{BoxStream, TryStreamExt}; +use futures::StreamExt; +use mz_ore::future::InTask; +use mz_repr::CatalogItemId; +use mz_storage_types::connections::aws::AwsConnection; +use mz_storage_types::connections::ConnectionContext; +use serde::{Deserialize, Serialize}; + +use crate::oneshot_source::{ + OneshotObject, OneshotSource, StorageErrorX, StorageErrorXContext, StorageErrorXKind, +}; + +#[derive(Clone, Derivative)] +#[derivative(Debug)] +pub struct AwsS3Source { + // Only used for initialization. + #[derivative(Debug = "ignore")] + connection: Arc, + connection_id: CatalogItemId, + #[derivative(Debug = "ignore")] + context: Arc, + + /// Name of the S3 bucket we'll list from. + bucket: String, + /// Optional prefix that can be specified via an S3 URI. + prefix: Option, + /// S3 client that is lazily initialized. + #[derivative(Debug = "ignore")] + client: std::sync::OnceLock, +} + +impl AwsS3Source { + pub fn new( + connection: AwsConnection, + connection_id: CatalogItemId, + context: ConnectionContext, + uri: String, + ) -> Self { + let uri = http::Uri::from_str(&uri).expect("validated URI in sequencing"); + + let bucket = uri + .host() + .expect("validated host in sequencing") + .to_string(); + let prefix = if uri.path().is_empty() || uri.path() == "/" { + None + } else { + // The S3 client expects a trailing `/` but no leading `/`. + let mut prefix = uri.path().to_string(); + + if let Some(suffix) = prefix.strip_prefix('/') { + prefix = suffix.to_string(); + } + if !prefix.ends_with('/') { + prefix = format!("{prefix}/"); + } + + Some(prefix) + }; + + AwsS3Source { + connection: Arc::new(connection), + context: Arc::new(context), + connection_id, + bucket, + prefix, + client: std::sync::OnceLock::new(), + } + } + + pub async fn initialize(&self) -> Result { + let sdk_config = self + .connection + .load_sdk_config(&self.context, self.connection_id, InTask::Yes) + .await?; + let s3_client = mz_aws_util::s3::new_client(&sdk_config); + + Ok(s3_client) + } + + pub async fn client(&self) -> Result<&mz_aws_util::s3::Client, anyhow::Error> { + if self.client.get().is_none() { + let client = self.initialize().await?; + let _ = self.client.set(client); + } + + Ok(self.client.get().expect("just initialized")) + } +} + +#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] +pub struct S3Object { + /// Key from S3 list operation. + key: String, + /// Name of the object, generally the last component of the key. + name: String, + /// Size of the object in bytes. + size: usize, +} + +impl OneshotObject for S3Object { + fn name(&self) -> &str { + &self.name + } + + fn encodings(&self) -> &[super::Encoding] { + &[] + } +} + +#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] +pub struct S3Checksum { + e_tag: Option, +} + +impl OneshotSource for AwsS3Source { + type Object = S3Object; + type Checksum = S3Checksum; + + async fn list<'a>( + &'a self, + ) -> Result, super::StorageErrorX> { + let client = self.client().await.map_err(StorageErrorXKind::generic)?; + let mut objects_request = client.list_objects_v2().bucket(&self.bucket); + + // Users can optionally specify a prefix via the S3 uri they originally specify. + if let Some(prefix) = &self.prefix { + objects_request = objects_request.prefix(prefix); + } + + let objects = objects_request + .send() + .await + .map_err(StorageErrorXKind::generic) + .context("list_objects_v2")?; + + // TODO(cf1): Pagination. + + let objects: Vec<_> = objects + .contents() + .iter() + .map(|o| { + let key = o + .key() + .ok_or_else(|| StorageErrorXKind::MissingField("key".into()))? + .to_owned(); + let name = Path::new(&key) + .file_name() + .and_then(|os_name| os_name.to_str()) + .ok_or_else(|| StorageErrorXKind::Generic(format!("malformed key: {key}")))? + .to_string(); + let size = o + .size() + .ok_or_else(|| StorageErrorXKind::MissingField("size".into()))?; + let size: usize = size.try_into().map_err(StorageErrorXKind::generic)?; + + let object = S3Object { key, name, size }; + let checksum = S3Checksum { + e_tag: o.e_tag().map(|x| x.to_owned()), + }; + + Ok::<_, StorageErrorXKind>((object, checksum)) + }) + .collect::>() + .context("list")?; + + Ok(objects) + } + + fn get<'s>( + &'s self, + object: Self::Object, + _checksum: Self::Checksum, + range: Option>, + ) -> BoxStream<'s, Result> { + let initial_response = async move { + tracing::info!(name = %object.name(), ?range, "fetching object"); + + // TODO(cf1): Validate our checksum. + let client = self.client().await.map_err(StorageErrorXKind::generic)?; + + let mut request = client.get_object().bucket(&self.bucket).key(&object.name); + if let Some(range) = range { + // See the below link for the specifics of this format. + // + // + let range = format!("byte={}-{}", range.start(), range.end()); + request = request.range(range); + } + + let object = request + .send() + .await + .map_err(|err| StorageErrorXKind::AwsS3Request(err.to_string()))?; + // AWS's ByteStream doesn't implement the Stream trait. + let stream = mz_aws_util::s3::ByteStreamAdapter::new(object.body) + .err_into() + .boxed(); + + Ok::<_, StorageErrorXKind>(stream) + }; + + futures::stream::once(initial_response) + .try_flatten() + .boxed() + } +} diff --git a/src/storage-operators/src/oneshot_source/http_source.rs b/src/storage-operators/src/oneshot_source/http_source.rs index b87555ba135b3..bfbc1236f70c9 100644 --- a/src/storage-operators/src/oneshot_source/http_source.rs +++ b/src/storage-operators/src/oneshot_source/http_source.rs @@ -10,6 +10,7 @@ //! Generic HTTP oneshot source that will fetch a file from the public internet. use bytes::Bytes; +use derivative::Derivative; use futures::stream::{BoxStream, StreamExt}; use futures::TryStreamExt; use reqwest::Client; @@ -21,8 +22,10 @@ use crate::oneshot_source::{ }; /// Generic oneshot source that fetches a file from a URL on the public internet. -#[derive(Clone)] +#[derive(Clone, Derivative)] +#[derivative(Debug)] pub struct HttpOneshotSource { + #[derivative(Debug = "ignore")] client: Client, origin: Url, } @@ -122,7 +125,7 @@ impl OneshotSource for HttpOneshotSource { &'s self, object: Self::Object, _checksum: Self::Checksum, - _range: Option>, + _range: Option>, ) -> BoxStream<'s, Result> { // TODO(cf1): Support the range param. // TODO(cf1): Validate our checksum. diff --git a/src/storage-types/src/oneshot_sources.proto b/src/storage-types/src/oneshot_sources.proto index af10f61f637fa..eb88a65a2ccf4 100644 --- a/src/storage-types/src/oneshot_sources.proto +++ b/src/storage-types/src/oneshot_sources.proto @@ -11,22 +11,46 @@ syntax = "proto3"; package mz_storage_types.oneshot_sources; +import "google/protobuf/empty.proto"; import "pgcopy/src/copy.proto"; +import "repr/src/catalog_item_id.proto"; +import "storage-types/src/connections/aws.proto"; message ProtoOneshotIngestionRequest { oneof source { ProtoHttpContentSource http = 1; + ProtoAwsS3Source aws_s3 = 4; } oneof format { ProtoCsvContentFormat csv = 2; } + + oneof filter { + google.protobuf.Empty none = 5; + ProtoFilterFiles files = 6; + ProtoFilterPattern pattern = 7; + } } message ProtoHttpContentSource { string url = 1; } +message ProtoAwsS3Source { + mz_storage_types.connections.aws.ProtoAwsConnection connection = 1; + mz_repr.catalog_item_id.ProtoCatalogItemId connection_id = 2; + string uri = 3; +} + message ProtoCsvContentFormat { mz_pgcopy.copy.ProtoCopyCsvFormatParams params = 1; } + +message ProtoFilterFiles { + repeated string files = 1; +} + +message ProtoFilterPattern { + string pattern = 1; +} diff --git a/src/storage-types/src/oneshot_sources.rs b/src/storage-types/src/oneshot_sources.rs index 25167220c3932..5e679081269ee 100644 --- a/src/storage-types/src/oneshot_sources.rs +++ b/src/storage-types/src/oneshot_sources.rs @@ -11,12 +11,15 @@ use mz_pgcopy::CopyCsvFormatParams; use mz_proto::{IntoRustIfSome, RustType}; +use mz_repr::CatalogItemId; use mz_timely_util::builder_async::PressOnDropButton; use serde::{Deserialize, Serialize}; use tokio::sync::mpsc::UnboundedReceiver; use url::Url; +use crate::connections::aws::AwsConnection; + include!(concat!( env!("OUT_DIR"), "/mz_storage_types.oneshot_sources.rs" @@ -37,6 +40,7 @@ pub struct OneshotIngestionDescription { pub struct OneshotIngestionRequest { pub source: ContentSource, pub format: ContentFormat, + pub filter: ContentFilter, } impl RustType for OneshotIngestionRequest { @@ -44,6 +48,7 @@ impl RustType for OneshotIngestionRequest { ProtoOneshotIngestionRequest { source: Some(self.source.into_proto()), format: Some(self.format.into_proto()), + filter: Some(self.filter.into_proto()), } } @@ -56,14 +61,28 @@ impl RustType for OneshotIngestionRequest { let format = proto .format .into_rust_if_some("ProtoOneshotIngestionRequest::format")?; - - Ok(OneshotIngestionRequest { source, format }) + let filter = proto + .filter + .into_rust_if_some("ProtoOneshotIngestionRequest::filter")?; + + Ok(OneshotIngestionRequest { + source, + format, + filter, + }) } } #[derive(Debug, Clone, PartialEq, Eq, Deserialize, Serialize)] pub enum ContentSource { - Http { url: Url }, + Http { + url: Url, + }, + AwsS3 { + connection: AwsConnection, + connection_id: CatalogItemId, + uri: String, + }, } impl RustType for ContentSource { @@ -74,6 +93,15 @@ impl RustType for ContentSource { url: url.to_string(), }) } + ContentSource::AwsS3 { + connection, + connection_id, + uri, + } => proto_oneshot_ingestion_request::Source::AwsS3(ProtoAwsS3Source { + connection: Some(connection.into_proto()), + connection_id: Some(connection_id.into_proto()), + uri: uri.to_string(), + }), } } @@ -85,6 +113,17 @@ impl RustType for ContentSource { let url = Url::parse(&source.url).expect("failed to roundtrip Url"); Ok(ContentSource::Http { url }) } + proto_oneshot_ingestion_request::Source::AwsS3(source) => { + let connection = source.connection.into_rust_if_some("AwsS3::connection")?; + let connection_id = source + .connection_id + .into_rust_if_some("AwsS3::connection_id")?; + Ok(ContentSource::AwsS3 { + connection, + connection_id, + uri: source.uri, + }) + } } } } @@ -116,3 +155,47 @@ impl RustType for ContentFormat { } } } + +#[derive(Debug, Clone, PartialEq, Eq, Deserialize, Serialize)] +pub enum ContentFilter { + /// No filtering, fetch everything at a remote resource. + None, + /// Filter to only the files specified in this list. + Files(Vec), + /// Regex pattern to filter the files with. + Pattern(String), +} + +impl RustType for ContentFilter { + fn into_proto(&self) -> proto_oneshot_ingestion_request::Filter { + match self { + ContentFilter::None => { + proto_oneshot_ingestion_request::Filter::None(Default::default()) + } + ContentFilter::Files(files) => { + proto_oneshot_ingestion_request::Filter::Files(ProtoFilterFiles { + files: files.clone(), + }) + } + ContentFilter::Pattern(pattern) => { + proto_oneshot_ingestion_request::Filter::Pattern(ProtoFilterPattern { + pattern: pattern.clone(), + }) + } + } + } + + fn from_proto( + proto: proto_oneshot_ingestion_request::Filter, + ) -> Result { + match proto { + proto_oneshot_ingestion_request::Filter::None(()) => Ok(ContentFilter::None), + proto_oneshot_ingestion_request::Filter::Files(files) => { + Ok(ContentFilter::Files(files.files)) + } + proto_oneshot_ingestion_request::Filter::Pattern(pattern) => { + Ok(ContentFilter::Pattern(pattern.pattern)) + } + } + } +} diff --git a/src/storage/src/render.rs b/src/storage/src/render.rs index d9882e77f365f..98c6b1f20f0d8 100644 --- a/src/storage/src/render.rs +++ b/src/storage/src/render.rs @@ -489,11 +489,16 @@ pub(crate) fn build_oneshot_ingestion_dataflow( // here, but that might run into the infamous async-Drop problem. let _ = results_tx.send(result); }; + let connection_context = storage_state + .storage_configuration + .connection_context + .clone(); let tokens = timely_worker.dataflow(|scope| { mz_storage_operators::oneshot_source::render( scope.clone(), Arc::clone(&storage_state.persist_clients), + connection_context, collection_id, collection_meta, description,