From 50323969b974b35690827da275a3c5a4dca0b5cd Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Wed, 12 Jul 2023 16:01:37 +0200 Subject: [PATCH 01/10] Revert "Merge pull request #20124 from umanwizard/backport_57.9_reverts" This reverts commit 086e49baccc7d0efafb6cf96ee468cddf1431b45, reversing changes made to c4a8c41dc8a15ce7b5620cde60c9762e7448c3b3. --- Cargo.lock | 2 +- clippy.toml | 20 +- .../content/sql/system-catalog/mz_internal.md | 45 ++- src/adapter/src/catalog/builtin.rs | 145 ++++++-- src/cluster/src/types.rs | 2 +- src/compute-client/src/logging.proto | 4 + src/compute-client/src/logging.rs | 27 +- src/compute/Cargo.toml | 1 + src/compute/src/compute_state.rs | 2 +- src/compute/src/extensions/arrange.rs | 317 +++++++++++++++++ src/compute/src/extensions/collection.rs | 108 ++++++ src/compute/src/extensions/mod.rs | 16 + .../src => compute/src/extensions}/reduce.rs | 45 ++- src/compute/src/lib.rs | 1 + src/compute/src/logging/compute.rs | 331 ++++++++++++++++-- src/compute/src/logging/differential.rs | 79 ++++- src/compute/src/logging/initialize.rs | 47 +-- src/compute/src/logging/mod.rs | 13 + src/compute/src/logging/reachability.rs | 16 +- src/compute/src/logging/timely.rs | 107 +++--- src/compute/src/render/context.rs | 10 +- src/compute/src/render/join/linear_join.rs | 5 +- src/compute/src/render/mod.rs | 16 +- src/compute/src/render/reduce.rs | 63 ++-- src/compute/src/render/threshold.rs | 14 +- src/compute/src/render/top_k.rs | 19 +- src/compute/src/server.rs | 4 +- src/compute/src/typedefs.rs | 8 +- .../http/static/js/hierarchical-memory.jsx | 17 +- .../src/http/static/js/memory.jsx | 32 +- src/storage-client/src/types/errors.rs | 20 ++ src/storage/src/render/sinks.rs | 6 + src/timely-util/Cargo.toml | 1 - src/timely-util/src/lib.rs | 1 - src/timely-util/src/operator.rs | 60 +--- test/sqllogictest/cluster.slt | 14 +- .../information_schema_tables.slt | 24 ++ test/testdrive/catalog.td | 10 +- .../divergent-dataflow-cancellation.td | 6 +- test/testdrive/indexes.td | 4 + test/testdrive/introspection-sources.td | 33 +- test/testdrive/logging.td | 6 + 42 files changed, 1363 insertions(+), 338 deletions(-) create mode 100644 src/compute/src/extensions/arrange.rs create mode 100644 src/compute/src/extensions/collection.rs create mode 100644 src/compute/src/extensions/mod.rs rename src/{timely-util/src => compute/src/extensions}/reduce.rs (67%) diff --git a/Cargo.lock b/Cargo.lock index 40141ca2a98f..b2218631c141 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3704,6 +3704,7 @@ dependencies = [ name = "mz-compute" version = "0.0.0" dependencies = [ + "ahash", "anyhow", "async-trait", "bytesize", @@ -5260,7 +5261,6 @@ dependencies = [ name = "mz-timely-util" version = "0.0.0" dependencies = [ - "ahash", "differential-dataflow", "futures-util", "mz-ore", diff --git a/clippy.toml b/clippy.toml index dc75f17e45f0..0e5b6a50d050 100644 --- a/clippy.toml +++ b/clippy.toml @@ -22,17 +22,23 @@ disallowed-methods = [ { path = "aws_sdk_s3::Client::new", reason = "use the `mz_aws_s3_util::new_client` function instead" }, - # Prevent access to Differential APIs that want to use the default trace or use a default name. - { path = "differential_dataflow::Collection::consolidate", reason = "use the `differential_dataflow::Collection::consolidate_named` function instead" }, - { path = "differential_dataflow::operators::arrange::arrangement::Arrange::arrange", reason = "use the `arrange_named` function instead" }, - { path = "differential_dataflow::operators::arrange::arrangement::ArrangeByKey::arrange_by_key", reason = "use the `Arrange::arrange_named` function instead" }, - { path = "differential_dataflow::operators::arrange::arrangement::ArrangeByKey::arrange_by_key_named", reason = "use the `Arrange::arrange_named` function instead" }, - { path = "differential_dataflow::operators::arrange::arrangement::ArrangeBySelf::arrange_by_self", reason = "use the `Arrange::arrange_named` function instead" }, - { path = "differential_dataflow::operators::arrange::arrangement::ArrangeBySelf::arrange_by_self_named", reason = "use the `Arrange::arrange_named` function instead" }, + # Prevent access to Differential APIs that want to use the default trace or use a default name, or where we offer + # our own wrapper + { path = "differential_dataflow::Collection::consolidate", reason = "use the `Consolidate::mz_consolidate` function instead" }, + { path = "differential_dataflow::Collection::consolidate_named", reason = "use the `Consolidate::mz_consolidate` function instead" }, + { path = "differential_dataflow::operators::arrange::arrangement::Arrange::arrange", reason = "use the `MzArrange::mz_arrange_named` function instead" }, + { path = "differential_dataflow::operators::arrange::arrangement::Arrange::arrange_named", reason = "use the `MzArrange::mz_arrange_named` function instead" }, + { path = "differential_dataflow::operators::arrange::arrangement::Arrange::arrange_core", reason = "use the `MzArrange::mz_arrange_core` function instead" }, + { path = "differential_dataflow::operators::arrange::arrangement::ArrangeByKey::arrange_by_key", reason = "use the `MzArrange::mz_arrange_named` function instead" }, + { path = "differential_dataflow::operators::arrange::arrangement::ArrangeByKey::arrange_by_key_named", reason = "use the `MzArrange::mz_arrange_named` function instead" }, + { path = "differential_dataflow::operators::arrange::arrangement::ArrangeBySelf::arrange_by_self", reason = "use the `MzArrange::mz_arrange_named` function instead" }, + { path = "differential_dataflow::operators::arrange::arrangement::ArrangeBySelf::arrange_by_self_named", reason = "use the `MzArrange::mz_arrange_named` function instead" }, { path = "differential_dataflow::operators::reduce::Count::count", reason = "use the `differential_dataflow::operators::reduce::ReduceCore::reduce_abelian` function instead" }, { path = "differential_dataflow::operators::reduce::Count::count_core", reason = "use the `differential_dataflow::operators::reduce::ReduceCore::reduce_abelian` function instead" }, { path = "differential_dataflow::operators::reduce::Reduce::reduce", reason = "use the `differential_dataflow::operators::reduce::ReduceCore::reduce_abelian` function instead" }, { path = "differential_dataflow::operators::reduce::Reduce::reduce_named", reason = "use the `differential_dataflow::operators::reduce::ReduceCore::reduce_abelian` function instead" }, + { path = "differential_dataflow::operators::reduce::ReduceCore::reduce_abelian", reason = "use the `differential_dataflow::operators::reduce::ReduceCore::reduce_abelian` function instead" }, + { path = "differential_dataflow::operators::reduce::ReduceCore::reduce_core", reason = "use the `differential_dataflow::operators::reduce::ReduceCore::reduce_abelian` function instead" }, { path = "differential_dataflow::operators::reduce::Threshold::distinct", reason = "use the `differential_dataflow::operators::reduce::ReduceCore::reduce_abelian` function instead" }, { path = "differential_dataflow::operators::reduce::Threshold::distinct_core", reason = "use the `differential_dataflow::operators::reduce::ReduceCore::reduce_abelian` function instead" }, { path = "differential_dataflow::operators::reduce::Threshold::threshold", reason = "use the `differential_dataflow::operators::reduce::ReduceCore::reduce_abelian` function instead" }, diff --git a/doc/user/content/sql/system-catalog/mz_internal.md b/doc/user/content/sql/system-catalog/mz_internal.md index 5b9a6cd184a7..e8762570e96b 100644 --- a/doc/user/content/sql/system-catalog/mz_internal.md +++ b/doc/user/content/sql/system-catalog/mz_internal.md @@ -371,12 +371,18 @@ The `mz_arrangement_sharing` view describes how many times each [arrangement] in The `mz_arrangement_sizes` view describes the size of each [arrangement] in the system. +The size, capacity, and allocations are an approximation, which may underestimate the actual size in memory. +Specifically, reductions can use more memory than we show here. + | Field | Type | Meaning | | -------------- |-------------| -------- | | `operator_id` | [`uint8`] | The ID of the operator that created the arrangement. Corresponds to [`mz_dataflow_operators.id`](#mz_dataflow_operators). | | `records` | [`numeric`] | The number of records in the arrangement. | | `batches` | [`numeric`] | The number of batches in the arrangement. | +| `size` | [`bigint`] | The utilized size in bytes of the arrangement. | +| `capacity` | [`bigint`] | The capacity in bytes of the arrangement. Can be larger than the size. | +| `allocations` | [`bigint`] | The number of separate memory allocations backing the arrangement. | @@ -484,6 +490,22 @@ The `mz_dataflow_addresses` view describes how the [dataflow] channels and opera +### `mz_dataflow_arrangement_sizes` + +The `mz_dataflow_arrangement_sizes` view describes how many records and batches +are contained in operators under each dataflow. + + +| Field | Type | Meaning | +|---------------|------------|------------------------------------------------------------------------------| +| `id` | [`bigint`] | The ID of the [dataflow]. Corresponds to [`mz_dataflows.id`](#mz_dataflows). | +| `name` | [`bigint`] | The name of the object (e.g., index) maintained by the dataflow. | +| `records` | [`bigint`] | The number of records in all arrangements in the dataflow. | +| `batches` | [`bigint`] | The number of batches in all arrangements in the dataflow. | +| `size` | [`bigint`] | The utilized size in bytes of the arrangements. | +| `capacity` | [`bigint`] | The capacity in bytes of the arrangements. Can be larger than the size. | +| `allocations` | [`bigint`] | The number of separate memory allocations backing the arrangements. | + ### `mz_dataflow_channels` The `mz_dataflow_channels` view describes the communication channels between [dataflow] operators. @@ -553,18 +575,15 @@ The `mz_dataflow_operator_parents` view describes how [dataflow] operators are n -### `mz_dataflow_arrangement_sizes` +### `mz_dataflow_shutdown_durations_histogram` -The `mz_dataflow_arrangement_sizes` view describes how many records and batches -are contained in operators under each dataflow. +The `mz_dataflow_shutdown_durations_histogram` view describes a histogram of the time in nanoseconds required to fully shut down dropped [dataflows][dataflow]. - -| Field | Type | Meaning | -|-----------|-------------|------------------------------------------------------------------------------| -| `id` | [`uint8`] | The ID of the [dataflow]. Corresponds to [`mz_dataflows.id`](#mz_dataflows). | -| `name` | [`text`] | The name of the object (e.g., index) maintained by the dataflow. | -| `records` | [`numeric`] | The number of records in all arrangements in the dataflow. | -| `batches` | [`numeric`] | The number of batches in all arrangements in the dataflow. | + +| Field | Type | Meaning | +| -------------- | ------------ | -------- | +| `duration_ns` | [`bigint`] | The upper bound of the bucket in nanoseconds. | +| `count` | [`bigint`] | The (noncumulative) count of dataflows in this bucket. | ### `mz_message_counts` @@ -604,6 +623,9 @@ The `mz_records_per_dataflow` view describes the number of records in each [data | `id` | [`uint8`] | The ID of the dataflow. Corresponds to [`mz_dataflows.id`](#mz_dataflows). | | `name` | [`text`] | The internal name of the dataflow. | | `records` | [`numeric`] | The number of records in the dataflow. | +| `size` | [`bigint`] | The utilized size in bytes of the arrangements. | +| `capacity` | [`bigint`] | The capacity in bytes of the arrangements. Can be larger than the size. | +| `allocations` | [`bigint`] | The number of separate memory allocations backing the arrangements. | @@ -618,6 +640,9 @@ The `mz_records_per_dataflow_operator` view describes the number of records in e | `name` | [`text`] | The internal name of the operator. | | `dataflow_id` | [`uint8`] | The ID of the dataflow. Corresponds to [`mz_dataflows.id`](#mz_dataflows). | | `records` | [`numeric`] | The number of records in the operator. | +| `size` | [`bigint`] | The utilized size in bytes of the arrangement. | +| `capacity` | [`bigint`] | The capacity in bytes of the arrangement. Can be larger than the size. | +| `allocations` | [`bigint`] | The number of separate memory allocations backing the arrangement. | diff --git a/src/adapter/src/catalog/builtin.rs b/src/adapter/src/catalog/builtin.rs index 35bf32cca5fb..b2ac47f2dbb8 100644 --- a/src/adapter/src/catalog/builtin.rs +++ b/src/adapter/src/catalog/builtin.rs @@ -1319,6 +1319,30 @@ pub const MZ_PEEK_DURATIONS_HISTOGRAM_RAW: BuiltinLog = BuiltinLog { variant: LogVariant::Compute(ComputeLog::PeekDuration), }; +pub const MZ_DATAFLOW_SHUTDOWN_DURATIONS_HISTOGRAM_RAW: BuiltinLog = BuiltinLog { + name: "mz_dataflow_shutdown_durations_histogram_raw", + schema: MZ_INTERNAL_SCHEMA, + variant: LogVariant::Compute(ComputeLog::ShutdownDuration), +}; + +pub const MZ_ARRANGEMENT_HEAP_SIZE_RAW: BuiltinLog = BuiltinLog { + name: "mz_arrangement_heap_size_raw", + schema: MZ_INTERNAL_SCHEMA, + variant: LogVariant::Compute(ComputeLog::ArrangementHeapSize), +}; + +pub const MZ_ARRANGEMENT_HEAP_CAPACITY_RAW: BuiltinLog = BuiltinLog { + name: "mz_arrangement_heap_capacity_raw", + schema: MZ_INTERNAL_SCHEMA, + variant: LogVariant::Compute(ComputeLog::ArrangementHeapCapacity), +}; + +pub const MZ_ARRANGEMENT_HEAP_ALLOCATIONS_RAW: BuiltinLog = BuiltinLog { + name: "mz_arrangement_heap_allocations_raw", + schema: MZ_INTERNAL_SCHEMA, + variant: LogVariant::Compute(ComputeLog::ArrangementHeapAllocations), +}; + pub const MZ_MESSAGE_COUNTS_RECEIVED_RAW: BuiltinLog = BuiltinLog { name: "mz_message_counts_received_raw", schema: MZ_INTERNAL_SCHEMA, @@ -2247,28 +2271,21 @@ pub const MZ_RECORDS_PER_DATAFLOW_OPERATOR_PER_WORKER: BuiltinView = BuiltinView name: "mz_records_per_dataflow_operator_per_worker", schema: MZ_INTERNAL_SCHEMA, sql: "CREATE VIEW mz_internal.mz_records_per_dataflow_operator_per_worker AS -WITH records_cte AS ( - SELECT - operator_id, - worker_id, - pg_catalog.count(*) AS records - FROM - mz_internal.mz_arrangement_records_raw - GROUP BY - operator_id, worker_id -) SELECT dod.id, dod.name, dod.worker_id, dod.dataflow_id, - records_cte.records + ar_size.records, + ar_size.size, + ar_size.capacity, + ar_size.allocations FROM - records_cte, + mz_internal.mz_arrangement_sizes_per_worker ar_size, mz_internal.mz_dataflow_operator_dataflows_per_worker dod WHERE - dod.id = records_cte.operator_id AND - dod.worker_id = records_cte.worker_id", + dod.id = ar_size.operator_id AND + dod.worker_id = ar_size.worker_id", }; pub const MZ_RECORDS_PER_DATAFLOW_OPERATOR: BuiltinView = BuiltinView { @@ -2279,7 +2296,10 @@ SELECT id, name, dataflow_id, - pg_catalog.sum(records) AS records + pg_catalog.sum(records) AS records, + pg_catalog.sum(size) AS size, + pg_catalog.sum(capacity) AS capacity, + pg_catalog.sum(allocations) AS allocations FROM mz_internal.mz_records_per_dataflow_operator_per_worker GROUP BY id, name, dataflow_id", }; @@ -2287,11 +2307,15 @@ GROUP BY id, name, dataflow_id", pub const MZ_RECORDS_PER_DATAFLOW_PER_WORKER: BuiltinView = BuiltinView { name: "mz_records_per_dataflow_per_worker", schema: MZ_INTERNAL_SCHEMA, - sql: "CREATE VIEW mz_internal.mz_records_per_dataflow_per_worker AS SELECT + sql: "CREATE VIEW mz_internal.mz_records_per_dataflow_per_worker AS +SELECT rdo.dataflow_id as id, dfs.name, rdo.worker_id, - pg_catalog.SUM(rdo.records) as records + pg_catalog.SUM(rdo.records) as records, + pg_catalog.SUM(rdo.size) as size, + pg_catalog.SUM(rdo.capacity) as capacity, + pg_catalog.SUM(rdo.allocations) as allocations FROM mz_internal.mz_records_per_dataflow_operator_per_worker rdo, mz_internal.mz_dataflows_per_worker dfs @@ -2307,10 +2331,14 @@ GROUP BY pub const MZ_RECORDS_PER_DATAFLOW: BuiltinView = BuiltinView { name: "mz_records_per_dataflow", schema: MZ_INTERNAL_SCHEMA, - sql: "CREATE VIEW mz_internal.mz_records_per_dataflow AS SELECT + sql: "CREATE VIEW mz_internal.mz_records_per_dataflow AS +SELECT id, name, - pg_catalog.SUM(records) as records + pg_catalog.SUM(records) as records, + pg_catalog.SUM(size) as size, + pg_catalog.SUM(capacity) as capacity, + pg_catalog.SUM(allocations) as allocations FROM mz_internal.mz_records_per_dataflow_per_worker GROUP BY @@ -2806,6 +2834,28 @@ FROM mz_internal.mz_peek_durations_histogram_per_worker GROUP BY duration_ns", }; +pub const MZ_DATAFLOW_SHUTDOWN_DURATIONS_HISTOGRAM_PER_WORKER: BuiltinView = BuiltinView { + name: "mz_dataflow_shutdown_durations_histogram_per_worker", + schema: MZ_INTERNAL_SCHEMA, + sql: "CREATE VIEW mz_internal.mz_dataflow_shutdown_durations_histogram_per_worker AS SELECT + worker_id, duration_ns, pg_catalog.count(*) AS count +FROM + mz_internal.mz_dataflow_shutdown_durations_histogram_raw +GROUP BY + worker_id, duration_ns", +}; + +pub const MZ_DATAFLOW_SHUTDOWN_DURATIONS_HISTOGRAM: BuiltinView = BuiltinView { + name: "mz_dataflow_shutdown_durations_histogram", + schema: MZ_INTERNAL_SCHEMA, + sql: "CREATE VIEW mz_internal.mz_dataflow_shutdown_durations_histogram AS +SELECT + duration_ns, + pg_catalog.sum(count) AS count +FROM mz_internal.mz_dataflow_shutdown_durations_histogram_per_worker +GROUP BY duration_ns", +}; + pub const MZ_SCHEDULING_ELAPSED_PER_WORKER: BuiltinView = BuiltinView { name: "mz_scheduling_elapsed_per_worker", schema: MZ_INTERNAL_SCHEMA, @@ -3006,13 +3056,50 @@ records_cte AS ( mz_internal.mz_arrangement_records_raw GROUP BY operator_id, worker_id +), +heap_size_cte AS ( + SELECT + operator_id, + worker_id, + pg_catalog.count(*) AS size + FROM + mz_internal.mz_arrangement_heap_size_raw + GROUP BY + operator_id, worker_id +), +heap_capacity_cte AS ( + SELECT + operator_id, + worker_id, + pg_catalog.count(*) AS capacity + FROM + mz_internal.mz_arrangement_heap_capacity_raw + GROUP BY + operator_id, worker_id +), +heap_allocations_cte AS ( + SELECT + operator_id, + worker_id, + pg_catalog.count(*) AS allocations + FROM + mz_internal.mz_arrangement_heap_allocations_raw + GROUP BY + operator_id, worker_id ) SELECT batches_cte.operator_id, batches_cte.worker_id, records_cte.records, - batches_cte.batches -FROM batches_cte JOIN records_cte USING (operator_id, worker_id)", + batches_cte.batches, + heap_size_cte.size, + heap_capacity_cte.capacity, + heap_allocations_cte.allocations +FROM batches_cte +JOIN records_cte USING (operator_id, worker_id) +JOIN heap_size_cte USING (operator_id, worker_id) +JOIN heap_capacity_cte USING (operator_id, worker_id) +JOIN heap_allocations_cte USING (operator_id, worker_id)", }; pub const MZ_ARRANGEMENT_SIZES: BuiltinView = BuiltinView { @@ -3022,7 +3109,10 @@ pub const MZ_ARRANGEMENT_SIZES: BuiltinView = BuiltinView { SELECT operator_id, pg_catalog.sum(records) AS records, - pg_catalog.sum(batches) AS batches + pg_catalog.sum(batches) AS batches, + pg_catalog.sum(size) AS size, + pg_catalog.sum(capacity) AS capacity, + pg_catalog.sum(allocations) AS allocations FROM mz_internal.mz_arrangement_sizes_per_worker GROUP BY operator_id", }; @@ -3108,7 +3198,10 @@ pub const MZ_DATAFLOW_ARRANGEMENT_SIZES: BuiltinView = BuiltinView { mdod.dataflow_id AS id, mo.name, COALESCE(sum(mas.records), 0) AS records, - COALESCE(sum(mas.batches), 0) AS batches + COALESCE(sum(mas.batches), 0) AS batches, + COALESCE(sum(mas.size), 0) AS size, + COALESCE(sum(mas.capacity), 0) AS capacity, + COALESCE(sum(mas.allocations), 0) AS allocations FROM mz_internal.mz_dataflow_operators AS mdo LEFT JOIN mz_internal.mz_arrangement_sizes AS mas ON mdo.id = mas.operator_id @@ -4061,6 +4154,10 @@ pub static BUILTINS_STATIC: Lazy>> = Lazy::new(|| { Builtin::Log(&MZ_MESSAGE_COUNTS_SENT_RAW), Builtin::Log(&MZ_ACTIVE_PEEKS_PER_WORKER), Builtin::Log(&MZ_PEEK_DURATIONS_HISTOGRAM_RAW), + Builtin::Log(&MZ_DATAFLOW_SHUTDOWN_DURATIONS_HISTOGRAM_RAW), + Builtin::Log(&MZ_ARRANGEMENT_HEAP_CAPACITY_RAW), + Builtin::Log(&MZ_ARRANGEMENT_HEAP_ALLOCATIONS_RAW), + Builtin::Log(&MZ_ARRANGEMENT_HEAP_SIZE_RAW), Builtin::Log(&MZ_SCHEDULING_ELAPSED_RAW), Builtin::Log(&MZ_COMPUTE_OPERATOR_DURATIONS_HISTOGRAM_RAW), Builtin::Log(&MZ_SCHEDULING_PARKS_HISTOGRAM_RAW), @@ -4151,6 +4248,8 @@ pub static BUILTINS_STATIC: Lazy>> = Lazy::new(|| { Builtin::View(&MZ_RECORDS_PER_DATAFLOW), Builtin::View(&MZ_PEEK_DURATIONS_HISTOGRAM_PER_WORKER), Builtin::View(&MZ_PEEK_DURATIONS_HISTOGRAM), + Builtin::View(&MZ_DATAFLOW_SHUTDOWN_DURATIONS_HISTOGRAM_PER_WORKER), + Builtin::View(&MZ_DATAFLOW_SHUTDOWN_DURATIONS_HISTOGRAM), Builtin::View(&MZ_SCHEDULING_ELAPSED_PER_WORKER), Builtin::View(&MZ_SCHEDULING_ELAPSED), Builtin::View(&MZ_SCHEDULING_PARKS_HISTOGRAM_PER_WORKER), diff --git a/src/cluster/src/types.rs b/src/cluster/src/types.rs index 267071cfb53f..082ad8becdf3 100644 --- a/src/cluster/src/types.rs +++ b/src/cluster/src/types.rs @@ -30,7 +30,7 @@ pub trait AsRunnableWorker { /// Build and continuously run a worker. Called on each timely /// thread. - fn build_and_run( + fn build_and_run( config: Self, timely_worker: &mut TimelyWorker, client_rx: crossbeam_channel::Receiver<( diff --git a/src/compute-client/src/logging.proto b/src/compute-client/src/logging.proto index 090f53d54109..db787ad17db2 100644 --- a/src/compute-client/src/logging.proto +++ b/src/compute-client/src/logging.proto @@ -52,6 +52,10 @@ message ProtoComputeLog { google.protobuf.Empty peek_duration = 5; google.protobuf.Empty frontier_delay = 6; google.protobuf.Empty import_frontier_current = 7; + google.protobuf.Empty arrangement_heap_size = 8; + google.protobuf.Empty arrangement_heap_capacity = 9; + google.protobuf.Empty arrangement_heap_allocations = 10; + google.protobuf.Empty shutdown_duration = 11; } } message ProtoLogVariant { diff --git a/src/compute-client/src/logging.rs b/src/compute-client/src/logging.rs index db6f27219bc7..915aa7d0aebb 100644 --- a/src/compute-client/src/logging.rs +++ b/src/compute-client/src/logging.rs @@ -220,6 +220,10 @@ pub enum ComputeLog { PeekDuration, FrontierDelay, ImportFrontierCurrent, + ArrangementHeapSize, + ArrangementHeapCapacity, + ArrangementHeapAllocations, + ShutdownDuration, } impl RustType for ComputeLog { @@ -234,6 +238,10 @@ impl RustType for ComputeLog { ComputeLog::PeekDuration => PeekDuration(()), ComputeLog::FrontierDelay => FrontierDelay(()), ComputeLog::ImportFrontierCurrent => ImportFrontierCurrent(()), + ComputeLog::ArrangementHeapSize => ArrangementHeapSize(()), + ComputeLog::ArrangementHeapCapacity => ArrangementHeapCapacity(()), + ComputeLog::ArrangementHeapAllocations => ArrangementHeapAllocations(()), + ComputeLog::ShutdownDuration => ShutdownDuration(()), }), } } @@ -248,6 +256,10 @@ impl RustType for ComputeLog { Some(PeekDuration(())) => Ok(ComputeLog::PeekDuration), Some(FrontierDelay(())) => Ok(ComputeLog::FrontierDelay), Some(ImportFrontierCurrent(())) => Ok(ComputeLog::ImportFrontierCurrent), + Some(ArrangementHeapSize(())) => Ok(ComputeLog::ArrangementHeapSize), + Some(ArrangementHeapCapacity(())) => Ok(ComputeLog::ArrangementHeapCapacity), + Some(ArrangementHeapAllocations(())) => Ok(ComputeLog::ArrangementHeapAllocations), + Some(ShutdownDuration(())) => Ok(ComputeLog::ShutdownDuration), None => Err(TryFromProtoError::missing_field("ProtoComputeLog::kind")), } } @@ -365,7 +377,10 @@ impl LogVariant { LogVariant::Differential(DifferentialLog::ArrangementBatches) | LogVariant::Differential(DifferentialLog::ArrangementRecords) - | LogVariant::Differential(DifferentialLog::Sharing) => RelationDesc::empty() + | LogVariant::Differential(DifferentialLog::Sharing) + | LogVariant::Compute(ComputeLog::ArrangementHeapSize) + | LogVariant::Compute(ComputeLog::ArrangementHeapCapacity) + | LogVariant::Compute(ComputeLog::ArrangementHeapAllocations) => RelationDesc::empty() .with_column("operator_id", ScalarType::UInt64.nullable(false)) .with_column("worker_id", ScalarType::UInt64.nullable(false)), @@ -407,6 +422,10 @@ impl LogVariant { LogVariant::Compute(ComputeLog::PeekDuration) => RelationDesc::empty() .with_column("worker_id", ScalarType::UInt64.nullable(false)) .with_column("duration_ns", ScalarType::UInt64.nullable(false)), + + LogVariant::Compute(ComputeLog::ShutdownDuration) => RelationDesc::empty() + .with_column("worker_id", ScalarType::UInt64.nullable(false)) + .with_column("duration_ns", ScalarType::UInt64.nullable(false)), } } @@ -445,7 +464,10 @@ impl LogVariant { LogVariant::Timely(TimelyLog::Reachability) => vec![], LogVariant::Differential(DifferentialLog::ArrangementBatches) | LogVariant::Differential(DifferentialLog::ArrangementRecords) - | LogVariant::Differential(DifferentialLog::Sharing) => vec![( + | LogVariant::Differential(DifferentialLog::Sharing) + | LogVariant::Compute(ComputeLog::ArrangementHeapSize) + | LogVariant::Compute(ComputeLog::ArrangementHeapCapacity) + | LogVariant::Compute(ComputeLog::ArrangementHeapAllocations) => vec![( LogVariant::Timely(TimelyLog::Operates), vec![(0, 0), (1, 1)], )], @@ -456,6 +478,7 @@ impl LogVariant { LogVariant::Compute(ComputeLog::FrontierDelay) => vec![], LogVariant::Compute(ComputeLog::PeekCurrent) => vec![], LogVariant::Compute(ComputeLog::PeekDuration) => vec![], + LogVariant::Compute(ComputeLog::ShutdownDuration) => vec![], } } } diff --git a/src/compute/Cargo.toml b/src/compute/Cargo.toml index 6b6c86c63a54..7fa561d8adb6 100644 --- a/src/compute/Cargo.toml +++ b/src/compute/Cargo.toml @@ -7,6 +7,7 @@ rust-version.workspace = true publish = false [dependencies] +ahash = { version = "0.8.0", default_features = false } anyhow = "1.0.66" async-trait = "0.1.68" bytesize = "1.1.0" diff --git a/src/compute/src/compute_state.rs b/src/compute/src/compute_state.rs index c2b1770bcf1c..448bf87b0fc9 100644 --- a/src/compute/src/compute_state.rs +++ b/src/compute/src/compute_state.rs @@ -118,7 +118,7 @@ impl SinkToken { } } -impl<'a, A: Allocate> ActiveComputeState<'a, A> { +impl<'a, A: Allocate + 'static> ActiveComputeState<'a, A> { /// Entrypoint for applying a compute command. #[tracing::instrument(level = "debug", skip(self))] pub fn handle_compute_command(&mut self, cmd: ComputeCommand) { diff --git a/src/compute/src/extensions/arrange.rs b/src/compute/src/extensions/arrange.rs new file mode 100644 index 000000000000..c1d8271de871 --- /dev/null +++ b/src/compute/src/extensions/arrange.rs @@ -0,0 +1,317 @@ +// 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. + +use differential_dataflow::difference::Semigroup; +use differential_dataflow::lattice::Lattice; +use differential_dataflow::operators::arrange::{Arrange, Arranged, TraceAgent}; +use differential_dataflow::trace::{Batch, Trace, TraceReader}; +use differential_dataflow::{Collection, Data, ExchangeData, Hashable}; +use timely::container::columnation::Columnation; +use timely::dataflow::channels::pact::{ParallelizationContract, Pipeline}; +use timely::dataflow::operators::Operator; +use timely::dataflow::{Scope, ScopeParent}; +use timely::progress::{Antichain, Timestamp}; + +use crate::logging::compute::ComputeEvent; +use crate::typedefs::{RowKeySpine, RowSpine}; + +/// Extension trait to arrange data. +pub trait MzArrange +where + ::Timestamp: Lattice, +{ + /// The current scope. + type Scope: Scope; + /// The key type. + type Key: Data; + /// The value type. + type Val: Data; + /// The difference type. + type R: Data + Semigroup; + + /// Arranges a stream of `(Key, Val)` updates by `Key` into a trace of type `Tr`. + /// + /// This operator arranges a stream of values into a shared trace, whose contents it maintains. + /// This trace is current for all times marked completed in the output stream, and probing this stream + /// is the correct way to determine that times in the shared trace are committed. + fn mz_arrange(&self, name: &str) -> Arranged> + where + Self::Key: ExchangeData + Hashable, + Self::Val: ExchangeData, + Self::R: ExchangeData, + Tr: Trace + + TraceReader< + Key = Self::Key, + Val = Self::Val, + Time = ::Timestamp, + R = Self::R, + > + 'static, + Tr::Batch: Batch, + Arranged>: ArrangementSize; + + /// Arranges a stream of `(Key, Val)` updates by `Key` into a trace of type `Tr`. Partitions + /// the data according to `pact`. + /// + /// This operator arranges a stream of values into a shared trace, whose contents it maintains. + /// This trace is current for all times marked completed in the output stream, and probing this stream + /// is the correct way to determine that times in the shared trace are committed. + fn mz_arrange_core(&self, pact: P, name: &str) -> Arranged> + where + P: ParallelizationContract< + ::Timestamp, + ( + (Self::Key, Self::Val), + ::Timestamp, + Self::R, + ), + >, + Tr: Trace + + TraceReader< + Key = Self::Key, + Val = Self::Val, + Time = ::Timestamp, + R = Self::R, + > + 'static, + Tr::Batch: Batch, + Arranged>: ArrangementSize; +} + +impl MzArrange for Collection +where + G: Scope, + G::Timestamp: Lattice, + K: Data + Columnation, + V: Data + Columnation, + R: Semigroup, +{ + type Scope = G; + type Key = K; + type Val = V; + type R = R; + + fn mz_arrange(&self, name: &str) -> Arranged> + where + K: ExchangeData + Hashable, + V: ExchangeData, + R: ExchangeData, + Tr: Trace + TraceReader + 'static, + Tr::Batch: Batch, + Arranged>: ArrangementSize, + { + // Allow access to `arrange_named` because we're within Mz's wrapper. + #[allow(clippy::disallowed_methods)] + self.arrange_named(name).log_arrangement_size() + } + + fn mz_arrange_core(&self, pact: P, name: &str) -> Arranged> + where + P: ParallelizationContract, + Tr: Trace + TraceReader + 'static, + Tr::Batch: Batch, + Arranged>: ArrangementSize, + { + // Allow access to `arrange_named` because we're within Mz's wrapper. + #[allow(clippy::disallowed_methods)] + self.arrange_core(pact, name).log_arrangement_size() + } +} + +/// A specialized collection where data only has a key, but no associated value. +/// +/// Created by calling `collection.into()`. +pub struct KeyCollection(Collection); + +impl From> for KeyCollection { + fn from(value: Collection) -> Self { + KeyCollection(value) + } +} + +impl MzArrange for KeyCollection +where + G: Scope, + K: Data + Columnation, + G::Timestamp: Lattice, + R: Semigroup, +{ + type Scope = G; + type Key = K; + type Val = (); + type R = R; + + fn mz_arrange(&self, name: &str) -> Arranged> + where + K: ExchangeData + Hashable, + R: ExchangeData, + Tr: Trace + TraceReader + 'static, + Tr::Batch: Batch, + Arranged>: ArrangementSize, + { + self.0.map(|d| (d, ())).mz_arrange(name) + } + + fn mz_arrange_core(&self, pact: P, name: &str) -> Arranged> + where + P: ParallelizationContract, + Tr: Trace + TraceReader + 'static, + Tr::Batch: Batch, + Arranged>: ArrangementSize, + { + self.0.map(|d| (d, ())).mz_arrange_core(pact, name) + } +} + +/// A type that can log its heap size. +pub trait ArrangementSize { + /// Install a logger to track the heap size of the target. + fn log_arrangement_size(self) -> Self; +} + +/// Helper to compute the size of a vector in memory. +/// +/// The function only considers the immediate allocation of the vector, but is oblivious of any +/// pointers to owned allocations. +#[inline] +fn vec_size(data: &Vec, mut callback: impl FnMut(usize, usize)) { + let size_of_t = std::mem::size_of::(); + callback(data.len() * size_of_t, data.capacity() * size_of_t); +} + +/// Helper for [`ArrangementSize`] to install a common operator holding on to a trace. +/// +/// * `arranged`: The arrangement to inspect. +/// * `logic`: Closure that calculates the heap size/capacity/allocations for a trace. The return +/// value are size and capacity in bytes, and number of allocations, all in absolute values. +fn log_arrangement_size_inner( + arranged: Arranged>, + mut logic: L, +) -> Arranged> +where + G: Scope, + G::Timestamp: Timestamp + Lattice + Ord, + Tr: TraceReader + 'static, + Tr::Time: Timestamp + Lattice + Ord + Clone + 'static, + L: FnMut(&TraceAgent) -> (usize, usize, usize) + 'static, +{ + let scope = arranged.stream.scope(); + let Some(logger) = scope.log_register().get::("materialize/compute") else {return arranged}; + let mut trace = arranged.trace.clone(); + let operator = trace.operator().global_id; + + let (mut old_size, mut old_capacity, mut old_allocations) = (0isize, 0isize, 0isize); + + let stream = arranged + .stream + .unary(Pipeline, "ArrangementSize", |_cap, info| { + let mut buffer = Default::default(); + let address = info.address; + logger.log(ComputeEvent::ArrangementHeapSizeOperator { operator, address }); + move |input, output| { + while let Some((time, data)) = input.next() { + data.swap(&mut buffer); + output.session(&time).give_container(&mut buffer); + } + + // We don't want to block compaction. + let mut upper = Antichain::new(); + trace.read_upper(&mut upper); + trace.set_logical_compaction(upper.borrow()); + trace.set_physical_compaction(upper.borrow()); + + let (size, capacity, allocations) = logic(&trace); + + let size = size.try_into().expect("must fit"); + if size != old_size { + logger.log(ComputeEvent::ArrangementHeapSize { + operator, + delta_size: size - old_size, + }); + } + + let capacity = capacity.try_into().expect("must fit"); + if capacity != old_capacity { + logger.log(ComputeEvent::ArrangementHeapCapacity { + operator, + delta_capacity: capacity - old_capacity, + }); + } + + let allocations = allocations.try_into().expect("must fit"); + if allocations != old_allocations { + logger.log(ComputeEvent::ArrangementHeapAllocations { + operator, + delta_allocations: allocations - old_allocations, + }); + } + + old_size = size; + old_capacity = capacity; + old_allocations = allocations; + } + }); + Arranged { + trace: arranged.trace, + stream, + } +} + +impl ArrangementSize for Arranged>> +where + G: Scope, + G::Timestamp: Lattice + Ord, + K: Data + Columnation, + V: Data + Columnation, + T: Lattice + Timestamp, + R: Semigroup, +{ + fn log_arrangement_size(self) -> Self { + log_arrangement_size_inner(self, |trace| { + let (mut size, mut capacity, mut allocations) = (0, 0, 0); + let mut callback = |siz, cap| { + allocations += 1; + size += siz; + capacity += cap + }; + trace.map_batches(|batch| { + batch.layer.keys.heap_size(&mut callback); + batch.layer.vals.keys.heap_size(&mut callback); + vec_size(&batch.layer.offs, &mut callback); + vec_size(&batch.layer.vals.offs, &mut callback); + vec_size(&batch.layer.vals.vals.vals, &mut callback); + }); + (size, capacity, allocations) + }) + } +} + +impl ArrangementSize for Arranged>> +where + G: Scope, + G::Timestamp: Lattice + Ord, + K: Data + Columnation, + T: Lattice + Timestamp, + R: Semigroup, +{ + fn log_arrangement_size(self) -> Self { + log_arrangement_size_inner(self, |trace| { + let (mut size, mut capacity, mut allocations) = (0, 0, 0); + let mut callback = |siz, cap| { + allocations += 1; + size += siz; + capacity += cap + }; + trace.map_batches(|batch| { + batch.layer.keys.heap_size(&mut callback); + vec_size(&batch.layer.offs, &mut callback); + vec_size(&batch.layer.vals.vals, &mut callback); + }); + (size, capacity, allocations) + }) + } +} diff --git a/src/compute/src/extensions/collection.rs b/src/compute/src/extensions/collection.rs new file mode 100644 index 000000000000..513e814c3055 --- /dev/null +++ b/src/compute/src/extensions/collection.rs @@ -0,0 +1,108 @@ +// 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. + +//! Extensions to Differential collections. + +use std::hash::{BuildHasher, Hash, Hasher}; + +use differential_dataflow::difference::Semigroup; +use differential_dataflow::lattice::Lattice; +use differential_dataflow::operators::arrange::{Arranged, TraceAgent}; +use differential_dataflow::trace::{Batch, Trace, TraceReader}; +use differential_dataflow::{Collection, Data, ExchangeData}; +use timely::container::columnation::Columnation; +use timely::dataflow::channels::pact::Exchange; +use timely::dataflow::Scope; + +use crate::extensions::arrange::{ArrangementSize, MzArrange}; + +/// Extension methods for differential [`Collection`]s. +pub(crate) trait ConsolidateExt +where + G: Scope, + G::Timestamp: Lattice + Data, + D1: ExchangeData + Hash, + R: Semigroup + ExchangeData, +{ + /// Consolidates the collection if `must_consolidate` is `true` and leaves it + /// untouched otherwise. + fn mz_consolidate_if(&self, must_consolidate: bool, name: &str) -> Self + where + Tr: Trace + TraceReader + 'static, + Tr::Batch: Batch, + Arranged>: ArrangementSize; + + /// Consolidates the collection. + fn mz_consolidate(&self, name: &str) -> Self + where + Tr: Trace + TraceReader + 'static, + Tr::Batch: Batch, + Arranged>: ArrangementSize; +} + +impl ConsolidateExt for Collection +where + G: Scope, + G::Timestamp: Lattice + Data, + D1: ExchangeData + Hash + Columnation, + R: Semigroup + ExchangeData + Columnation, +{ + fn mz_consolidate_if(&self, must_consolidate: bool, name: &str) -> Self + where + Tr: Trace + TraceReader + 'static, + Tr::Batch: Batch, + Arranged>: ArrangementSize, + { + if must_consolidate { + self.mz_consolidate(name) + } else { + self.clone() + } + } + + fn mz_consolidate(&self, name: &str) -> Self + where + Tr: Trace + TraceReader + 'static, + Tr::Batch: Batch, + Arranged>: ArrangementSize, + { + // We employ AHash below instead of the default hasher in DD to obtain + // a better distribution of data to workers. AHash claims empirically + // both speed and high quality, according to + // https://github.com/tkaitchuck/aHash/blob/master/compare/readme.md. + // TODO(vmarcos): Consider here if it is worth it to spend the time to + // implement twisted tabulation hashing as proposed in Mihai Patrascu, + // Mikkel Thorup: Twisted Tabulation Hashing. SODA 2013: 209-228, available + // at https://epubs.siam.org/doi/epdf/10.1137/1.9781611973105.16. The latter + // would provide good bounds for balls-into-bins problems when the number of + // bins is small (as is our case), so we'd have a theoretical guarantee. + // NOTE: We fix the seeds of a RandomState instance explicity with the same + // seeds that would be given by `AHash` via ahash::AHasher::default() so as + // to avoid a different selection due to compile-time features being differently + // selected in other dependencies using `AHash` vis-à-vis cargo's strategy + // of unioning features. + // NOTE: Depending on target features, we may end up employing the fallback + // hasher of `AHash`, but it should be sufficient for our needs. + let random_state = ahash::RandomState::with_seeds( + 0x243f_6a88_85a3_08d3, + 0x1319_8a2e_0370_7344, + 0xa409_3822_299f_31d0, + 0x082e_fa98_ec4e_6c89, + ); + let exchange = Exchange::new(move |update: &((D1, _), G::Timestamp, R)| { + let data = &(update.0).0; + let mut h = random_state.build_hasher(); + data.hash(&mut h); + h.finish() + }); + self.map(|k| (k, ())) + .mz_arrange_core::<_, Tr>(exchange, name) + .as_collection(|d: &D1, _| d.clone()) + } +} diff --git a/src/compute/src/extensions/mod.rs b/src/compute/src/extensions/mod.rs new file mode 100644 index 000000000000..72a9abc2aadd --- /dev/null +++ b/src/compute/src/extensions/mod.rs @@ -0,0 +1,16 @@ +// 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. + +#![warn(missing_docs)] + +//! Operator extensions to Timely and Differential + +pub(crate) mod arrange; +pub(crate) mod collection; +pub(crate) mod reduce; diff --git a/src/timely-util/src/reduce.rs b/src/compute/src/extensions/reduce.rs similarity index 67% rename from src/timely-util/src/reduce.rs rename to src/compute/src/extensions/reduce.rs index 775a545338da..8db4f5376f23 100644 --- a/src/timely-util/src/reduce.rs +++ b/src/compute/src/extensions/reduce.rs @@ -21,6 +21,41 @@ use differential_dataflow::trace::{Batch, Trace, TraceReader}; use differential_dataflow::Data; use timely::dataflow::Scope; +use crate::extensions::arrange::ArrangementSize; + +/// Extension trait for the `reduce_abelian` differential dataflow method. +pub(crate) trait MzReduce: + ReduceCore +where + G::Timestamp: Lattice + Ord, +{ + /// Applies `reduce` to arranged data, and returns an arrangement of output data. + fn mz_reduce_abelian(&self, name: &str, logic: L) -> Arranged> + where + T2: Trace + TraceReader + 'static, + T2::Val: Data, + T2::R: Abelian, + T2::Batch: Batch, + L: FnMut(&K, &[(&V, R)], &mut Vec<(T2::Val, T2::R)>) + 'static, + Arranged>: ArrangementSize, + { + // Allow access to `reduce_abelian` since we're within Mz's wrapper. + #[allow(clippy::disallowed_methods)] + self.reduce_abelian::<_, T2>(name, logic) + .log_arrangement_size() + } +} + +impl MzReduce for Arranged +where + G::Timestamp: Lattice + Ord, + G: Scope, + K: Data, + V: Data, + R: Semigroup, + T1: TraceReader + Clone + 'static, +{ +} /// Extension trait for `ReduceCore`, currently providing a reduction based /// on an operator-pair approach. pub trait ReduceExt @@ -49,7 +84,9 @@ where T2::Val: Data, T2::R: Abelian, T2::Batch: Batch, - L2: FnMut(&K, &[(&V, R)], &mut Vec<(T2::Val, T2::R)>) + 'static; + L2: FnMut(&K, &[(&V, R)], &mut Vec<(T2::Val, T2::R)>) + 'static, + Arranged>: ArrangementSize, + Arranged>: ArrangementSize; } impl ReduceExt for Arranged @@ -75,9 +112,11 @@ where T2::R: Abelian, T2::Batch: Batch, L2: FnMut(&K, &[(&V, R)], &mut Vec<(T2::Val, T2::R)>) + 'static, + Arranged>: ArrangementSize, + Arranged>: ArrangementSize, { - let arranged1 = self.reduce_abelian::(name1, logic1); - let arranged2 = self.reduce_abelian::(name2, logic2); + let arranged1 = self.mz_reduce_abelian::(name1, logic1); + let arranged2 = self.mz_reduce_abelian::(name2, logic2); (arranged1, arranged2) } } diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index 72c0f6916223..657a4e54f3b0 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -78,6 +78,7 @@ pub(crate) mod arrangement; pub mod compute_state; +pub(crate) mod extensions; pub(crate) mod logging; pub(crate) mod metrics; pub(crate) mod render; diff --git a/src/compute/src/logging/compute.rs b/src/compute/src/logging/compute.rs index 0a64a348b679..50f92bd956fe 100644 --- a/src/compute/src/logging/compute.rs +++ b/src/compute/src/logging/compute.rs @@ -10,12 +10,12 @@ //! Logging dataflows for events generated by clusterd. use std::any::Any; -use std::collections::{BTreeMap, VecDeque}; +use std::cell::RefCell; +use std::collections::{BTreeMap, BTreeSet, VecDeque}; use std::rc::Rc; use std::time::Duration; use differential_dataflow::collection::AsCollection; -use differential_dataflow::operators::arrange::arrangement::Arrange; use differential_dataflow::operators::arrange::Arranged; use differential_dataflow::trace::TraceReader; use mz_expr::{permutation_for_arrangement, MirScalarExpr}; @@ -30,11 +30,14 @@ use timely::dataflow::operators::generic::builder_rc::OperatorBuilder; use timely::dataflow::operators::{Filter, InspectCore}; use timely::dataflow::{Scope, StreamCore}; use timely::logging::WorkerIdentifier; +use timely::scheduling::Scheduler; +use timely::worker::Worker; use timely::Container; use tracing::error; use uuid::Uuid; -use crate::logging::{ComputeLog, EventQueue, LogVariant}; +use crate::extensions::arrange::MzArrange; +use crate::logging::{ComputeLog, EventQueue, LogVariant, SharedLoggingState}; use crate::typedefs::{KeysValsHandle, RowSpine}; /// Type alias for a logger of compute events. @@ -77,6 +80,44 @@ pub enum ComputeEvent { time: Timestamp, diff: i8, }, + /// Arrangement heap size update + ArrangementHeapSize { + /// Operator index + operator: usize, + /// Delta of the heap size in bytes of the arrangement. + delta_size: isize, + }, + /// Arrangement heap size update + ArrangementHeapCapacity { + /// Operator index + operator: usize, + /// Delta of the heap capacity in bytes of the arrangement. + delta_capacity: isize, + }, + /// Arrangement heap size update + ArrangementHeapAllocations { + /// Operator index + operator: usize, + /// Delta of distinct heap allocations backing the arrangement. + delta_allocations: isize, + }, + /// Arrangement size operator address + ArrangementHeapSizeOperator { + /// Operator index + operator: usize, + /// The address of the operator. + address: Vec, + }, + /// Arrangement size operator dropped + ArrangementHeapSizeOperatorDrop { + /// Operator index + operator: usize, + }, + /// All operators of a dataflow have shut down. + DataflowShutdown { + /// Timely worker index of the dataflow. + dataflow_index: usize, + }, } /// A logged peek event. @@ -105,13 +146,15 @@ impl Peek { /// * `event_queue`: The source to read compute log events from. /// /// Returns a map from log variant to a tuple of a trace handle and a dataflow drop token. -pub(super) fn construct( +pub(super) fn construct( worker: &mut timely::worker::Worker, config: &mz_compute_client::logging::LoggingConfig, event_queue: EventQueue, + shared_state: Rc>, ) -> BTreeMap)> { let logging_interval_ms = std::cmp::max(1, config.interval.as_millis()); let worker_id = worker.index(); + let worker2 = worker.clone(); worker.dataflow_named("Dataflow: compute logging", move |scope| { let (mut logs, token) = Some(event_queue.link).mz_replay( @@ -138,8 +181,13 @@ pub(super) fn construct( let (mut frontier_delay_out, frontier_delay) = demux.new_output(); let (mut peek_out, peek) = demux.new_output(); let (mut peek_duration_out, peek_duration) = demux.new_output(); + let (mut shutdown_duration_out, shutdown_duration) = demux.new_output(); + let (mut arrangement_heap_size_out, arrangement_heap_size) = demux.new_output(); + let (mut arrangement_heap_capacity_out, arrangement_heap_capacity) = demux.new_output(); + let (mut arrangement_heap_allocations_out, arrangement_heap_allocations) = + demux.new_output(); - let mut demux_state = DemuxState::default(); + let mut demux_state = DemuxState::new(worker2); let mut demux_buffer = Vec::new(); demux.build(move |_capability| { move |_frontiers| { @@ -150,6 +198,10 @@ pub(super) fn construct( let mut frontier_delay = frontier_delay_out.activate(); let mut peek = peek_out.activate(); let mut peek_duration = peek_duration_out.activate(); + let mut shutdown_duration = shutdown_duration_out.activate(); + let mut arrangement_heap_size = arrangement_heap_size_out.activate(); + let mut arrangement_heap_capacity = arrangement_heap_capacity_out.activate(); + let mut arrangement_heap_allocations = arrangement_heap_allocations_out.activate(); input.for_each(|cap, data| { data.swap(&mut demux_buffer); @@ -162,6 +214,10 @@ pub(super) fn construct( frontier_delay: frontier_delay.session(&cap), peek: peek.session(&cap), peek_duration: peek_duration.session(&cap), + shutdown_duration: shutdown_duration.session(&cap), + arrangement_heap_size: arrangement_heap_size.session(&cap), + arrangement_heap_capacity: arrangement_heap_capacity.session(&cap), + arrangement_heap_allocations: arrangement_heap_allocations.session(&cap), }; for (time, logger_id, event) in demux_buffer.drain(..) { @@ -172,6 +228,7 @@ pub(super) fn construct( DemuxHandler { state: &mut demux_state, + shared_state: &mut shared_state.borrow_mut(), output: &mut output_sessions, logging_interval_ms, time, @@ -231,38 +288,54 @@ pub(super) fn construct( let peek_duration = peek_duration.as_collection().map(move |bucket| { Row::pack_slice(&[ Datum::UInt64(u64::cast_from(worker_id)), - Datum::UInt64(bucket.try_into().expect("pow too big")), + Datum::UInt64(bucket.try_into().expect("bucket too big")), ]) }); + let shutdown_duration = shutdown_duration.as_collection().map(move |bucket| { + Row::pack_slice(&[ + Datum::UInt64(u64::cast_from(worker_id)), + Datum::UInt64(bucket.try_into().expect("bucket too big")), + ]) + }); + + let arrangement_heap_datum_to_row = move |ArrangementHeapDatum { operator_id }| { + Row::pack_slice(&[ + Datum::UInt64(operator_id.try_into().expect("operator_id too big")), + Datum::UInt64(u64::cast_from(worker_id)), + ]) + }; + + let arrangement_heap_size = arrangement_heap_size + .as_collection() + .map(arrangement_heap_datum_to_row.clone()); + let arrangement_heap_capacity = arrangement_heap_capacity + .as_collection() + .map(arrangement_heap_datum_to_row.clone()); + + let arrangement_heap_allocations = arrangement_heap_allocations + .as_collection() + .map(arrangement_heap_datum_to_row); + + use ComputeLog::*; let logs = [ - ( - LogVariant::Compute(ComputeLog::DataflowCurrent), - dataflow_current, - ), - ( - LogVariant::Compute(ComputeLog::DataflowDependency), - dataflow_dependency, - ), - ( - LogVariant::Compute(ComputeLog::FrontierCurrent), - frontier_current, - ), - ( - LogVariant::Compute(ComputeLog::ImportFrontierCurrent), - import_frontier_current, - ), - ( - LogVariant::Compute(ComputeLog::FrontierDelay), - frontier_delay, - ), - (LogVariant::Compute(ComputeLog::PeekCurrent), peek_current), - (LogVariant::Compute(ComputeLog::PeekDuration), peek_duration), + (DataflowCurrent, dataflow_current), + (DataflowDependency, dataflow_dependency), + (FrontierCurrent, frontier_current), + (ImportFrontierCurrent, import_frontier_current), + (FrontierDelay, frontier_delay), + (PeekCurrent, peek_current), + (PeekDuration, peek_duration), + (ShutdownDuration, shutdown_duration), + (ArrangementHeapSize, arrangement_heap_size), + (ArrangementHeapCapacity, arrangement_heap_capacity), + (ArrangementHeapAllocations, arrangement_heap_allocations), ]; // Build the output arrangements. let mut traces = BTreeMap::new(); for (variant, collection) in logs { + let variant = LogVariant::Compute(variant); if config.index_logs.contains_key(&variant) { let key = variant.index_by(); let (_, value) = permutation_for_arrangement( @@ -285,7 +358,7 @@ pub(super) fn construct( (row_key, row_val) } }) - .arrange_named::>(&format!("ArrangeByKey {:?}", variant)) + .mz_arrange::>(&format!("ArrangeByKey {:?}", variant)) .trace; traces.insert(variant.clone(), (trace, Rc::clone(&token))); } @@ -296,14 +369,38 @@ pub(super) fn construct( } /// State maintained by the demux operator. -#[derive(Default)] -struct DemuxState { +struct DemuxState { + /// The worker hosting this operator. + worker: Worker, /// Maps dataflow exports to dataflow IDs. export_dataflows: BTreeMap, /// Maps dataflow exports to their imports and frontier delay tracking state. export_imports: BTreeMap>, - /// Maps pending peeks to their installation time (in ns). + /// Maps live dataflows to counts of their exports. + dataflow_export_counts: BTreeMap, + /// Maps dropped dataflows to their drop time. + dataflow_drop_times: BTreeMap, + /// Contains dataflows that have shut down but not yet been dropped. + shutdown_dataflows: BTreeSet, + /// Maps pending peeks to their installation time. peek_stash: BTreeMap, + /// Arrangement size stash + arrangement_size: BTreeMap, +} + +impl DemuxState { + fn new(worker: Worker) -> Self { + Self { + worker, + export_dataflows: Default::default(), + export_imports: Default::default(), + dataflow_export_counts: Default::default(), + dataflow_drop_times: Default::default(), + shutdown_dataflows: Default::default(), + peek_stash: Default::default(), + arrangement_size: Default::default(), + } + } } /// State for tracking import-export frontier lag. @@ -312,7 +409,7 @@ struct FrontierDelayState { /// A list of input timestamps that have appeared on the input /// frontier, but that the output frontier has not yet advanced beyond, /// and the time at which we were informed of their availability. - time_deque: VecDeque<(mz_repr::Timestamp, Duration)>, + time_deque: VecDeque<(Timestamp, Duration)>, /// A histogram of emitted delays (bucket size to bucket_count). delay_map: BTreeMap, } @@ -330,6 +427,10 @@ struct DemuxOutput<'a> { frontier_delay: OutputSession<'a, FrontierDelayDatum>, peek: OutputSession<'a, Peek>, peek_duration: OutputSession<'a, u128>, + shutdown_duration: OutputSession<'a, u128>, + arrangement_heap_size: OutputSession<'a, ArrangementHeapDatum>, + arrangement_heap_capacity: OutputSession<'a, ArrangementHeapDatum>, + arrangement_heap_allocations: OutputSession<'a, ArrangementHeapDatum>, } #[derive(Clone)] @@ -364,10 +465,24 @@ struct FrontierDelayDatum { delay_pow: u128, } +#[derive(Clone)] +struct ArrangementHeapDatum { + operator_id: usize, +} + +#[derive(Default)] +struct ArrangementSizeState { + size: isize, + capacity: isize, + count: isize, +} + /// Event handler of the demux operator. -struct DemuxHandler<'a, 'b> { +struct DemuxHandler<'a, 'b, A: Allocate + 'static> { /// State kept by the demux operator. - state: &'a mut DemuxState, + state: &'a mut DemuxState, + /// State shared across log receivers. + shared_state: &'a mut SharedLoggingState, /// Demux output sessions. output: &'a mut DemuxOutput<'b>, /// The logging interval specifying the time granularity for the updates. @@ -376,7 +491,7 @@ struct DemuxHandler<'a, 'b> { time: Duration, } -impl DemuxHandler<'_, '_> { +impl DemuxHandler<'_, '_, A> { /// Return the timestamp associated with the current event, based on the event time and the /// logging interval. fn ts(&self) -> Timestamp { @@ -406,6 +521,25 @@ impl DemuxHandler<'_, '_> { time, diff, } => self.handle_import_frontier(import_id, export_id, time, diff), + ArrangementHeapSize { + operator, + delta_size: size, + } => self.handle_arrangement_heap_size(operator, size), + ArrangementHeapCapacity { + operator, + delta_capacity: capacity, + } => self.handle_arrangement_heap_capacity(operator, capacity), + ArrangementHeapAllocations { + operator, + delta_allocations: allocations, + } => self.handle_arrangement_heap_allocations(operator, allocations), + ArrangementHeapSizeOperator { operator, address } => { + self.handle_arrangement_heap_size_operator(operator, address) + } + ArrangementHeapSizeOperatorDrop { operator } => { + self.handle_arrangement_heap_size_operator_dropped(operator) + } + DataflowShutdown { dataflow_index } => self.handle_dataflow_shutdown(dataflow_index), } } @@ -416,6 +550,11 @@ impl DemuxHandler<'_, '_> { self.state.export_dataflows.insert(id, dataflow_id); self.state.export_imports.insert(id, BTreeMap::new()); + *self + .state + .dataflow_export_counts + .entry(dataflow_id) + .or_default() += 1; } fn handle_export_dropped(&mut self, id: GlobalId) { @@ -423,6 +562,18 @@ impl DemuxHandler<'_, '_> { if let Some(dataflow_id) = self.state.export_dataflows.remove(&id) { let datum = ExportDatum { id, dataflow_id }; self.output.export.give((datum, ts, -1)); + + match self.state.dataflow_export_counts.get_mut(&dataflow_id) { + entry @ Some(0) | entry @ None => { + error!( + export = ?id, + dataflow = ?dataflow_id, + "invalid dataflow_export_counts entry at time of export drop: {entry:?}", + ); + } + Some(1) => self.handle_dataflow_dropped(dataflow_id), + Some(count) => *count -= 1, + } } else { error!( export = ?id, @@ -456,6 +607,38 @@ impl DemuxHandler<'_, '_> { } } + fn handle_dataflow_dropped(&mut self, id: usize) { + self.state.dataflow_export_counts.remove(&id); + + if self.state.shutdown_dataflows.remove(&id) { + // Dataflow has already shut down before it was dropped. + self.output.shutdown_duration.give((0, self.ts(), 1)); + } else { + // Dataflow has not yet shut down. + let existing = self.state.dataflow_drop_times.insert(id, self.time); + if existing.is_some() { + error!(dataflow = ?id, "dataflow already dropped"); + } + } + } + + fn handle_dataflow_shutdown(&mut self, id: usize) { + if let Some(start) = self.state.dataflow_drop_times.remove(&id) { + // Dataflow has alredy been dropped. + let elapsed_ns = self.time.saturating_sub(start).as_nanos(); + let elapsed_pow = elapsed_ns.next_power_of_two(); + self.output + .shutdown_duration + .give((elapsed_pow, self.ts(), 1)); + } else { + // Dataflow has not yet been dropped. + let was_new = self.state.shutdown_dataflows.insert(id); + if !was_new { + error!(dataflow = ?id, "dataflow already shutdown"); + } + } + } + fn handle_export_dependency(&mut self, export_id: GlobalId, import_id: GlobalId) { let ts = self.ts(); let datum = DependencyDatum { @@ -584,6 +767,82 @@ impl DemuxHandler<'_, '_> { } } } + + /// Update the allocation size for an arrangement. + fn handle_arrangement_heap_size(&mut self, operator_id: usize, size: isize) { + let ts = self.ts(); + let Some(state) = self.state.arrangement_size.get_mut(&operator_id) else {return}; + + let datum = ArrangementHeapDatum { operator_id }; + self.output + .arrangement_heap_size + .give((datum, ts, Diff::cast_from(size))); + + state.size += size; + } + + /// Update the allocation capacity for an arrangement. + fn handle_arrangement_heap_capacity(&mut self, operator_id: usize, capacity: isize) { + let ts = self.ts(); + let Some(state) = self.state.arrangement_size.get_mut(&operator_id) else {return}; + + let datum = ArrangementHeapDatum { operator_id }; + self.output + .arrangement_heap_capacity + .give((datum, ts, Diff::cast_from(capacity))); + + state.capacity += capacity; + } + + /// Update the allocation count for an arrangement. + fn handle_arrangement_heap_allocations(&mut self, operator_id: usize, count: isize) { + let ts = self.ts(); + let Some(state) = self.state.arrangement_size.get_mut(&operator_id) else {return}; + + let datum = ArrangementHeapDatum { operator_id }; + self.output + .arrangement_heap_allocations + .give((datum, ts, Diff::cast_from(count))); + + state.count += count; + } + + /// Indicate that a new arrangement exists, start maintaining the heap size state. + fn handle_arrangement_heap_size_operator(&mut self, operator_id: usize, address: Vec) { + let activator = self.state.worker.activator_for(&address); + self.state + .arrangement_size + .insert(operator_id, Default::default()); + self.shared_state + .arrangement_size_activators + .insert(operator_id, activator); + } + + /// Indicate that an arrangement has been dropped and we can cleanup the heap size state. + fn handle_arrangement_heap_size_operator_dropped(&mut self, operator_id: usize) { + if let Some(state) = self.state.arrangement_size.remove(&operator_id) { + let ts = self.ts(); + let datum = ArrangementHeapDatum { operator_id }; + self.output.arrangement_heap_size.give(( + datum.clone(), + ts, + -Diff::cast_from(state.size), + )); + self.output.arrangement_heap_capacity.give(( + datum.clone(), + ts, + -Diff::cast_from(state.capacity), + )); + self.output.arrangement_heap_allocations.give(( + datum, + ts, + -Diff::cast_from(state.count), + )); + } + self.shared_state + .arrangement_size_activators + .remove(&operator_id); + } } pub(crate) trait LogImportFrontiers { diff --git a/src/compute/src/logging/differential.rs b/src/compute/src/logging/differential.rs index 12a61f77d960..94d4bb164c0a 100644 --- a/src/compute/src/logging/differential.rs +++ b/src/compute/src/logging/differential.rs @@ -10,6 +10,7 @@ //! Logging dataflows for events generated by differential dataflow. use std::any::Any; +use std::cell::RefCell; use std::collections::BTreeMap; use std::rc::Rc; use std::time::Duration; @@ -18,19 +19,20 @@ use differential_dataflow::collection::AsCollection; use differential_dataflow::logging::{ BatchEvent, DifferentialEvent, DropEvent, MergeEvent, TraceShare, }; -use differential_dataflow::operators::arrange::arrangement::Arrange; use mz_expr::{permutation_for_arrangement, MirScalarExpr}; use mz_ore::cast::CastFrom; use mz_repr::{Datum, DatumVec, Diff, Row, Timestamp}; use mz_timely_util::buffer::ConsolidateBuffer; use mz_timely_util::replay::MzReplay; use timely::communication::Allocate; -use timely::dataflow::channels::pact::Pipeline; +use timely::dataflow::channels::pact::{Exchange, Pipeline}; use timely::dataflow::channels::pushers::Tee; use timely::dataflow::operators::generic::builder_rc::OperatorBuilder; use timely::dataflow::operators::{Filter, InputCapability}; -use crate::logging::{DifferentialLog, EventQueue, LogVariant}; +use crate::extensions::arrange::MzArrange; +use crate::logging::compute::ComputeEvent; +use crate::logging::{DifferentialLog, EventQueue, LogVariant, SharedLoggingState}; use crate::typedefs::{KeysValsHandle, RowSpine}; /// Constructs the logging dataflow for differential logs. @@ -45,6 +47,7 @@ pub(super) fn construct( worker: &mut timely::worker::Worker, config: &mz_compute_client::logging::LoggingConfig, event_queue: EventQueue, + shared_state: Rc>, ) -> BTreeMap)> { let logging_interval_ms = std::cmp::max(1, config.interval.as_millis()); let worker_id = worker.index(); @@ -73,6 +76,7 @@ pub(super) fn construct( let (mut sharing_out, sharing) = demux.new_output(); let mut demux_buffer = Vec::new(); + let mut demux_state = Default::default(); demux.build(move |_capability| { move |_frontiers| { let mut batches = batches_out.activate(); @@ -95,10 +99,12 @@ pub(super) fn construct( assert_eq!(logger_id, worker_id); DemuxHandler { + state: &mut demux_state, output: &mut output_buffers, logging_interval_ms, time, cap: &cap, + shared_state: &mut shared_state.borrow_mut(), } .handle(event); } @@ -119,28 +125,32 @@ pub(super) fn construct( Datum::UInt64(u64::cast_from(worker_id)), ]) }); - let sharing = sharing.as_collection().map(move |op| { + + let sharing = sharing + .as_collection() + .mz_arrange_core::<_, RowSpine<_, _, _, _>>( + Exchange::new(move |_| u64::cast_from(worker_id)), + "PreArrange Differential sharing", + ); + + let sharing = sharing.as_collection(move |op, ()| { Row::pack_slice(&[ - Datum::UInt64(u64::cast_from(op)), + Datum::UInt64(u64::cast_from(*op)), Datum::UInt64(u64::cast_from(worker_id)), ]) }); + use DifferentialLog::*; let logs = [ - ( - LogVariant::Differential(DifferentialLog::ArrangementBatches), - arrangement_batches, - ), - ( - LogVariant::Differential(DifferentialLog::ArrangementRecords), - arrangement_records, - ), - (LogVariant::Differential(DifferentialLog::Sharing), sharing), + (ArrangementBatches, arrangement_batches), + (ArrangementRecords, arrangement_records), + (Sharing, sharing), ]; // Build the output arrangements. let mut traces = BTreeMap::new(); for (variant, collection) in logs { + let variant = LogVariant::Differential(variant); if config.index_logs.contains_key(&variant) { let key = variant.index_by(); let (_, value) = permutation_for_arrangement( @@ -163,7 +173,7 @@ pub(super) fn construct( (row_key, row_val) } }) - .arrange_named::>(&format!("ArrangeByKey {:?}", variant)) + .mz_arrange::>(&format!("ArrangeByKey {:?}", variant)) .trace; traces.insert(variant.clone(), (trace, Rc::clone(&token))); } @@ -180,11 +190,20 @@ type OutputBuffer<'a, 'b, D> = ConsolidateBuffer<'a, 'b, Timestamp, D, Diff, Pus struct DemuxOutput<'a, 'b> { batches: OutputBuffer<'a, 'b, usize>, records: OutputBuffer<'a, 'b, usize>, - sharing: OutputBuffer<'a, 'b, usize>, + sharing: OutputBuffer<'a, 'b, (usize, ())>, +} + +/// State maintained by the demux operator. +#[derive(Default)] +struct DemuxState { + /// Arrangement trace sharing + sharing: BTreeMap, } /// Event handler of the demux operator. struct DemuxHandler<'a, 'b, 'c> { + /// State kept by the demux operator + state: &'a mut DemuxState, /// Demux output buffers. output: &'a mut DemuxOutput<'b, 'c>, /// The logging interval specifying the time granularity for the updates. @@ -193,6 +212,8 @@ struct DemuxHandler<'a, 'b, 'c> { time: Duration, /// A capability usable for emitting outputs. cap: &'a InputCapability, + /// State shared across log receivers. + shared_state: &'a mut SharedLoggingState, } impl DemuxHandler<'_, '_, '_> { @@ -225,6 +246,7 @@ impl DemuxHandler<'_, '_, '_> { let diff = Diff::try_from(event.length).expect("must fit"); self.output.records.give(self.cap, (op, ts, diff)); + self.notify_arrangement_size(op); } fn handle_merge(&mut self, event: MergeEvent) { @@ -239,6 +261,7 @@ impl DemuxHandler<'_, '_, '_> { if diff != 0 { self.output.records.give(self.cap, (op, ts, diff)); } + self.notify_arrangement_size(op); } fn handle_drop(&mut self, event: DropEvent) { @@ -250,6 +273,7 @@ impl DemuxHandler<'_, '_, '_> { if diff != 0 { self.output.records.give(self.cap, (op, ts, diff)); } + self.notify_arrangement_size(op); } fn handle_trace_share(&mut self, event: TraceShare) { @@ -257,6 +281,27 @@ impl DemuxHandler<'_, '_, '_> { let op = event.operator; let diff = Diff::cast_from(event.diff); debug_assert_ne!(diff, 0); - self.output.sharing.give(self.cap, (op, ts, diff)); + self.output.sharing.give(self.cap, ((op, ()), ts, diff)); + + if let Some(logger) = &mut self.shared_state.compute_logger { + let sharing = self.state.sharing.entry(op).or_default(); + *sharing = (i64::try_from(*sharing).expect("must fit") + diff) + .try_into() + .expect("under/overflow"); + if *sharing == 0 { + self.state.sharing.remove(&op); + logger.log(ComputeEvent::ArrangementHeapSizeOperatorDrop { operator: op }); + } + } + } + + fn notify_arrangement_size(&self, operator: usize) { + // While every arrangement should have a corresponding arrangement size operator, + // we have no guarantee that it already/still exists. Otherwise we could print a warning + // here, but it's difficult to implement without maintaining state for a longer period than + // while the arrangement actually exists. + if let Some(activator) = self.shared_state.arrangement_size_activators.get(&operator) { + activator.activate(); + } } } diff --git a/src/compute/src/logging/initialize.rs b/src/compute/src/logging/initialize.rs index 63b6e6faeeb8..0a84e90d73fc 100644 --- a/src/compute/src/logging/initialize.rs +++ b/src/compute/src/logging/initialize.rs @@ -5,11 +5,12 @@ //! Initialization of logging dataflows. +use std::cell::RefCell; use std::collections::BTreeMap; +use std::rc::Rc; use std::time::{Duration, Instant}; use differential_dataflow::logging::DifferentialEvent; -use differential_dataflow::operators::arrange::Arrange; use differential_dataflow::Collection; use mz_compute_client::logging::{LogVariant, LoggingConfig}; use mz_repr::{Diff, Timestamp}; @@ -20,15 +21,16 @@ use timely::logging::{Logger, TimelyEvent}; use timely::progress::reachability::logging::TrackerEvent; use crate::arrangement::manager::TraceBundle; +use crate::extensions::arrange::{KeyCollection, MzArrange}; use crate::logging::compute::ComputeEvent; use crate::logging::reachability::ReachabilityEvent; -use crate::logging::{BatchLogger, EventQueue}; +use crate::logging::{BatchLogger, EventQueue, SharedLoggingState}; /// Initialize logging dataflows. /// /// Returns a logger for compute events, and for each `LogVariant` a trace bundle usable for /// retrieving logged records. -pub fn initialize( +pub fn initialize( worker: &mut timely::worker::Worker, config: &LoggingConfig, ) -> (super::compute::Logger, BTreeMap) { @@ -54,6 +56,7 @@ pub fn initialize( r_event_queue: EventQueue::new("r"), d_event_queue: EventQueue::new("d"), c_event_queue: EventQueue::new("c"), + shared_state: Default::default(), }; // Depending on whether we should log the creation of the logging dataflows, we register the @@ -81,15 +84,17 @@ struct LoggingContext<'a, A: Allocate> { r_event_queue: EventQueue, d_event_queue: EventQueue, c_event_queue: EventQueue, + shared_state: Rc>, } -impl LoggingContext<'_, A> { +impl LoggingContext<'_, A> { fn construct_dataflows(&mut self) -> BTreeMap { let mut traces = BTreeMap::new(); traces.extend(super::timely::construct( self.worker, self.config, self.t_event_queue.clone(), + Rc::clone(&self.shared_state), )); traces.extend(super::reachability::construct( self.worker, @@ -100,19 +105,21 @@ impl LoggingContext<'_, A> { self.worker, self.config, self.d_event_queue.clone(), + Rc::clone(&self.shared_state), )); traces.extend(super::compute::construct( self.worker, self.config, self.c_event_queue.clone(), + Rc::clone(&self.shared_state), )); let errs = self .worker .dataflow_named("Dataflow: logging errors", |scope| { - Collection::<_, DataflowError, Diff>::empty(scope) - .arrange_named("Arrange logging err") - .trace + let collection: KeyCollection<_, DataflowError, Diff> = + Collection::empty(scope).into(); + collection.mz_arrange("Arrange logging err").trace }); traces @@ -125,20 +132,18 @@ impl LoggingContext<'_, A> { } fn register_loggers(&self) { - self.worker - .log_register() - .insert_logger("timely", self.simple_logger(self.t_event_queue.clone())); - self.worker - .log_register() - .insert_logger("timely/reachability", self.reachability_logger()); - self.worker.log_register().insert_logger( - "differential/arrange", - self.simple_logger(self.d_event_queue.clone()), - ); - self.worker.log_register().insert_logger( - "materialize/compute", - self.simple_logger(self.c_event_queue.clone()), - ); + 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); + register.insert_logger("differential/arrange", d_logger); + register.insert_logger("materialize/compute", c_logger.clone()); + + self.shared_state.borrow_mut().compute_logger = Some(c_logger); } fn simple_logger(&self, event_queue: EventQueue) -> Logger { diff --git a/src/compute/src/logging/mod.rs b/src/compute/src/logging/mod.rs index dcf7f269a2cc..3f82b47f3e7c 100644 --- a/src/compute/src/logging/mod.rs +++ b/src/compute/src/logging/mod.rs @@ -15,16 +15,20 @@ mod initialize; mod reachability; mod timely; +use std::collections::BTreeMap; use std::rc::Rc; use std::time::Duration; use ::timely::dataflow::operators::capture::{Event, EventLink, EventPusher}; use ::timely::logging::WorkerIdentifier; use ::timely::progress::Timestamp as TimelyTimestamp; +use ::timely::scheduling::Activator; use mz_compute_client::logging::{ComputeLog, DifferentialLog, LogVariant, TimelyLog}; use mz_repr::Timestamp; use mz_timely_util::activator::RcActivator; +use crate::logging::compute::Logger as ComputeLogger; + pub use crate::logging::initialize::initialize; /// Logs events as a timely stream, with progress statements. @@ -142,3 +146,12 @@ impl EventQueue { } } } + +/// State shared between different logging dataflows. +#[derive(Default)] +struct SharedLoggingState { + /// Activators for arrangement heap size operators. + arrangement_size_activators: BTreeMap, + /// Shared compute logger. + compute_logger: Option, +} diff --git a/src/compute/src/logging/reachability.rs b/src/compute/src/logging/reachability.rs index 5e029cb6e24c..58e570781c58 100644 --- a/src/compute/src/logging/reachability.rs +++ b/src/compute/src/logging/reachability.rs @@ -14,7 +14,6 @@ use std::collections::BTreeMap; use std::convert::TryInto; use std::rc::Rc; -use differential_dataflow::operators::arrange::arrangement::Arrange; use differential_dataflow::AsCollection; use mz_compute_client::logging::LoggingConfig; use mz_expr::{permutation_for_arrangement, MirScalarExpr}; @@ -27,6 +26,7 @@ use timely::communication::Allocate; use timely::dataflow::channels::pact::Exchange; use timely::dataflow::operators::Filter; +use crate::extensions::arrange::MzArrange; use crate::logging::{EventQueue, LogVariant, TimelyLog}; use crate::typedefs::{KeysValsHandle, RowSpine}; @@ -95,14 +95,8 @@ pub(super) fn construct( .try_into() .expect("must fit"); for (source, port, update_type, ts, diff) in massaged { - updates_session.give( - &cap, - ( - (update_type, addr.clone(), source, port, worker, ts), - time_ms, - diff, - ), - ); + let datum = (update_type, addr.clone(), source, port, worker, ts); + updates_session.give(&cap, ((datum, ()), time_ms, diff)); } } }); @@ -111,7 +105,7 @@ pub(super) fn construct( let updates = updates .as_collection() - .arrange_core::<_, RowSpine<_, _, _, _>>( + .mz_arrange_core::<_, RowSpine<_, _, _, _>>( Exchange::new(|(((_, _, _, _, w, _), ()), _, _)| u64::cast_from(*w)), "PreArrange Timely reachability", ); @@ -154,7 +148,7 @@ pub(super) fn construct( ); let trace = updates - .arrange_named::>(&format!("Arrange {:?}", variant)) + .mz_arrange::>(&format!("Arrange {:?}", variant)) .trace; result.insert(variant.clone(), (trace, Rc::clone(&token))); } diff --git a/src/compute/src/logging/timely.rs b/src/compute/src/logging/timely.rs index b745ae952faf..48fc0157c462 100644 --- a/src/compute/src/logging/timely.rs +++ b/src/compute/src/logging/timely.rs @@ -10,12 +10,12 @@ //! Logging dataflows for events generated by timely dataflow. use std::any::Any; +use std::cell::RefCell; use std::collections::BTreeMap; use std::rc::Rc; use std::time::Duration; use differential_dataflow::collection::AsCollection; -use differential_dataflow::operators::arrange::arrangement::Arrange; use mz_compute_client::logging::LoggingConfig; use mz_expr::{permutation_for_arrangement, MirScalarExpr}; use mz_ore::cast::CastFrom; @@ -35,7 +35,9 @@ use timely::logging::{ }; use tracing::error; -use crate::logging::{EventQueue, LogVariant, TimelyLog}; +use crate::extensions::arrange::MzArrange; +use crate::logging::compute::ComputeEvent; +use crate::logging::{EventQueue, LogVariant, SharedLoggingState, TimelyLog}; use crate::typedefs::{KeysValsHandle, RowSpine}; /// Constructs the logging dataflow for timely logs. @@ -50,6 +52,7 @@ pub(super) fn construct( worker: &mut timely::worker::Worker, config: &LoggingConfig, event_queue: EventQueue, + shared_state: Rc>, ) -> BTreeMap)> { let logging_interval_ms = std::cmp::max(1, config.interval.as_millis()); let worker_id = worker.index(); @@ -123,6 +126,7 @@ pub(super) fn construct( DemuxHandler { state: &mut demux_state, + shared_state: &mut shared_state.borrow_mut(), output: &mut output_buffers, logging_interval_ms, peers, @@ -140,11 +144,11 @@ pub(super) fn construct( // updates that reach `Row` encoding. let operates = operates .as_collection() - .arrange_core::<_, RowSpine<_, _, _, _>>( + .mz_arrange_core::<_, RowSpine<_, _, _, _>>( Exchange::new(move |_| u64::cast_from(worker_id)), "PreArrange Timely operates", ) - .as_collection(move |(id, name), _| { + .as_collection(move |id, name| { Row::pack_slice(&[ Datum::UInt64(u64::cast_from(*id)), Datum::UInt64(u64::cast_from(worker_id)), @@ -153,7 +157,7 @@ pub(super) fn construct( }); let channels = channels .as_collection() - .arrange_core::<_, RowSpine<_, _, _, _>>( + .mz_arrange_core::<_, RowSpine<_, _, _, _>>( Exchange::new(move |_| u64::cast_from(worker_id)), "PreArrange Timely operates", ) @@ -171,14 +175,14 @@ pub(super) fn construct( }); let addresses = addresses .as_collection() - .arrange_core::<_, RowSpine<_, _, _, _>>( + .mz_arrange_core::<_, RowSpine<_, _, _, _>>( Exchange::new(move |_| u64::cast_from(worker_id)), "PreArrange Timely addresses", ) - .as_collection(move |(id, address), _| create_address_row(*id, address, worker_id)); + .as_collection(move |id, address| create_address_row(*id, address, worker_id)); let parks = parks .as_collection() - .arrange_core::<_, RowSpine<_, _, _, _>>( + .mz_arrange_core::<_, RowSpine<_, _, _, _>>( Exchange::new(move |_| u64::cast_from(worker_id)), "PreArrange Timely parks", ) @@ -194,7 +198,7 @@ pub(super) fn construct( }); let messages_sent = messages_sent .as_collection() - .arrange_core::<_, RowSpine<_, _, _, _>>( + .mz_arrange_core::<_, RowSpine<_, _, _, _>>( Exchange::new(move |_| u64::cast_from(worker_id)), "PreArrange Timely messages sent", ) @@ -207,7 +211,7 @@ pub(super) fn construct( }); let messages_received = messages_received .as_collection() - .arrange_core::<_, RowSpine<_, _, _, _>>( + .mz_arrange_core::<_, RowSpine<_, _, _, _>>( Exchange::new(move |_| u64::cast_from(worker_id)), "PreArrange Timely messages received", ) @@ -220,7 +224,7 @@ pub(super) fn construct( }); let elapsed = schedules_duration .as_collection() - .arrange_core::<_, RowSpine<_, _, _, _>>( + .mz_arrange_core::<_, RowSpine<_, _, _, _>>( Exchange::new(move |_| u64::cast_from(worker_id)), "PreArrange Timely duration", ) @@ -232,7 +236,7 @@ pub(super) fn construct( }); let histogram = schedules_histogram .as_collection() - .arrange_core::<_, RowSpine<_, _, _, _>>( + .mz_arrange_core::<_, RowSpine<_, _, _, _>>( Exchange::new(move |_| u64::cast_from(worker_id)), "PreArrange Timely histogram", ) @@ -245,23 +249,22 @@ pub(super) fn construct( row }); + use TimelyLog::*; let logs = [ - (LogVariant::Timely(TimelyLog::Operates), operates), - (LogVariant::Timely(TimelyLog::Channels), channels), - (LogVariant::Timely(TimelyLog::Elapsed), elapsed), - (LogVariant::Timely(TimelyLog::Histogram), histogram), - (LogVariant::Timely(TimelyLog::Addresses), addresses), - (LogVariant::Timely(TimelyLog::Parks), parks), - (LogVariant::Timely(TimelyLog::MessagesSent), messages_sent), - ( - LogVariant::Timely(TimelyLog::MessagesReceived), - messages_received, - ), + (Operates, operates), + (Channels, channels), + (Elapsed, elapsed), + (Histogram, histogram), + (Addresses, addresses), + (Parks, parks), + (MessagesSent, messages_sent), + (MessagesReceived, messages_received), ]; // Build the output arrangements. let mut traces = BTreeMap::new(); for (variant, collection) in logs { + let variant = LogVariant::Timely(variant); if config.index_logs.contains_key(&variant) { let key = variant.index_by(); let (_, value) = permutation_for_arrangement( @@ -284,7 +287,7 @@ pub(super) fn construct( (row_key, row_val) } }) - .arrange_named::>(&format!("ArrangeByKey {:?}", variant)) + .mz_arrange::>(&format!("ArrangeByKey {:?}", variant)) .trace; traces.insert(variant.clone(), (trace, Rc::clone(&token))); } @@ -354,13 +357,13 @@ type OutputBuffer<'a, 'b, D> = ConsolidateBuffer<'a, 'b, Timestamp, D, Diff, Pus // wouldn't be an issue. struct DemuxOutput<'a, 'b> { operates: OutputBuffer<'a, 'b, (usize, String)>, - channels: OutputBuffer<'a, 'b, ChannelDatum>, + channels: OutputBuffer<'a, 'b, (ChannelDatum, ())>, addresses: OutputBuffer<'a, 'b, (usize, Vec)>, - parks: OutputBuffer<'a, 'b, ParkDatum>, - messages_sent: OutputBuffer<'a, 'b, MessageDatum>, - messages_received: OutputBuffer<'a, 'b, MessageDatum>, - schedules_duration: OutputBuffer<'a, 'b, usize>, - schedules_histogram: OutputBuffer<'a, 'b, ScheduleHistogramDatum>, + parks: OutputBuffer<'a, 'b, (ParkDatum, ())>, + messages_sent: OutputBuffer<'a, 'b, (MessageDatum, ())>, + messages_received: OutputBuffer<'a, 'b, (MessageDatum, ())>, + schedules_duration: OutputBuffer<'a, 'b, (usize, ())>, + schedules_histogram: OutputBuffer<'a, 'b, (ScheduleHistogramDatum, ())>, } #[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Hash, Serialize, Deserialize)] @@ -408,6 +411,8 @@ impl Columnation for ScheduleHistogramDatum { struct DemuxHandler<'a, 'b, 'c> { /// State kept by the demux operator. state: &'a mut DemuxState, + /// State shared across log receivers. + shared_state: &'a mut SharedLoggingState, /// Demux output buffers. output: &'a mut DemuxOutput<'b, 'c>, /// The logging interval specifying the time granularity for the updates. @@ -463,15 +468,15 @@ impl DemuxHandler<'_, '_, '_> { source: event.source, target: event.target, }; - self.output.channels.give(self.cap, (datum, ts, 1)); + self.output.channels.give(self.cap, ((datum, ()), ts, 1)); let datum = (event.id, event.scope_addr.clone()); self.output.addresses.give(self.cap, (datum, ts, 1)); - let dataflow_id = event.scope_addr[0]; + let dataflow_index = event.scope_addr[0]; self.state .dataflow_channels - .entry(dataflow_id) + .entry(dataflow_index) .or_default() .push(event); } @@ -501,7 +506,7 @@ impl DemuxHandler<'_, '_, '_> { { self.output .schedules_duration - .give(self.cap, (event.id, ts, -elapsed_ns)); + .give(self.cap, ((event.id, ()), ts, -elapsed_ns)); let datum = ScheduleHistogramDatum { operator: event.id, @@ -510,23 +515,27 @@ impl DemuxHandler<'_, '_, '_> { let diff = Diff::cast_from(-count); self.output .schedules_histogram - .give(self.cap, (datum, ts, diff)); + .give(self.cap, ((datum, ()), ts, diff)); } } if operator.addr.len() == 1 { - let dataflow_id = operator.addr[0]; - self.handle_dataflow_shutdown(dataflow_id); + let dataflow_index = operator.addr[0]; + self.handle_dataflow_shutdown(dataflow_index); } let datum = (operator.id, operator.addr); self.output.addresses.give(self.cap, (datum, ts, -1)); } - fn handle_dataflow_shutdown(&mut self, dataflow_id: usize) { - // When a dataflow shuts down, we need to retract all its channels. + fn handle_dataflow_shutdown(&mut self, dataflow_index: usize) { + // Notify compute logging about the shutdown. + if let Some(logger) = &self.shared_state.compute_logger { + logger.log(ComputeEvent::DataflowShutdown { dataflow_index }); + } - let Some(channels) = self.state.dataflow_channels.remove(&dataflow_id) else { + // When a dataflow shuts down, we need to retract all its channels. + let Some(channels) = self.state.dataflow_channels.remove(&dataflow_index) else { return; }; @@ -538,7 +547,7 @@ impl DemuxHandler<'_, '_, '_> { source: channel.source, target: channel.target, }; - self.output.channels.give(self.cap, (datum, ts, -1)); + self.output.channels.give(self.cap, ((datum, ()), ts, -1)); let datum = (channel.id, channel.scope_addr); self.output.addresses.give(self.cap, (datum, ts, -1)); @@ -552,7 +561,7 @@ impl DemuxHandler<'_, '_, '_> { }; self.output .messages_sent - .give(self.cap, (datum, ts, -count)); + .give(self.cap, ((datum, ()), ts, -count)); } } if let Some(received) = self.state.messages_received.remove(&channel.id) { @@ -563,7 +572,7 @@ impl DemuxHandler<'_, '_, '_> { }; self.output .messages_received - .give(self.cap, (datum, ts, -count)); + .give(self.cap, ((datum, ()), ts, -count)); } } } @@ -596,7 +605,7 @@ impl DemuxHandler<'_, '_, '_> { duration_pow, requested_pow, }; - self.output.parks.give(self.cap, (datum, ts, 1)); + self.output.parks.give(self.cap, ((datum, ()), ts, 1)); } } } @@ -610,7 +619,9 @@ impl DemuxHandler<'_, '_, '_> { channel: event.channel, worker: event.target, }; - self.output.messages_sent.give(self.cap, (datum, ts, count)); + self.output + .messages_sent + .give(self.cap, ((datum, ()), ts, count)); let sent_counts = self .state @@ -625,7 +636,7 @@ impl DemuxHandler<'_, '_, '_> { }; self.output .messages_received - .give(self.cap, (datum, ts, count)); + .give(self.cap, ((datum, ()), ts, count)); let received_counts = self .state @@ -658,7 +669,7 @@ impl DemuxHandler<'_, '_, '_> { let datum = event.id; self.output .schedules_duration - .give(self.cap, (datum, ts, elapsed_diff)); + .give(self.cap, ((datum, ()), ts, elapsed_diff)); let datum = ScheduleHistogramDatum { operator: event.id, @@ -666,7 +677,7 @@ impl DemuxHandler<'_, '_, '_> { }; self.output .schedules_histogram - .give(self.cap, (datum, ts, 1)); + .give(self.cap, ((datum, ()), ts, 1)); // Record count and elapsed time for later retraction. let index = usize::cast_from(elapsed_pow.trailing_zeros()); diff --git a/src/compute/src/render/context.rs b/src/compute/src/render/context.rs index 1f2bdffdc609..2bce4c9937ed 100644 --- a/src/compute/src/render/context.rs +++ b/src/compute/src/render/context.rs @@ -14,7 +14,7 @@ use std::collections::BTreeMap; use std::rc::Weak; use differential_dataflow::lattice::Lattice; -use differential_dataflow::operators::arrange::{Arrange, Arranged}; +use differential_dataflow::operators::arrange::Arranged; use differential_dataflow::trace::wrappers::enter::TraceEnter; use differential_dataflow::trace::wrappers::frontier::TraceFrontier; use differential_dataflow::trace::{BatchReader, Cursor, TraceReader}; @@ -36,6 +36,7 @@ use timely::dataflow::{Scope, ScopeParent}; use timely::progress::timestamp::Refines; use timely::progress::{Antichain, Timestamp}; +use crate::extensions::arrange::{KeyCollection, MzArrange}; use crate::render::errors::ErrorLogger; use crate::render::join::LinearJoinImpl; use crate::typedefs::{ErrSpine, RowSpine, TraceErrHandle, TraceRowHandle}; @@ -781,10 +782,9 @@ where Ok::<(Row, Row), DataflowError>((key_row, val_row)) }); - let oks = oks_keyed.arrange_named::>(&name); - let errs = errs - .concat(&errs_keyed) - .arrange_named::>(&format!("{}-errors", name)); + let oks = oks_keyed.mz_arrange::>(&name); + let errs: KeyCollection<_, _, _> = errs.concat(&errs_keyed).into(); + let errs = errs.mz_arrange::>(&format!("{}-errors", name)); self.arranged .insert(key, ArrangementFlavor::Local(oks, errs)); } diff --git a/src/compute/src/render/join/linear_join.rs b/src/compute/src/render/join/linear_join.rs index ae00de9956bd..07350acae804 100644 --- a/src/compute/src/render/join/linear_join.rs +++ b/src/compute/src/render/join/linear_join.rs @@ -12,7 +12,7 @@ //! Consult [LinearJoinPlan] documentation for details. use differential_dataflow::lattice::Lattice; -use differential_dataflow::operators::arrange::arrangement::{Arrange, Arranged}; +use differential_dataflow::operators::arrange::arrangement::Arranged; use differential_dataflow::trace::TraceReader; use differential_dataflow::{AsCollection, Collection, Data}; use mz_compute_client::plan::join::linear_join::{LinearJoinPlan, LinearStagePlan}; @@ -24,6 +24,7 @@ use timely::dataflow::operators::OkErr; use timely::dataflow::Scope; use timely::progress::timestamp::{Refines, Timestamp}; +use crate::extensions::arrange::MzArrange; use crate::render::context::{ Arrangement, ArrangementFlavor, ArrangementImport, CollectionBundle, Context, }; @@ -246,7 +247,7 @@ where }); errors.push(errs); - let arranged = keyed.arrange_named::>("JoinStage"); + let arranged = keyed.mz_arrange::>("JoinStage"); joined = JoinedFlavor::Local(arranged); } diff --git a/src/compute/src/render/mod.rs b/src/compute/src/render/mod.rs index cd8772c651c0..8f8a0722d78d 100644 --- a/src/compute/src/render/mod.rs +++ b/src/compute/src/render/mod.rs @@ -106,8 +106,6 @@ use std::sync::Arc; use differential_dataflow::dynamic::pointstamp::PointStamp; use differential_dataflow::lattice::Lattice; -use differential_dataflow::operators::arrange::Arrange; -use differential_dataflow::operators::reduce::ReduceCore; use differential_dataflow::{AsCollection, Collection}; use itertools::izip; use mz_compute_client::plan::Plan; @@ -133,6 +131,9 @@ use timely::PartialOrder; use crate::arrangement::manager::TraceBundle; use crate::compute_state::ComputeState; +use crate::extensions::arrange::{KeyCollection, MzArrange}; +use crate::extensions::collection::ConsolidateExt; +use crate::extensions::reduce::MzReduce; use crate::logging::compute::LogImportFrontiers; use crate::render::context::{ArrangementFlavor, Context, ShutdownToken}; use crate::typedefs::{ErrSpine, RowKeySpine}; @@ -564,12 +565,12 @@ where let oks = oks .as_collection(|k, v| (k.clone(), v.clone())) .leave() - .arrange_named("Arrange export iterative"); + .mz_arrange("Arrange export iterative"); oks.stream.probe_notify_with(probes); let errs = errs .as_collection(|k, v| (k.clone(), v.clone())) .leave() - .arrange_named("Arrange export iterative err"); + .mz_arrange("Arrange export iterative err"); compute_state.traces.set( idx_id, TraceBundle::new(oks.trace, errs.trace).with_drop(needed_tokens), @@ -652,7 +653,7 @@ where let (oks_v, err_v) = variables.remove(&Id::Local(*id)).unwrap(); // Set oks variable to `oks` but consolidated to ensure iteration ceases at fixed point. - let mut oks = oks.consolidate_named::>("LetRecConsolidation"); + let mut oks = oks.mz_consolidate::>("LetRecConsolidation"); if let Some(token) = &self.shutdown_token.get_inner() { oks = oks.with_token(Weak::clone(token)); } @@ -686,9 +687,10 @@ where // say if the limit of `oks` has an error. This would result in non-terminating rather // than a clean report of the error. The trade-off is that we lose information about // multiplicities of errors, but .. this seems to be the better call. + let err: KeyCollection<_, _, _> = err.into(); let mut errs = err - .arrange_named::>("Arrange recursive err") - .reduce_abelian::<_, ErrSpine<_, _, _>>( + .mz_arrange::>("Arrange recursive err") + .mz_reduce_abelian::<_, ErrSpine<_, _, _>>( "Distinct recursive err", move |_k, _s, t| t.push(((), 1)), ) diff --git a/src/compute/src/render/reduce.rs b/src/compute/src/render/reduce.rs index bca251a4010c..8e96b1b25130 100644 --- a/src/compute/src/render/reduce.rs +++ b/src/compute/src/render/reduce.rs @@ -18,8 +18,6 @@ use differential_dataflow::collection::AsCollection; use differential_dataflow::difference::{Multiply, Semigroup}; use differential_dataflow::hashable::Hashable; use differential_dataflow::lattice::Lattice; -use differential_dataflow::operators::arrange::arrangement::Arrange; -use differential_dataflow::operators::reduce::ReduceCore; use differential_dataflow::Collection; use mz_compute_client::plan::reduce::{ AccumulablePlan, BasicPlan, BucketedPlan, HierarchicalPlan, KeyValPlan, MonotonicPlan, @@ -30,13 +28,15 @@ use mz_repr::adt::numeric::{self, Numeric, NumericAgg}; use mz_repr::{Datum, DatumList, DatumVec, Diff, Row, RowArena}; use mz_storage_client::types::errors::DataflowError; use mz_timely_util::operator::CollectionExt; -use mz_timely_util::reduce::ReduceExt; use serde::{Deserialize, Serialize}; use timely::dataflow::Scope; use timely::progress::timestamp::Refines; use timely::progress::Timestamp; use tracing::warn; +use crate::extensions::arrange::{KeyCollection, MzArrange}; +use crate::extensions::collection::ConsolidateExt; +use crate::extensions::reduce::{MzReduce, ReduceExt}; use crate::render::context::{Arrangement, CollectionBundle, Context, KeyArrangement}; use crate::render::errors::MaybeValidatingRow; use crate::render::reduce::monoids::ReductionMonoid; @@ -166,14 +166,10 @@ where { let mut errors = Default::default(); let arrangement = self.render_reduce_plan_inner(plan, collection, &mut errors, key_arity); + let errs: KeyCollection<_, _, _> = err_input.concatenate(errors).into(); CollectionBundle::from_columns( 0..key_arity, - ArrangementFlavor::Local( - arrangement, - err_input - .concatenate(errors) - .arrange_named("Arrange bundle err"), - ), + ArrangementFlavor::Local(arrangement, errs.mz_arrange("Arrange bundle err")), ) } @@ -300,7 +296,7 @@ where let aggregate_types_err = aggregate_types.clone(); use differential_dataflow::collection::concatenate; let (oks, errs) = concatenate(scope, to_concat) - .arrange_named::>("Arrange ReduceCollation") + .mz_arrange::>("Arrange ReduceCollation") .reduce_pair::<_, RowSpine<_, _, _, _>, _, ErrValSpine<_, _, _>>( "ReduceCollation", "ReduceCollation Errors", @@ -431,7 +427,7 @@ where let error_logger = self.error_logger(); let (output, errors) = collection - .arrange_named::>("Arranged DistinctBy") + .mz_arrange::>("Arranged DistinctBy") .reduce_pair::<_, RowSpine<_, _, _, _>, _, ErrValSpine<_, _, _>>( "DistinctBy", "DistinctByErrorCheck", @@ -491,8 +487,8 @@ where .push(result.as_collection(move |key, val| (key.clone(), (index, val.clone())))); } let output = differential_dataflow::collection::concatenate(&mut input.scope(), to_collect) - .arrange_named::>("Arranged ReduceFuseBasic input") - .reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceFuseBasic", { + .mz_arrange::>("Arranged ReduceFuseBasic input") + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceFuseBasic", { let mut row_buf = Row::default(); move |_key, input, output| { let mut row_packer = row_buf.packer(); @@ -560,9 +556,8 @@ where } } - let arranged = - partial.arrange_named::>("Arranged ReduceInaccumulable"); - let oks = arranged.reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceInaccumulable", { + let arranged = partial.mz_arrange::>("Arranged ReduceInaccumulable"); + let oks = arranged.mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceInaccumulable", { let mut row_buf = Row::default(); move |_key, source, target| { // We respect the multiplicity here (unlike in hierarchical aggregation) @@ -585,7 +580,7 @@ where if validating && err_output.is_none() { let error_logger = self.error_logger(); - let errs = arranged.reduce_abelian::<_, ErrValSpine<_, _, _>>( + let errs = arranged.mz_reduce_abelian::<_, ErrValSpine<_, _, _>>( "ReduceInaccumulable Error Check", move |_key, source, target| { // Negative counts would be surprising, but until we are 100% certain we won't @@ -619,9 +614,10 @@ where { let error_logger = self.error_logger(); + let input: KeyCollection<_, _, _> = input.into(); input - .arrange_named::>("Arranged ReduceInaccumulable") - .reduce_abelian::<_, RowSpine<_, _, _, _>>( + .mz_arrange::>("Arranged ReduceInaccumulable") + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>( "ReduceInaccumulable", move |_, source, t| { if let Some(err) = R::into_error() { @@ -723,9 +719,7 @@ where stage = negated_output .negate() .concat(&input) - .consolidate_named::>( - "Consolidated MinsMaxesHierarchical", - ); + .mz_consolidate::>("Consolidated MinsMaxesHierarchical"); } // Discard the hash from the key and return to the format of the input data. @@ -735,13 +729,13 @@ where // Arrange the final result into (key, Row) let error_logger = self.error_logger(); let arranged = - partial.arrange_named::, _, _>>("Arrange ReduceMinsMaxes"); + partial.mz_arrange::, _, _>>("Arrange ReduceMinsMaxes"); // Note that we would prefer to use `mz_timely_util::reduce::ReduceExt::reduce_pair` here, // but we then wouldn't be able to do this error check conditionally. See its documentation // for the rationale around using a second reduction here. if validating { let errs = arranged - .reduce_abelian::<_, ErrValSpine<_, _, _>>( + .mz_reduce_abelian::<_, ErrValSpine<_, _, _>>( "ReduceMinsMaxes Error Check", move |_key, source, target| { // Negative counts would be surprising, but until we are 100% certain we wont @@ -763,7 +757,7 @@ where err_output = Some(errs.leave_region()); } arranged - .reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceMinsMaxes", { + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceMinsMaxes", { let mut row_buf = Row::default(); move |_key, source: &[(&Vec, Diff)], target: &mut Vec<(Row, Diff)>| { let mut row_packer = row_buf.packer(); @@ -802,11 +796,11 @@ where R: MaybeValidatingRow, (Row, u64)>, { let error_logger = self.error_logger(); - let arranged_input = input - .arrange_named::, _, _>>("Arranged MinsMaxesHierarchical input"); + let arranged_input = + input.mz_arrange::, _, _>>("Arranged MinsMaxesHierarchical input"); arranged_input - .reduce_abelian::<_, RowSpine<_, _, _, _>>( + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>( "Reduced Fallibly MinsMaxesHierarchical", move |key, source, target| { if let Some(err) = R::into_error() { @@ -875,7 +869,7 @@ where (key, values) }) - .consolidate_named_if::>( + .mz_consolidate_if::>( must_consolidate, "Consolidated ReduceMonotonic input", ); @@ -902,9 +896,10 @@ where } (key, output) }); + let partial: KeyCollection<_, _, _> = partial.into(); let output = partial - .arrange_named::>>("ArrangeMonotonic") - .reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceMonotonic", { + .mz_arrange::>>("ArrangeMonotonic") + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceMonotonic", { let mut row_buf = Row::default(); move |_key, input, output| { let mut row_packer = row_buf.packer(); @@ -1174,8 +1169,8 @@ where (key, row_buf.clone()) }) .map(|k| (k, ())) - .arrange_named::>("Arranged Accumulable") - .reduce_abelian::<_, RowKeySpine<_, _, _>>( + .mz_arrange::>("Arranged Accumulable") + .mz_reduce_abelian::<_, RowKeySpine<_, _, _>>( "Reduced Accumulable", move |_k, _s, t| t.push(((), 1)), ) @@ -1203,7 +1198,7 @@ where let error_logger = self.error_logger(); let err_full_aggrs = full_aggrs.clone(); let (arranged_output, arranged_errs) = collection - .arrange_named::, Diff)>>("ArrangeAccumulable") + .mz_arrange::, Diff)>>("ArrangeAccumulable") .reduce_pair::<_, RowSpine<_, _, _, _>, _, ErrValSpine<_, _, _>>( "ReduceAccumulable", "AccumulableErrorCheck", diff --git a/src/compute/src/render/threshold.rs b/src/compute/src/render/threshold.rs index 20880de82407..3b289eda1e24 100644 --- a/src/compute/src/render/threshold.rs +++ b/src/compute/src/render/threshold.rs @@ -12,8 +12,7 @@ //! Consult [ThresholdPlan] documentation for details. use differential_dataflow::lattice::Lattice; -use differential_dataflow::operators::arrange::{Arrange, Arranged, TraceAgent}; -use differential_dataflow::operators::reduce::ReduceCore; +use differential_dataflow::operators::arrange::{Arranged, TraceAgent}; use mz_compute_client::plan::threshold::{BasicThresholdPlan, ThresholdPlan}; use mz_expr::MirScalarExpr; use mz_repr::{Diff, Row}; @@ -21,6 +20,8 @@ use timely::dataflow::Scope; use timely::progress::timestamp::Refines; use timely::progress::Timestamp; +use crate::extensions::arrange::{KeyCollection, MzArrange}; +use crate::extensions::reduce::MzReduce; use crate::render::context::{ArrangementFlavor, CollectionBundle, Context}; use crate::typedefs::RowSpine; @@ -34,10 +35,10 @@ where G: Scope, G::Timestamp: Lattice + Refines, T: Timestamp + Lattice, - R: ReduceCore, + R: MzReduce, L: Fn(&Diff) -> bool + 'static, { - arrangement.reduce_abelian(name, move |_key, s, t| { + arrangement.mz_reduce_abelian(name, move |_key, s, t| { for (record, count) in s.iter() { if logic(count) { t.push(((*record).clone(), *count)); @@ -69,9 +70,8 @@ where } ArrangementFlavor::Trace(_, oks, errs) => { let oks = threshold_arrangement(&oks, "Threshold trace", |count| *count > 0); - let errs = errs - .as_collection(|k, _| k.clone()) - .arrange_named("Arrange threshold basic err"); + let errs: KeyCollection<_, _, _> = errs.as_collection(|k, _| k.clone()).into(); + let errs = errs.mz_arrange("Arrange threshold basic err"); CollectionBundle::from_expressions(key, ArrangementFlavor::Local(oks, errs)) } } diff --git a/src/compute/src/render/top_k.rs b/src/compute/src/render/top_k.rs index cdf5d2148e95..f0c48b031a8a 100644 --- a/src/compute/src/render/top_k.rs +++ b/src/compute/src/render/top_k.rs @@ -17,8 +17,6 @@ use std::rc::Rc; use differential_dataflow::hashable::Hashable; use differential_dataflow::lattice::Lattice; -use differential_dataflow::operators::arrange::Arrange; -use differential_dataflow::operators::reduce::ReduceCore; use differential_dataflow::{AsCollection, Collection}; use mz_compute_client::plan::top_k::{ BasicTopKPlan, MonotonicTop1Plan, MonotonicTopKPlan, TopKPlan, @@ -32,6 +30,9 @@ use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::Operator; use timely::dataflow::Scope; +use crate::extensions::arrange::MzArrange; +use crate::extensions::collection::ConsolidateExt; +use crate::extensions::reduce::MzReduce; use crate::render::context::{CollectionBundle, Context}; use crate::render::errors::MaybeValidatingRow; use crate::typedefs::{RowKeySpine, RowSpine}; @@ -90,7 +91,7 @@ where }; (group_row, row) }) - .consolidate_named_if::>( + .mz_consolidate_if::>( must_consolidate, "Consolidated MonotonicTopK input", ); @@ -300,7 +301,7 @@ where ( oks.negate() .concat(&input) - .consolidate_named::>("Consolidated TopK"), + .mz_consolidate::>("Consolidated TopK"), errs, ) } @@ -334,7 +335,7 @@ where (group_key, row) } }) - .consolidate_named_if::>( + .mz_consolidate_if::>( must_consolidate, "Consolidated MonotonicTop1 input", ); @@ -360,8 +361,8 @@ where }); let result = partial .map(|k| (k, ())) - .arrange_named::>("Arranged MonotonicTop1 partial") - .reduce_abelian::<_, RowSpine<_, _, _, _>>("MonotonicTop1", { + .mz_arrange::>("Arranged MonotonicTop1 partial") + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("MonotonicTop1", { move |_key, input, output| { let accum: &monoids::Top1Monoid = &input[0].1; output.push((accum.row.clone(), 1)); @@ -387,8 +388,8 @@ where // We only want to arrange parts of the input that are not part of the actual output // such that `input.concat(&negated_output.negate())` yields the correct TopK input - .arrange_named::>("Arranged TopK input") - .reduce_abelian::<_, RowSpine<_, _, _, _>>("Reduced TopK input", { + .mz_arrange::>("Arranged TopK input") + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("Reduced TopK input", { move |_key, source, target: &mut Vec<(R, Diff)>| { if let Some(err) = R::into_error() { for (row, diff) in source.iter() { diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 441522980a6a..e4988d19d87a 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -151,7 +151,7 @@ struct Worker<'w, A: Allocate> { impl mz_cluster::types::AsRunnableWorker for Config { type Activatable = SyncActivator; - fn build_and_run( + fn build_and_run( config: Self, timely_worker: &mut TimelyWorker, client_rx: crossbeam_channel::Receiver<( @@ -175,7 +175,7 @@ impl mz_cluster::types::AsRunnableWorker for Co } } -impl<'w, A: Allocate> Worker<'w, A> { +impl<'w, A: Allocate + 'static> Worker<'w, A> { /// Waits for client connections and runs them to completion. pub fn run(&mut self) { let mut shutdown = false; diff --git a/src/compute/src/typedefs.rs b/src/compute/src/typedefs.rs index 5b6f90a062f3..d899bf3f330b 100644 --- a/src/compute/src/typedefs.rs +++ b/src/compute/src/typedefs.rs @@ -12,16 +12,14 @@ #![allow(missing_docs)] use differential_dataflow::operators::arrange::TraceAgent; -use differential_dataflow::trace::implementations::ord::{ - ColKeySpine, ColValSpine, OrdKeySpine, OrdValSpine, -}; +use differential_dataflow::trace::implementations::ord::{ColKeySpine, ColValSpine}; use mz_repr::{Diff, Row, Timestamp}; use mz_storage_client::types::errors::DataflowError; pub type RowSpine = ColValSpine; pub type RowKeySpine = ColKeySpine; -pub type ErrSpine = OrdKeySpine; -pub type ErrValSpine = OrdValSpine; +pub type ErrSpine = ColKeySpine; +pub type ErrValSpine = ColValSpine; pub type TraceRowHandle = TraceAgent>; pub type TraceErrHandle = TraceAgent>; pub type KeysValsHandle = TraceRowHandle; diff --git a/src/environmentd/src/http/static/js/hierarchical-memory.jsx b/src/environmentd/src/http/static/js/hierarchical-memory.jsx index d0bc2b1cb042..40967f612376 100644 --- a/src/environmentd/src/http/static/js/hierarchical-memory.jsx +++ b/src/environmentd/src/http/static/js/hierarchical-memory.jsx @@ -159,7 +159,7 @@ function Dataflows(props) { ON channels.id = counts.channel_id; SELECT - operator_id as id, records + operator_id as id, records, size FROM mz_internal.mz_arrangement_sizes; `); @@ -183,7 +183,10 @@ function Dataflows(props) { ); setChans(chans); - setRecords(Object.fromEntries(records_table.rows)); + const records = Object.fromEntries( + records_table.rows.map(([id, records, size]) => [id, [records, size]]) + ) + setRecords(records); try { const view = await getCreateView(stats.name); @@ -217,7 +220,10 @@ function Dataflows(props) { const id_to_addr = Object.fromEntries(Object.entries(addrs).map(([id, addr]) => [id, addr])); const id_to_name = Object.fromEntries(Object.entries(opers).map(([id, name]) => [id, name])); const addr_to_id = Object.fromEntries(Object.entries(opers).map(([id, name]) => [addrStr(id_to_addr[id]), id])); - const max_record_count = Math.max.apply(Math, Object.values(records)); + const max_record_count = Math.max.apply( + Math, + Object.values(records).map(([records, size]) => records) + ); // Map scopes to children. const scope_children = new Map(); @@ -284,9 +290,10 @@ function Dataflows(props) { } else { let my_records = records["".concat(id)]; if (my_records != null) { - return `${id} [label= "${id} : ${name} \n\t records : ${my_records}",style=filled,color=red,fillcolor="#ffbbbb"]`; + let my_size = Math.ceil(my_records[1]/1024); + return `${id} [label= "${id} : ${name}\nrecords: ${my_records[0]}, ${my_size} KiB",style=filled,color=red,fillcolor="#ffbbbb",shape=box]`; } else { - return `${id} [label="${id} : ${name}"]`; + return `${id} [label="${id} : ${name}",shape=box]`; } } } else { diff --git a/src/environmentd/src/http/static/js/memory.jsx b/src/environmentd/src/http/static/js/memory.jsx index 62ceec6904aa..a201d5721e29 100644 --- a/src/environmentd/src/http/static/js/memory.jsx +++ b/src/environmentd/src/http/static/js/memory.jsx @@ -164,7 +164,7 @@ function Views(props) { SET cluster = ${formatNameForQuery(props.clusterName)}; SET cluster_replica = ${formatNameForQuery(props.replicaName)}; SELECT - id, name, records + id, name, records, size, capacity, allocations FROM mz_internal.mz_records_per_dataflow ${whereFragment} @@ -215,6 +215,9 @@ function Views(props) { dataflow id index name records + size [KiB] + capacity [KiB] + allocations @@ -230,6 +233,9 @@ function Views(props) { {v[1]} {v[2]} + {Math.round(v[3]/1024)} + {Math.round(v[4]/1024)} + {v[5]} ))} @@ -271,7 +277,7 @@ function View(props) { SET cluster = ${formatNameForQuery(props.clusterName)}; SET cluster_replica = ${formatNameForQuery(props.replicaName)}; SELECT - name, records + name, records, size FROM mz_internal.mz_records_per_dataflow WHERE @@ -335,7 +341,7 @@ function View(props) { ); SELECT - id, records + id, records, size FROM mz_internal.mz_records_per_dataflow_operator WHERE @@ -348,6 +354,7 @@ function View(props) { const stats = { name: stats_row[0], records: stats_row[1], + size: stats_row[2], }; setStats(stats); @@ -370,7 +377,10 @@ function View(props) { ); setChans(chans); - setRecords(Object.fromEntries(records_table.rows)); + const records = Object.fromEntries( + records_table.rows.map(([id, records, size]) => [id, [records, size]]) + ) + setRecords(records); setElapsed(Object.fromEntries(elapsed_table.rows)); @@ -399,7 +409,10 @@ function View(props) { const lookup = Object.fromEntries( Object.entries(addrs).map(([id, addr]) => [addrStr(addr), id]) ); - const max_record_count = Math.max.apply(Math, Object.values(records)); + const max_record_count = Math.max.apply( + Math, + Object.values(records).map(([records, size]) => records) + ); const scopes = {}; // Find all the scopes. Object.entries(opers).forEach(([id, name]) => { @@ -452,13 +465,14 @@ function View(props) { const notes = [`id: ${id}`]; let style = ''; if (id in records) { - const record_count = records[id]; + const record_count = records[id][0]; + const size = Math.ceil(records[id][1]/1024); // Any operator that can have records will have a red border (even if it // currently has 0 records). The fill color is a deeper red based on how many // records this operator has compared to the operator with the most records. - const pct = record_count ? Math.floor((record_count / max_record_count) * 0xff) : 0; + const pct = record_count ? Math.floor((record_count / max_record_count) * 0xa0) : 0; const alpha = pct.toString(16).padStart(2, '0'); - notes.push(`${record_count} records`); + notes.push(`${record_count} rows, ${size} KiB`); style = `,style=filled,color=red,fillcolor="#ff0000${alpha}"`; } // Only display elapsed time if it's more than 1s. @@ -469,7 +483,7 @@ function View(props) { if (name.length > maxLen + 3) { name = name.slice(0, maxLen) + '...'; } - return `_${id} [label="${name} (${notes.join(', ')})"${style}]`; + return `_${id} [label="${name}\n${notes.join(', ')}"${style},shape=box]`; }); oper_labels.unshift(''); clusters.unshift(''); diff --git a/src/storage-client/src/types/errors.rs b/src/storage-client/src/types/errors.rs index b4b5a8e73bad..6c7ca5a86ef6 100644 --- a/src/storage-client/src/types/errors.rs +++ b/src/storage-client/src/types/errors.rs @@ -411,6 +411,26 @@ pub enum DataflowError { impl Error for DataflowError {} +mod columnation { + use crate::types::errors::DataflowError; + use timely::container::columnation::{CloneRegion, Columnation}; + + impl Columnation for DataflowError { + // Discussion of `Region` for `DataflowError`: Although `DataflowError` contains pointers, + // we treat it as a type that doesn't and can simply be cloned. The reason for this is + // threefold: + // 1. Cloning the type does not violate correctness. It comes with the disadvantage that its + // `heap_size` will not be accurate. + // 2. It is hard to implement a region allocator for `DataflowError`, because it contains + // many pointers across various enum types. Some are boxed, and it contains a box to + // itself, meaning we have to pass the outer region inwards to avoid creating recursive + // regions. + // 3. We accept the performance implication of not storing the errors in a region allocator, + // which should be similar to storing errors in vectors on the heap. + type InnerRegion = CloneRegion; + } +} + impl RustType for DataflowError { fn into_proto(&self) -> ProtoDataflowError { use proto_dataflow_error::Kind::*; diff --git a/src/storage/src/render/sinks.rs b/src/storage/src/render/sinks.rs index 644de2a78288..6acdd58aa606 100644 --- a/src/storage/src/render/sinks.rs +++ b/src/storage/src/render/sinks.rs @@ -174,6 +174,9 @@ where // (As part of doing so, it asserts that there are not multiple conflicting values at the same timestamp) let collection = match sink.envelope { Some(SinkEnvelope::Debezium) => { + // Allow access to `arrange_named` because we cannot access Mz's wrapper from here. + // TODO(#17413): Revisit with cluster unification. + #[allow(clippy::disallowed_methods)] let combined = combine_at_timestamp( keyed.arrange_named::>("Arrange Debezium"), ); @@ -204,6 +207,9 @@ where collection } Some(SinkEnvelope::Upsert) => { + // Allow access to `arrange_named` because we cannot access Mz's wrapper from here. + // TODO(#17413): Revisit with cluster unification. + #[allow(clippy::disallowed_methods)] let combined = combine_at_timestamp( keyed.arrange_named::>("Arrange Upsert"), ); diff --git a/src/timely-util/Cargo.toml b/src/timely-util/Cargo.toml index f2d1a1ef3e04..dc454a9f92b1 100644 --- a/src/timely-util/Cargo.toml +++ b/src/timely-util/Cargo.toml @@ -17,7 +17,6 @@ polonius-the-crab = "0.3.1" workspace-hack = { version = "0.0.0", path = "../workspace-hack" } tokio = { version = "1.24.2", features = ["macros", "rt-multi-thread", "time"] } num-traits = "0.2" -ahash = { version = "0.8.0", default_features = false } [package.metadata.cargo-udeps.ignore] normal = ["workspace-hack"] diff --git a/src/timely-util/src/lib.rs b/src/timely-util/src/lib.rs index 2acd81ea57db..feac1e42ca12 100644 --- a/src/timely-util/src/lib.rs +++ b/src/timely-util/src/lib.rs @@ -93,5 +93,4 @@ pub mod pact; pub mod panic; pub mod probe; pub mod progress; -pub mod reduce; pub mod replay; diff --git a/src/timely-util/src/operator.rs b/src/timely-util/src/operator.rs index 5ce8e8ed8482..dd2bf1d07d74 100644 --- a/src/timely-util/src/operator.rs +++ b/src/timely-util/src/operator.rs @@ -10,15 +10,12 @@ //! Common operator transformations on timely streams and differential collections. use std::future::Future; -use std::hash::{BuildHasher, Hash, Hasher}; use std::rc::Weak; use differential_dataflow::difference::{Multiply, Semigroup}; use differential_dataflow::lattice::Lattice; -use differential_dataflow::operators::arrange::Arrange; -use differential_dataflow::trace::{Batch, Trace, TraceReader}; use differential_dataflow::{AsCollection, Collection}; -use timely::dataflow::channels::pact::{Exchange, ParallelizationContract, Pipeline}; +use timely::dataflow::channels::pact::{ParallelizationContract, Pipeline}; use timely::dataflow::channels::pushers::Tee; use timely::dataflow::operators::generic::builder_rc::OperatorBuilder as OperatorBuilderRc; use timely::dataflow::operators::generic::operator::{self, Operator}; @@ -224,16 +221,6 @@ where /// the provided token can be upgraded. Once the token cannot be upgraded anymore, all data /// flowing into the operator is dropped. fn with_token(&self, token: Weak<()>) -> Collection; - - /// Consolidates the collection if `must_consolidate` is `true` and leaves it - /// untouched otherwise. - fn consolidate_named_if(self, must_consolidate: bool, name: &str) -> Self - where - D1: differential_dataflow::ExchangeData + Hash, - R: Semigroup + differential_dataflow::ExchangeData, - G::Timestamp: Lattice, - Tr: Trace + TraceReader + 'static, - Tr::Batch: Batch; } impl StreamExt for Stream @@ -539,51 +526,6 @@ where fn with_token(&self, token: Weak<()>) -> Collection { self.inner.with_token(token).as_collection() } - - fn consolidate_named_if(self, must_consolidate: bool, name: &str) -> Self - where - D1: differential_dataflow::ExchangeData + Hash, - R: Semigroup + differential_dataflow::ExchangeData, - G::Timestamp: Lattice + Ord, - Tr: Trace + TraceReader + 'static, - Tr::Batch: Batch, - { - if must_consolidate { - // We employ AHash below instead of the default hasher in DD to obtain - // a better distribution of data to workers. AHash claims empirically - // both speed and high quality, according to - // https://github.com/tkaitchuck/aHash/blob/master/compare/readme.md. - // TODO(vmarcos): Consider here if it is worth it to spend the time to - // implement twisted tabulation hashing as proposed in Mihai Patrascu, - // Mikkel Thorup: Twisted Tabulation Hashing. SODA 2013: 209-228, available - // at https://epubs.siam.org/doi/epdf/10.1137/1.9781611973105.16. The latter - // would provide good bounds for balls-into-bins problems when the number of - // bins is small (as is our case), so we'd have a theoretical guarantee. - // NOTE: We fix the seeds of a RandomState instance explicity with the same - // seeds that would be given by `AHash` via ahash::AHasher::default() so as - // to avoid a different selection due to compile-time features being differently - // selected in other dependencies using `AHash` vis-à-vis cargo's strategy - // of unioning features. - // NOTE: Depending on target features, we may end up employing the fallback - // hasher of `AHash`, but it should be sufficient for our needs. - let random_state = ahash::RandomState::with_seeds( - 0x243f_6a88_85a3_08d3, - 0x1319_8a2e_0370_7344, - 0xa409_3822_299f_31d0, - 0x082e_fa98_ec4e_6c89, - ); - let exchange = Exchange::new(move |update: &((D1, _), G::Timestamp, R)| { - let data = &(update.0).0; - let mut h = random_state.build_hasher(); - data.hash(&mut h); - h.finish() - }); - self.arrange_core::<_, Tr>(exchange, name) - .as_collection(|k, _v| k.clone()) - } else { - self - } - } } /// Creates a new async data stream source for a scope. diff --git a/test/sqllogictest/cluster.slt b/test/sqllogictest/cluster.slt index c892f8840f4f..26010bf7e899 100644 --- a/test/sqllogictest/cluster.slt +++ b/test/sqllogictest/cluster.slt @@ -198,6 +198,12 @@ bar mz_active_peeks_per_worker mz_active_peeks_per_worker_u7_primary_idx 1 i bar mz_active_peeks_per_worker mz_active_peeks_per_worker_u7_primary_idx 2 worker_id NULL false bar mz_arrangement_batches_raw mz_arrangement_batches_raw_u7_primary_idx 1 operator_id NULL false bar mz_arrangement_batches_raw mz_arrangement_batches_raw_u7_primary_idx 2 worker_id NULL false +bar mz_arrangement_heap_allocations_raw mz_arrangement_heap_allocations_raw_u7_primary_idx 1 operator_id NULL false +bar mz_arrangement_heap_allocations_raw mz_arrangement_heap_allocations_raw_u7_primary_idx 2 worker_id NULL false +bar mz_arrangement_heap_capacity_raw mz_arrangement_heap_capacity_raw_u7_primary_idx 1 operator_id NULL false +bar mz_arrangement_heap_capacity_raw mz_arrangement_heap_capacity_raw_u7_primary_idx 2 worker_id NULL false +bar mz_arrangement_heap_size_raw mz_arrangement_heap_size_raw_u7_primary_idx 1 operator_id NULL false +bar mz_arrangement_heap_size_raw mz_arrangement_heap_size_raw_u7_primary_idx 2 worker_id NULL false bar mz_arrangement_records_raw mz_arrangement_records_raw_u7_primary_idx 1 operator_id NULL false bar mz_arrangement_records_raw mz_arrangement_records_raw_u7_primary_idx 2 worker_id NULL false bar mz_arrangement_sharing_raw mz_arrangement_sharing_raw_u7_primary_idx 1 operator_id NULL false @@ -232,6 +238,8 @@ bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_ra bar mz_dataflow_operator_reachability_raw mz_dataflow_operator_reachability_raw_u7_primary_idx 5 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 +bar mz_dataflow_shutdown_durations_histogram_raw mz_dataflow_shutdown_durations_histogram_raw_u7_primary_idx 2 duration_ns NULL false bar mz_message_counts_received_raw mz_message_counts_received_raw_u7_primary_idx 1 channel_id NULL false bar mz_message_counts_received_raw mz_message_counts_received_raw_u7_primary_idx 2 from_worker_id NULL false bar mz_message_counts_received_raw mz_message_counts_received_raw_u7_primary_idx 3 to_worker_id NULL false @@ -394,7 +402,7 @@ DROP CLUSTER foo, foo2, foo3, foo4 CASCADE query I SELECT COUNT(name) FROM mz_indexes WHERE cluster_id = 'u1'; ---- -19 +23 query I SELECT COUNT(name) FROM mz_indexes WHERE cluster_id <> 'u1' AND cluster_id NOT LIKE 's%'; @@ -407,7 +415,7 @@ CREATE CLUSTER test REPLICAS (foo (SIZE '1')); query I SELECT COUNT(name) FROM mz_indexes; ---- -100 +116 statement ok DROP CLUSTER test CASCADE @@ -415,7 +423,7 @@ DROP CLUSTER test CASCADE query T SELECT COUNT(name) FROM mz_indexes; ---- -81 +93 statement error nvalid SIZE: must provide a string value CREATE CLUSTER REPLICA default.size_1 SIZE; diff --git a/test/sqllogictest/information_schema_tables.slt b/test/sqllogictest/information_schema_tables.slt index e3a962ad4841..524c1161a4f2 100644 --- a/test/sqllogictest/information_schema_tables.slt +++ b/test/sqllogictest/information_schema_tables.slt @@ -217,6 +217,18 @@ mz_arrangement_batches_raw SOURCE materialize mz_internal +mz_arrangement_heap_allocations_raw +SOURCE +materialize +mz_internal +mz_arrangement_heap_capacity_raw +SOURCE +materialize +mz_internal +mz_arrangement_heap_size_raw +SOURCE +materialize +mz_internal mz_arrangement_records_raw SOURCE materialize @@ -393,6 +405,18 @@ mz_dataflow_operators_per_worker SOURCE materialize mz_internal +mz_dataflow_shutdown_durations_histogram +VIEW +materialize +mz_internal +mz_dataflow_shutdown_durations_histogram_per_worker +VIEW +materialize +mz_internal +mz_dataflow_shutdown_durations_histogram_raw +SOURCE +materialize +mz_internal mz_dataflows VIEW materialize diff --git a/test/testdrive/catalog.td b/test/testdrive/catalog.td index 9863364ec672..901d2879cfb5 100644 --- a/test/testdrive/catalog.td +++ b/test/testdrive/catalog.td @@ -519,6 +519,9 @@ mz_active_peeks_per_worker log mz_arrangement_batches_raw log mz_arrangement_records_raw log mz_arrangement_sharing_raw log +mz_arrangement_heap_size_raw log +mz_arrangement_heap_capacity_raw log +mz_arrangement_heap_allocations_raw log mz_compute_delays_histogram_raw log mz_compute_dependencies_per_worker log mz_compute_exports_per_worker log @@ -529,6 +532,7 @@ mz_dataflow_addresses_per_worker log mz_dataflow_channels_per_worker log mz_dataflow_operator_reachability_raw log mz_dataflow_operators_per_worker log +mz_dataflow_shutdown_durations_histogram_raw log mz_message_counts_received_raw log mz_message_counts_sent_raw log mz_peek_durations_histogram_raw log @@ -578,6 +582,7 @@ mz_compute_import_frontiers mz_compute_operator_durations_histogram mz_compute_operator_durations_histogram_per_worker mz_dataflow_addresses +mz_dataflow_arrangement_sizes mz_dataflow_channel_operators mz_dataflow_channel_operators_per_worker mz_dataflow_channels @@ -585,10 +590,11 @@ mz_dataflow_operator_dataflows mz_dataflow_operator_dataflows_per_worker mz_dataflow_operator_parents mz_dataflow_operator_parents_per_worker -mz_dataflow_arrangement_sizes mz_dataflow_operator_reachability mz_dataflow_operator_reachability_per_worker mz_dataflow_operators +mz_dataflow_shutdown_durations_histogram +mz_dataflow_shutdown_durations_histogram_per_worker mz_dataflows mz_dataflows_per_worker mz_message_counts @@ -627,7 +633,7 @@ test_table # There is one entry in mz_indexes for each field_number/expression of the index. > SELECT COUNT(id) FROM mz_indexes WHERE id LIKE 's%' -81 +93 # Create a second schema with the same table name as above > CREATE SCHEMA tester2 diff --git a/test/testdrive/divergent-dataflow-cancellation.td b/test/testdrive/divergent-dataflow-cancellation.td index 48571095ada8..04a817d1faef 100644 --- a/test/testdrive/divergent-dataflow-cancellation.td +++ b/test/testdrive/divergent-dataflow-cancellation.td @@ -93,7 +93,7 @@ contains: canceling statement due to statement timeout > SELECT count(*) FROM mz_internal.mz_compute_frontiers_per_worker WHERE worker_id = 0 -19 +23 > SELECT count(*) FROM mz_internal.mz_compute_import_frontiers_per_worker 0 @@ -113,6 +113,10 @@ contains: canceling statement due to statement timeout > SELECT count(*) FROM mz_internal.mz_dataflow_operators_per_worker 0 +# This source never sees retractions. +> SELECT count(*) > 0 FROM mz_internal.mz_dataflow_shutdown_durations_histogram_raw +true + > SELECT count(*) FROM mz_internal.mz_message_counts_received_raw 0 diff --git a/test/testdrive/indexes.td b/test/testdrive/indexes.td index 2e61450304a6..18e0a292499b 100644 --- a/test/testdrive/indexes.td +++ b/test/testdrive/indexes.td @@ -290,6 +290,9 @@ mz_active_peeks_per_worker_s2_primary_idx mz_active_peeks_per_ mz_arrangement_batches_raw_s2_primary_idx mz_arrangement_batches_raw mz_introspection {operator_id,worker_id} mz_arrangement_records_raw_s2_primary_idx mz_arrangement_records_raw mz_introspection {operator_id,worker_id} mz_arrangement_sharing_raw_s2_primary_idx mz_arrangement_sharing_raw mz_introspection {operator_id,worker_id} +mz_arrangement_heap_capacity_raw_s2_primary_idx mz_arrangement_heap_capacity_raw mz_introspection {operator_id,worker_id} +mz_arrangement_heap_allocations_raw_s2_primary_idx mz_arrangement_heap_allocations_raw mz_introspection {operator_id,worker_id} +mz_arrangement_heap_size_raw_s2_primary_idx mz_arrangement_heap_size_raw mz_introspection {operator_id,worker_id} mz_cluster_replica_metrics_ind mz_cluster_replica_metrics mz_introspection {replica_id} mz_cluster_replica_sizes_ind mz_cluster_replica_sizes mz_introspection {size} mz_cluster_replica_statuses_ind mz_cluster_replica_statuses mz_introspection {replica_id} @@ -305,6 +308,7 @@ mz_dataflow_addresses_per_worker_s2_primary_idx mz_dataflow_addresse mz_dataflow_channels_per_worker_s2_primary_idx mz_dataflow_channels_per_worker mz_introspection {id,worker_id} mz_dataflow_operator_reachability_raw_s2_primary_idx mz_dataflow_operator_reachability_raw mz_introspection {address,port,worker_id,update_type,time} mz_dataflow_operators_per_worker_s2_primary_idx mz_dataflow_operators_per_worker mz_introspection {id,worker_id} +mz_dataflow_shutdown_durations_histogram_raw_s2_primary_idx mz_dataflow_shutdown_durations_histogram_raw mz_introspection {worker_id,duration_ns} mz_message_counts_received_raw_s2_primary_idx mz_message_counts_received_raw mz_introspection {channel_id,from_worker_id,to_worker_id} mz_message_counts_sent_raw_s2_primary_idx mz_message_counts_sent_raw mz_introspection {channel_id,from_worker_id,to_worker_id} mz_peek_durations_histogram_raw_s2_primary_idx mz_peek_durations_histogram_raw mz_introspection {worker_id,duration_ns} diff --git a/test/testdrive/introspection-sources.td b/test/testdrive/introspection-sources.td index eeb8c1c647c7..2c51a4c09c8e 100644 --- a/test/testdrive/introspection-sources.td +++ b/test/testdrive/introspection-sources.td @@ -282,9 +282,9 @@ mv1_primary_idx mv1 > CREATE DEFAULT INDEX ii_arr ON vv_arr -> SELECT records >= 300 FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_arr' OR name='mv_arr' -true -true +> SELECT records >= 300, size >= 300*96, capacity >= 300*96 FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_arr' OR name='mv_arr' +true true true +true true true # Test that non-arranging dataflows show up in `mz_dataflow_arrangement_sizes` @@ -292,5 +292,28 @@ true > CREATE DEFAULT INDEX ii_empty ON t3 -> SELECT records, batches FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_empty' -0 0 +> SELECT records, batches, size, capacity, allocations FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_empty' +0 0 0 0 0 + +# Tests that arrangement sizes are approximate + +> CREATE TABLE t4 (c int8) + +> CREATE INDEX ii_t4 ON t4(c) + +> SELECT records, batches, size, capacity, allocations FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_t4' +0 0 0 0 0 + +> INSERT INTO t4 SELECT 1 + +> SELECT records, batches, size > 96 AND size < 2*96, capacity > 96, allocations > 0 FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_t4' +1 1 true true true + +> INSERT INTO t4 SELECT generate_series(1, 1000) + +> SELECT records >= 1000 AND records <= 1001, batches > 0, size > 96*1000 AND size < 2*96*1000, capacity > 96*1000, allocations > 0 FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_t4' +true true true true true + +> DROP INDEX ii_t4 + +> SELECT records, batches, size, capacity, allocations FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_t4' diff --git a/test/testdrive/logging.td b/test/testdrive/logging.td index a36f17ffbf0c..a400c289772b 100644 --- a/test/testdrive/logging.td +++ b/test/testdrive/logging.td @@ -75,6 +75,9 @@ $ set-regex match=s\d+ replacement=SID > SELECT count(*) FROM (SELECT count (*) FROM mz_internal.mz_peek_durations_histogram); 1 +> SELECT count(*) FROM (SELECT count (*) FROM mz_internal.mz_dataflow_shutdown_durations_histogram); +1 + > SELECT count(*) FROM (SELECT count (*) FROM mz_internal.mz_records_per_dataflow); 1 @@ -170,6 +173,9 @@ SID operator_id 1 uint8 SID worker_id 2 uint8 SID records 3 bigint SID batches 4 bigint +SID size 5 bigint +SID capacity 6 bigint +SID allocations 7 bigint > SELECT mz_columns.id, mz_columns.name, position, type FROM mz_views JOIN mz_columns USING (id) From 4c217ec745af9bfbd3c64ce14176fb3e13c40062 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Wed, 28 Jun 2023 14:02:45 +0200 Subject: [PATCH 02/10] Revert "Backport 0.57.9 reverts" Signed-off-by: Moritz Hoffmann --- Cargo.lock | 2 +- clippy.toml | 3 +- src/compute/Cargo.toml | 1 - src/compute/src/extensions/collection.rs | 108 ----------------------- src/compute/src/extensions/mod.rs | 1 - src/compute/src/render/mod.rs | 3 +- src/compute/src/render/reduce.rs | 7 +- src/compute/src/render/top_k.rs | 7 +- src/timely-util/Cargo.toml | 1 + src/timely-util/src/operator.rs | 62 ++++++++++++- 10 files changed, 72 insertions(+), 123 deletions(-) delete mode 100644 src/compute/src/extensions/collection.rs diff --git a/Cargo.lock b/Cargo.lock index b2218631c141..40141ca2a98f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3704,7 +3704,6 @@ dependencies = [ name = "mz-compute" version = "0.0.0" dependencies = [ - "ahash", "anyhow", "async-trait", "bytesize", @@ -5261,6 +5260,7 @@ dependencies = [ name = "mz-timely-util" version = "0.0.0" dependencies = [ + "ahash", "differential-dataflow", "futures-util", "mz-ore", diff --git a/clippy.toml b/clippy.toml index 0e5b6a50d050..d88225b6f1e9 100644 --- a/clippy.toml +++ b/clippy.toml @@ -24,8 +24,7 @@ disallowed-methods = [ # Prevent access to Differential APIs that want to use the default trace or use a default name, or where we offer # our own wrapper - { path = "differential_dataflow::Collection::consolidate", reason = "use the `Consolidate::mz_consolidate` function instead" }, - { path = "differential_dataflow::Collection::consolidate_named", reason = "use the `Consolidate::mz_consolidate` function instead" }, + { path = "differential_dataflow::Collection::consolidate", reason = "use the `differential_dataflow::Collection::consolidate_named` function instead" }, { path = "differential_dataflow::operators::arrange::arrangement::Arrange::arrange", reason = "use the `MzArrange::mz_arrange_named` function instead" }, { path = "differential_dataflow::operators::arrange::arrangement::Arrange::arrange_named", reason = "use the `MzArrange::mz_arrange_named` function instead" }, { path = "differential_dataflow::operators::arrange::arrangement::Arrange::arrange_core", reason = "use the `MzArrange::mz_arrange_core` function instead" }, diff --git a/src/compute/Cargo.toml b/src/compute/Cargo.toml index 7fa561d8adb6..6b6c86c63a54 100644 --- a/src/compute/Cargo.toml +++ b/src/compute/Cargo.toml @@ -7,7 +7,6 @@ rust-version.workspace = true publish = false [dependencies] -ahash = { version = "0.8.0", default_features = false } anyhow = "1.0.66" async-trait = "0.1.68" bytesize = "1.1.0" diff --git a/src/compute/src/extensions/collection.rs b/src/compute/src/extensions/collection.rs deleted file mode 100644 index 513e814c3055..000000000000 --- a/src/compute/src/extensions/collection.rs +++ /dev/null @@ -1,108 +0,0 @@ -// 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. - -//! Extensions to Differential collections. - -use std::hash::{BuildHasher, Hash, Hasher}; - -use differential_dataflow::difference::Semigroup; -use differential_dataflow::lattice::Lattice; -use differential_dataflow::operators::arrange::{Arranged, TraceAgent}; -use differential_dataflow::trace::{Batch, Trace, TraceReader}; -use differential_dataflow::{Collection, Data, ExchangeData}; -use timely::container::columnation::Columnation; -use timely::dataflow::channels::pact::Exchange; -use timely::dataflow::Scope; - -use crate::extensions::arrange::{ArrangementSize, MzArrange}; - -/// Extension methods for differential [`Collection`]s. -pub(crate) trait ConsolidateExt -where - G: Scope, - G::Timestamp: Lattice + Data, - D1: ExchangeData + Hash, - R: Semigroup + ExchangeData, -{ - /// Consolidates the collection if `must_consolidate` is `true` and leaves it - /// untouched otherwise. - fn mz_consolidate_if(&self, must_consolidate: bool, name: &str) -> Self - where - Tr: Trace + TraceReader + 'static, - Tr::Batch: Batch, - Arranged>: ArrangementSize; - - /// Consolidates the collection. - fn mz_consolidate(&self, name: &str) -> Self - where - Tr: Trace + TraceReader + 'static, - Tr::Batch: Batch, - Arranged>: ArrangementSize; -} - -impl ConsolidateExt for Collection -where - G: Scope, - G::Timestamp: Lattice + Data, - D1: ExchangeData + Hash + Columnation, - R: Semigroup + ExchangeData + Columnation, -{ - fn mz_consolidate_if(&self, must_consolidate: bool, name: &str) -> Self - where - Tr: Trace + TraceReader + 'static, - Tr::Batch: Batch, - Arranged>: ArrangementSize, - { - if must_consolidate { - self.mz_consolidate(name) - } else { - self.clone() - } - } - - fn mz_consolidate(&self, name: &str) -> Self - where - Tr: Trace + TraceReader + 'static, - Tr::Batch: Batch, - Arranged>: ArrangementSize, - { - // We employ AHash below instead of the default hasher in DD to obtain - // a better distribution of data to workers. AHash claims empirically - // both speed and high quality, according to - // https://github.com/tkaitchuck/aHash/blob/master/compare/readme.md. - // TODO(vmarcos): Consider here if it is worth it to spend the time to - // implement twisted tabulation hashing as proposed in Mihai Patrascu, - // Mikkel Thorup: Twisted Tabulation Hashing. SODA 2013: 209-228, available - // at https://epubs.siam.org/doi/epdf/10.1137/1.9781611973105.16. The latter - // would provide good bounds for balls-into-bins problems when the number of - // bins is small (as is our case), so we'd have a theoretical guarantee. - // NOTE: We fix the seeds of a RandomState instance explicity with the same - // seeds that would be given by `AHash` via ahash::AHasher::default() so as - // to avoid a different selection due to compile-time features being differently - // selected in other dependencies using `AHash` vis-à-vis cargo's strategy - // of unioning features. - // NOTE: Depending on target features, we may end up employing the fallback - // hasher of `AHash`, but it should be sufficient for our needs. - let random_state = ahash::RandomState::with_seeds( - 0x243f_6a88_85a3_08d3, - 0x1319_8a2e_0370_7344, - 0xa409_3822_299f_31d0, - 0x082e_fa98_ec4e_6c89, - ); - let exchange = Exchange::new(move |update: &((D1, _), G::Timestamp, R)| { - let data = &(update.0).0; - let mut h = random_state.build_hasher(); - data.hash(&mut h); - h.finish() - }); - self.map(|k| (k, ())) - .mz_arrange_core::<_, Tr>(exchange, name) - .as_collection(|d: &D1, _| d.clone()) - } -} diff --git a/src/compute/src/extensions/mod.rs b/src/compute/src/extensions/mod.rs index 72a9abc2aadd..aecfbc006b12 100644 --- a/src/compute/src/extensions/mod.rs +++ b/src/compute/src/extensions/mod.rs @@ -12,5 +12,4 @@ //! Operator extensions to Timely and Differential pub(crate) mod arrange; -pub(crate) mod collection; pub(crate) mod reduce; diff --git a/src/compute/src/render/mod.rs b/src/compute/src/render/mod.rs index 8f8a0722d78d..3244324cd530 100644 --- a/src/compute/src/render/mod.rs +++ b/src/compute/src/render/mod.rs @@ -132,7 +132,6 @@ use timely::PartialOrder; use crate::arrangement::manager::TraceBundle; use crate::compute_state::ComputeState; use crate::extensions::arrange::{KeyCollection, MzArrange}; -use crate::extensions::collection::ConsolidateExt; use crate::extensions::reduce::MzReduce; use crate::logging::compute::LogImportFrontiers; use crate::render::context::{ArrangementFlavor, Context, ShutdownToken}; @@ -653,7 +652,7 @@ where let (oks_v, err_v) = variables.remove(&Id::Local(*id)).unwrap(); // Set oks variable to `oks` but consolidated to ensure iteration ceases at fixed point. - let mut oks = oks.mz_consolidate::>("LetRecConsolidation"); + let mut oks = oks.consolidate_named::>("LetRecConsolidation"); if let Some(token) = &self.shutdown_token.get_inner() { oks = oks.with_token(Weak::clone(token)); } diff --git a/src/compute/src/render/reduce.rs b/src/compute/src/render/reduce.rs index 8e96b1b25130..9421f00cbab5 100644 --- a/src/compute/src/render/reduce.rs +++ b/src/compute/src/render/reduce.rs @@ -35,7 +35,6 @@ use timely::progress::Timestamp; use tracing::warn; use crate::extensions::arrange::{KeyCollection, MzArrange}; -use crate::extensions::collection::ConsolidateExt; use crate::extensions::reduce::{MzReduce, ReduceExt}; use crate::render::context::{Arrangement, CollectionBundle, Context, KeyArrangement}; use crate::render::errors::MaybeValidatingRow; @@ -719,7 +718,9 @@ where stage = negated_output .negate() .concat(&input) - .mz_consolidate::>("Consolidated MinsMaxesHierarchical"); + .consolidate_named::>( + "Consolidated MinsMaxesHierarchical", + ); } // Discard the hash from the key and return to the format of the input data. @@ -869,7 +870,7 @@ where (key, values) }) - .mz_consolidate_if::>( + .consolidate_named_if::>( must_consolidate, "Consolidated ReduceMonotonic input", ); diff --git a/src/compute/src/render/top_k.rs b/src/compute/src/render/top_k.rs index f0c48b031a8a..80b83f4ebe4a 100644 --- a/src/compute/src/render/top_k.rs +++ b/src/compute/src/render/top_k.rs @@ -31,7 +31,6 @@ use timely::dataflow::operators::Operator; use timely::dataflow::Scope; use crate::extensions::arrange::MzArrange; -use crate::extensions::collection::ConsolidateExt; use crate::extensions::reduce::MzReduce; use crate::render::context::{CollectionBundle, Context}; use crate::render::errors::MaybeValidatingRow; @@ -91,7 +90,7 @@ where }; (group_row, row) }) - .mz_consolidate_if::>( + .consolidate_named_if::>( must_consolidate, "Consolidated MonotonicTopK input", ); @@ -301,7 +300,7 @@ where ( oks.negate() .concat(&input) - .mz_consolidate::>("Consolidated TopK"), + .consolidate_named::>("Consolidated TopK"), errs, ) } @@ -335,7 +334,7 @@ where (group_key, row) } }) - .mz_consolidate_if::>( + .consolidate_named_if::>( must_consolidate, "Consolidated MonotonicTop1 input", ); diff --git a/src/timely-util/Cargo.toml b/src/timely-util/Cargo.toml index dc454a9f92b1..f2d1a1ef3e04 100644 --- a/src/timely-util/Cargo.toml +++ b/src/timely-util/Cargo.toml @@ -17,6 +17,7 @@ polonius-the-crab = "0.3.1" workspace-hack = { version = "0.0.0", path = "../workspace-hack" } tokio = { version = "1.24.2", features = ["macros", "rt-multi-thread", "time"] } num-traits = "0.2" +ahash = { version = "0.8.0", default_features = false } [package.metadata.cargo-udeps.ignore] normal = ["workspace-hack"] diff --git a/src/timely-util/src/operator.rs b/src/timely-util/src/operator.rs index dd2bf1d07d74..10907b370ecc 100644 --- a/src/timely-util/src/operator.rs +++ b/src/timely-util/src/operator.rs @@ -10,12 +10,15 @@ //! Common operator transformations on timely streams and differential collections. use std::future::Future; +use std::hash::{BuildHasher, Hash, Hasher}; use std::rc::Weak; use differential_dataflow::difference::{Multiply, Semigroup}; use differential_dataflow::lattice::Lattice; +use differential_dataflow::operators::arrange::Arrange; +use differential_dataflow::trace::{Batch, Trace, TraceReader}; use differential_dataflow::{AsCollection, Collection}; -use timely::dataflow::channels::pact::{ParallelizationContract, Pipeline}; +use timely::dataflow::channels::pact::{Exchange, ParallelizationContract, Pipeline}; use timely::dataflow::channels::pushers::Tee; use timely::dataflow::operators::generic::builder_rc::OperatorBuilder as OperatorBuilderRc; use timely::dataflow::operators::generic::operator::{self, Operator}; @@ -221,6 +224,16 @@ where /// the provided token can be upgraded. Once the token cannot be upgraded anymore, all data /// flowing into the operator is dropped. fn with_token(&self, token: Weak<()>) -> Collection; + + /// Consolidates the collection if `must_consolidate` is `true` and leaves it + /// untouched otherwise. + fn consolidate_named_if(self, must_consolidate: bool, name: &str) -> Self + where + D1: differential_dataflow::ExchangeData + Hash, + R: Semigroup + differential_dataflow::ExchangeData, + G::Timestamp: Lattice, + Tr: Trace + TraceReader + 'static, + Tr::Batch: Batch; } impl StreamExt for Stream @@ -526,6 +539,53 @@ where fn with_token(&self, token: Weak<()>) -> Collection { self.inner.with_token(token).as_collection() } + + fn consolidate_named_if(self, must_consolidate: bool, name: &str) -> Self + where + D1: differential_dataflow::ExchangeData + Hash, + R: Semigroup + differential_dataflow::ExchangeData, + G::Timestamp: Lattice + Ord, + Tr: Trace + TraceReader + 'static, + Tr::Batch: Batch, + { + if must_consolidate { + // We employ AHash below instead of the default hasher in DD to obtain + // a better distribution of data to workers. AHash claims empirically + // both speed and high quality, according to + // https://github.com/tkaitchuck/aHash/blob/master/compare/readme.md. + // TODO(vmarcos): Consider here if it is worth it to spend the time to + // implement twisted tabulation hashing as proposed in Mihai Patrascu, + // Mikkel Thorup: Twisted Tabulation Hashing. SODA 2013: 209-228, available + // at https://epubs.siam.org/doi/epdf/10.1137/1.9781611973105.16. The latter + // would provide good bounds for balls-into-bins problems when the number of + // bins is small (as is our case), so we'd have a theoretical guarantee. + // NOTE: We fix the seeds of a RandomState instance explicity with the same + // seeds that would be given by `AHash` via ahash::AHasher::default() so as + // to avoid a different selection due to compile-time features being differently + // selected in other dependencies using `AHash` vis-à-vis cargo's strategy + // of unioning features. + // NOTE: Depending on target features, we may end up employing the fallback + // hasher of `AHash`, but it should be sufficient for our needs. + let random_state = ahash::RandomState::with_seeds( + 0x243f_6a88_85a3_08d3, + 0x1319_8a2e_0370_7344, + 0xa409_3822_299f_31d0, + 0x082e_fa98_ec4e_6c89, + ); + let exchange = Exchange::new(move |update: &((D1, _), G::Timestamp, R)| { + let data = &(update.0).0; + let mut h = random_state.build_hasher(); + data.hash(&mut h); + h.finish() + }); + // Access to `arrange_core` is OK because we specify the trace and don't hold on to it. + #[allow(clippy::disallowed_methods)] + self.arrange_core::<_, Tr>(exchange, name) + .as_collection(|k, _v| k.clone()) + } else { + self + } + } } /// Creates a new async data stream source for a scope. From 7cefacf3edc18791c641d343b431d752ced18d38 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Tue, 11 Jul 2023 14:12:59 +0200 Subject: [PATCH 03/10] Track trace usage in Materialize through a custom wrapper type MzArranged. This tracks whether any client uses the trace, and only then installs the arrangement size logging operator. Signed-off-by: Moritz Hoffmann --- src/compute/src/extensions/arrange.rs | 252 ++++++++++++++++++--- src/compute/src/extensions/reduce.rs | 44 ++-- src/compute/src/logging/compute.rs | 2 +- src/compute/src/logging/differential.rs | 2 +- src/compute/src/logging/initialize.rs | 2 +- src/compute/src/logging/reachability.rs | 2 +- src/compute/src/logging/timely.rs | 2 +- src/compute/src/render/context.rs | 21 +- src/compute/src/render/join/delta_join.rs | 15 +- src/compute/src/render/join/linear_join.rs | 11 +- src/compute/src/render/mod.rs | 12 +- src/compute/src/render/threshold.rs | 8 +- 12 files changed, 282 insertions(+), 91 deletions(-) diff --git a/src/compute/src/extensions/arrange.rs b/src/compute/src/extensions/arrange.rs index c1d8271de871..56fd35d51f5f 100644 --- a/src/compute/src/extensions/arrange.rs +++ b/src/compute/src/extensions/arrange.rs @@ -12,13 +12,18 @@ use differential_dataflow::lattice::Lattice; use differential_dataflow::operators::arrange::{Arrange, Arranged, TraceAgent}; use differential_dataflow::trace::{Batch, Trace, TraceReader}; use differential_dataflow::{Collection, Data, ExchangeData, Hashable}; +use std::cell::RefCell; +use std::rc::Rc; use timely::container::columnation::Columnation; use timely::dataflow::channels::pact::{ParallelizationContract, Pipeline}; use timely::dataflow::operators::Operator; -use timely::dataflow::{Scope, ScopeParent}; +use timely::dataflow::scopes::Child; +use timely::dataflow::{Scope, ScopeParent, StreamCore}; +use timely::progress::timestamp::Refines; use timely::progress::{Antichain, Timestamp}; use crate::logging::compute::ComputeEvent; +use crate::render::context::{ArrangementImport, ErrArrangementImport}; use crate::typedefs::{RowKeySpine, RowSpine}; /// Extension trait to arrange data. @@ -40,7 +45,7 @@ where /// This operator arranges a stream of values into a shared trace, whose contents it maintains. /// This trace is current for all times marked completed in the output stream, and probing this stream /// is the correct way to determine that times in the shared trace are committed. - fn mz_arrange(&self, name: &str) -> Arranged> + fn mz_arrange(&self, name: &str) -> MzArranged> where Self::Key: ExchangeData + Hashable, Self::Val: ExchangeData, @@ -52,8 +57,7 @@ where Time = ::Timestamp, R = Self::R, > + 'static, - Tr::Batch: Batch, - Arranged>: ArrangementSize; + Tr::Batch: Batch; /// Arranges a stream of `(Key, Val)` updates by `Key` into a trace of type `Tr`. Partitions /// the data according to `pact`. @@ -61,7 +65,11 @@ where /// This operator arranges a stream of values into a shared trace, whose contents it maintains. /// This trace is current for all times marked completed in the output stream, and probing this stream /// is the correct way to determine that times in the shared trace are committed. - fn mz_arrange_core(&self, pact: P, name: &str) -> Arranged> + fn mz_arrange_core( + &self, + pact: P, + name: &str, + ) -> MzArranged> where P: ParallelizationContract< ::Timestamp, @@ -78,8 +86,7 @@ where Time = ::Timestamp, R = Self::R, > + 'static, - Tr::Batch: Batch, - Arranged>: ArrangementSize; + Tr::Batch: Batch; } impl MzArrange for Collection @@ -95,30 +102,28 @@ where type Val = V; type R = R; - fn mz_arrange(&self, name: &str) -> Arranged> + fn mz_arrange(&self, name: &str) -> MzArranged> where K: ExchangeData + Hashable, V: ExchangeData, R: ExchangeData, Tr: Trace + TraceReader + 'static, Tr::Batch: Batch, - Arranged>: ArrangementSize, { // Allow access to `arrange_named` because we're within Mz's wrapper. #[allow(clippy::disallowed_methods)] - self.arrange_named(name).log_arrangement_size() + self.arrange_named(name).into() } - fn mz_arrange_core(&self, pact: P, name: &str) -> Arranged> + fn mz_arrange_core(&self, pact: P, name: &str) -> MzArranged> where P: ParallelizationContract, Tr: Trace + TraceReader + 'static, Tr::Batch: Batch, - Arranged>: ArrangementSize, { // Allow access to `arrange_named` because we're within Mz's wrapper. #[allow(clippy::disallowed_methods)] - self.arrange_core(pact, name).log_arrangement_size() + self.arrange_core(pact, name).into() } } @@ -145,23 +150,21 @@ where type Val = (); type R = R; - fn mz_arrange(&self, name: &str) -> Arranged> + fn mz_arrange(&self, name: &str) -> MzArranged> where K: ExchangeData + Hashable, R: ExchangeData, Tr: Trace + TraceReader + 'static, Tr::Batch: Batch, - Arranged>: ArrangementSize, { self.0.map(|d| (d, ())).mz_arrange(name) } - fn mz_arrange_core(&self, pact: P, name: &str) -> Arranged> + fn mz_arrange_core(&self, pact: P, name: &str) -> MzArranged> where P: ParallelizationContract, Tr: Trace + TraceReader + 'static, Tr::Batch: Batch, - Arranged>: ArrangementSize, { self.0.map(|d| (d, ())).mz_arrange_core(pact, name) } @@ -170,7 +173,7 @@ where /// A type that can log its heap size. pub trait ArrangementSize { /// Install a logger to track the heap size of the target. - fn log_arrangement_size(self) -> Self; + fn log_arrangement_size(&self); } /// Helper to compute the size of a vector in memory. @@ -188,10 +191,7 @@ fn vec_size(data: &Vec, mut callback: impl FnMut(usize, usize)) { /// * `arranged`: The arrangement to inspect. /// * `logic`: Closure that calculates the heap size/capacity/allocations for a trace. The return /// value are size and capacity in bytes, and number of allocations, all in absolute values. -fn log_arrangement_size_inner( - arranged: Arranged>, - mut logic: L, -) -> Arranged> +fn log_arrangement_size_inner(arranged: &Arranged>, mut logic: L) where G: Scope, G::Timestamp: Timestamp + Lattice + Ord, @@ -200,13 +200,20 @@ where L: FnMut(&TraceAgent) -> (usize, usize, usize) + 'static, { let scope = arranged.stream.scope(); - let Some(logger) = scope.log_register().get::("materialize/compute") else {return arranged}; + let logger = if let Some(logger) = scope + .log_register() + .get::("materialize/compute") + { + logger + } else { + return; + }; let mut trace = arranged.trace.clone(); let operator = trace.operator().global_id; let (mut old_size, mut old_capacity, mut old_allocations) = (0isize, 0isize, 0isize); - let stream = arranged + arranged .stream .unary(Pipeline, "ArrangementSize", |_cap, info| { let mut buffer = Default::default(); @@ -255,13 +262,9 @@ where old_allocations = allocations; } }); - Arranged { - trace: arranged.trace, - stream, - } } -impl ArrangementSize for Arranged>> +impl ArrangementSize for MzArranged>> where G: Scope, G::Timestamp: Lattice + Ord, @@ -270,8 +273,8 @@ where T: Lattice + Timestamp, R: Semigroup, { - fn log_arrangement_size(self) -> Self { - log_arrangement_size_inner(self, |trace| { + fn log_arrangement_size(&self) { + log_arrangement_size_inner(&self.arranged, |trace| { let (mut size, mut capacity, mut allocations) = (0, 0, 0); let mut callback = |siz, cap| { allocations += 1; @@ -286,11 +289,11 @@ where vec_size(&batch.layer.vals.vals.vals, &mut callback); }); (size, capacity, allocations) - }) + }); } } -impl ArrangementSize for Arranged>> +impl ArrangementSize for MzArranged>> where G: Scope, G::Timestamp: Lattice + Ord, @@ -298,8 +301,8 @@ where T: Lattice + Timestamp, R: Semigroup, { - fn log_arrangement_size(self) -> Self { - log_arrangement_size_inner(self, |trace| { + fn log_arrangement_size(&self) { + log_arrangement_size_inner(&self.arranged, |trace| { let (mut size, mut capacity, mut allocations) = (0, 0, 0); let mut callback = |siz, cap| { allocations += 1; @@ -312,6 +315,183 @@ where vec_size(&batch.layer.vals.vals, &mut callback); }); (size, capacity, allocations) - }) + }); + } +} + +impl ArrangementSize for ArrangementImport +where + G: Scope, + G::Timestamp: Lattice, + G::Timestamp: Refines, + T: Timestamp + Lattice, + V: Data + Columnation, +{ + fn log_arrangement_size(&self) { + // Imported traces are logged during export, so we don't have to log anything here. + } +} + +impl ArrangementSize for ErrArrangementImport +where + G: Scope, + G::Timestamp: Lattice, + G::Timestamp: Refines, + T: Timestamp + Lattice, +{ + fn log_arrangement_size(&self) { + // Imported traces are logged during export, so we don't have to log anything here. + } +} + +/// A wrapper behaving similar to Differential's [`Arranged`] type. +/// +/// This type enables logging arrangement sizes, but only if the trace is consumed downstream. +#[derive(Clone)] +pub struct MzArranged +where + G: Scope, + G::Timestamp: Lattice, + Tr: TraceReader, + compute_state: &ComputeState, config: &LoggingConfig, ) -> (super::compute::Logger, BTreeMap) { let interval_ms = std::cmp::max(1, config.interval.as_millis()) @@ -49,6 +51,7 @@ pub fn initialize( let mut context = LoggingContext { worker, config, + compute_state, interval_ms, now, start_offset, @@ -77,6 +80,7 @@ pub fn initialize( struct LoggingContext<'a, A: Allocate> { worker: &'a mut timely::worker::Worker, config: &'a LoggingConfig, + compute_state: &'a ComputeState, interval_ms: u64, now: Instant, start_offset: Duration, @@ -95,23 +99,27 @@ impl LoggingContext<'_, A> { self.config, self.t_event_queue.clone(), Rc::clone(&self.shared_state), + self.compute_state, )); traces.extend(super::reachability::construct( self.worker, self.config, self.r_event_queue.clone(), + self.compute_state, )); traces.extend(super::differential::construct( self.worker, self.config, self.d_event_queue.clone(), Rc::clone(&self.shared_state), + self.compute_state, )); traces.extend(super::compute::construct( self.worker, self.config, self.c_event_queue.clone(), Rc::clone(&self.shared_state), + self.compute_state, )); let errs = self @@ -119,7 +127,9 @@ impl LoggingContext<'_, A> { .dataflow_named("Dataflow: logging errors", |scope| { let collection: KeyCollection<_, DataflowError, Diff> = Collection::empty(scope).into(); - collection.mz_arrange("Arrange logging err").trace() + collection + .mz_arrange("Arrange logging err") + .trace(self.compute_state.enable_arrangement_size_logging) }); traces diff --git a/src/compute/src/logging/reachability.rs b/src/compute/src/logging/reachability.rs index be16313dd035..fe83f965070b 100644 --- a/src/compute/src/logging/reachability.rs +++ b/src/compute/src/logging/reachability.rs @@ -14,6 +14,7 @@ use std::collections::BTreeMap; use std::convert::TryInto; use std::rc::Rc; +use crate::compute_state::ComputeState; use differential_dataflow::AsCollection; use mz_compute_client::logging::LoggingConfig; use mz_expr::{permutation_for_arrangement, MirScalarExpr}; @@ -48,6 +49,7 @@ pub(super) fn construct( worker: &mut timely::worker::Worker, config: &LoggingConfig, event_queue: EventQueue, + compute_state: &ComputeState, ) -> BTreeMap)> { let interval_ms = std::cmp::max(1, config.interval.as_millis()); @@ -149,7 +151,7 @@ pub(super) fn construct( let trace = updates .mz_arrange::>(&format!("Arrange {:?}", variant)) - .trace(); + .trace(compute_state.enable_arrangement_size_logging); result.insert(variant.clone(), (trace, Rc::clone(&token))); } } diff --git a/src/compute/src/logging/timely.rs b/src/compute/src/logging/timely.rs index fef916fd9a06..17cf1273ce22 100644 --- a/src/compute/src/logging/timely.rs +++ b/src/compute/src/logging/timely.rs @@ -15,6 +15,7 @@ use std::collections::BTreeMap; use std::rc::Rc; use std::time::Duration; +use crate::compute_state::ComputeState; use differential_dataflow::collection::AsCollection; use mz_compute_client::logging::LoggingConfig; use mz_expr::{permutation_for_arrangement, MirScalarExpr}; @@ -53,6 +54,7 @@ pub(super) fn construct( config: &LoggingConfig, event_queue: EventQueue, shared_state: Rc>, + compute_state: &ComputeState, ) -> BTreeMap)> { let logging_interval_ms = std::cmp::max(1, config.interval.as_millis()); let worker_id = worker.index(); @@ -288,7 +290,7 @@ pub(super) fn construct( } }) .mz_arrange::>(&format!("ArrangeByKey {:?}", variant)) - .trace(); + .trace(compute_state.enable_arrangement_size_logging); traces.insert(variant.clone(), (trace, Rc::clone(&token))); } } diff --git a/src/compute/src/render/context.rs b/src/compute/src/render/context.rs index 3088583659c2..69df925b8e36 100644 --- a/src/compute/src/render/context.rs +++ b/src/compute/src/render/context.rs @@ -95,6 +95,7 @@ where pub(super) shutdown_token: ShutdownToken, /// The implementation to use for rendering linear joins. pub(super) linear_join_impl: LinearJoinImpl, + pub(super) enable_arrangement_size_logging: bool, } impl Context @@ -122,6 +123,7 @@ where bindings: BTreeMap::new(), shutdown_token: Default::default(), linear_join_impl: Default::default(), + enable_arrangement_size_logging: Default::default(), } } } diff --git a/src/compute/src/render/join/delta_join.rs b/src/compute/src/render/join/delta_join.rs index 3b17a803c9b0..24c4159fc0a0 100644 --- a/src/compute/src/render/join/delta_join.rs +++ b/src/compute/src/render/join/delta_join.rs @@ -203,6 +203,7 @@ where |t1, t2| t1.le(t2), closure, self.shutdown_token.clone(), + self.enable_arrangement_size_logging, ) } else { build_halfjoin( @@ -213,6 +214,7 @@ where |t1, t2| t1.lt(t2), closure, self.shutdown_token.clone(), + self.enable_arrangement_size_logging, ) } } @@ -226,6 +228,7 @@ where |t1, t2| t1.le(t2), closure, self.shutdown_token.clone(), + self.enable_arrangement_size_logging, ) } else { build_halfjoin( @@ -236,6 +239,7 @@ where |t1, t2| t1.lt(t2), closure, self.shutdown_token.clone(), + self.enable_arrangement_size_logging, ) } } @@ -317,6 +321,7 @@ fn build_halfjoin( comparison: CF, closure: JoinClosure, shutdown_token: ShutdownToken, + enable_arrangement_size_logging: bool, ) -> ( Collection, Collection, @@ -356,7 +361,7 @@ where if closure.could_error() { let (oks, errs2) = dogsdogsdogs::operators::half_join::half_join_internal_unsafe( &updates, - unsafe { trace.inner() }, + unsafe { trace.inner(enable_arrangement_size_logging) }, |time| time.step_back(), comparison, // TODO(mcsherry): investigate/establish trade-offs here; time based had problems, @@ -392,7 +397,7 @@ where } else { let oks = dogsdogsdogs::operators::half_join::half_join_internal_unsafe( &updates, - unsafe { trace.inner() }, + unsafe { trace.inner(enable_arrangement_size_logging) }, |time| time.step_back(), comparison, // TODO(mcsherry): investigate/establish trade-offs here; time based had problems, diff --git a/src/compute/src/render/join/linear_join.rs b/src/compute/src/render/join/linear_join.rs index 73c5016bf37d..19ed61cdd46d 100644 --- a/src/compute/src/render/join/linear_join.rs +++ b/src/compute/src/render/join/linear_join.rs @@ -160,6 +160,7 @@ where inputs[stage_plan.lookup_relation].enter_region(inner), stage_plan, &mut errors, + self.enable_arrangement_size_logging, ); // Update joined results and capture any errors. joined = JoinedFlavor::Collection(stream); @@ -217,6 +218,7 @@ fn differential_join( lookup_relation: _, }: LinearStagePlan, errors: &mut Vec>, + enable_arrangement_size_logging: bool, ) -> Collection where G: Scope, @@ -261,13 +263,25 @@ where } JoinedFlavor::Local(local) => match arrangement { ArrangementFlavor::Local(oks, errs1) => { - let (oks, errs2) = differential_join_inner(join_impl, local, oks, closure); + let (oks, errs2) = differential_join_inner( + join_impl, + local, + oks, + closure, + enable_arrangement_size_logging, + ); errors.push(errs1.as_collection(|k, _v| k.clone())); errors.extend(errs2); oks } ArrangementFlavor::Trace(_gid, oks, errs1) => { - let (oks, errs2) = differential_join_inner(join_impl, local, oks, closure); + let (oks, errs2) = differential_join_inner( + join_impl, + local, + oks, + closure, + enable_arrangement_size_logging, + ); errors.push(errs1.as_collection(|k, _v| k.clone())); errors.extend(errs2); oks @@ -275,13 +289,25 @@ where }, JoinedFlavor::Trace(trace) => match arrangement { ArrangementFlavor::Local(oks, errs1) => { - let (oks, errs2) = differential_join_inner(join_impl, trace, oks, closure); + let (oks, errs2) = differential_join_inner( + join_impl, + trace, + oks, + closure, + enable_arrangement_size_logging, + ); errors.push(errs1.as_collection(|k, _v| k.clone())); errors.extend(errs2); oks } ArrangementFlavor::Trace(_gid, oks, errs1) => { - let (oks, errs2) = differential_join_inner(join_impl, trace, oks, closure); + let (oks, errs2) = differential_join_inner( + join_impl, + trace, + oks, + closure, + enable_arrangement_size_logging, + ); errors.push(errs1.as_collection(|k, _v| k.clone())); errors.extend(errs2); oks @@ -301,6 +327,7 @@ fn differential_join_inner( prev_keyed: MzArranged, next_input: MzArranged, closure: JoinClosure, + enable_arrangement_size_logging: bool, ) -> ( Collection, Option>, @@ -318,8 +345,8 @@ where let mut datums = DatumVec::new(); let mut row_builder = Row::default(); - let prev_keyed = unsafe { prev_keyed.inner() }; - let next_input = unsafe { next_input.inner() }; + let prev_keyed = unsafe { prev_keyed.inner(enable_arrangement_size_logging) }; + let next_input = unsafe { next_input.inner(enable_arrangement_size_logging) }; if closure.could_error() { let (oks, err) = join_impl diff --git a/src/compute/src/render/mod.rs b/src/compute/src/render/mod.rs index 86f1bf34b5ed..4e05a5bef7b4 100644 --- a/src/compute/src/render/mod.rs +++ b/src/compute/src/render/mod.rs @@ -275,6 +275,8 @@ pub fn build_compute_dataflow( scope.clone().iterative::, _, _>(|region| { let mut context = Context::for_dataflow_in(&dataflow, region.clone()); context.linear_join_impl = compute_state.linear_join_impl; + context.enable_arrangement_size_logging = + compute_state.enable_arrangement_size_logging; for (id, (oks, errs)) in imported_sources.into_iter() { let bundle = crate::render::CollectionBundle::from_collections( @@ -329,6 +331,8 @@ pub fn build_compute_dataflow( scope.clone().region_named(&build_name, |region| { let mut context = Context::for_dataflow_in(&dataflow, region.clone()); context.linear_join_impl = compute_state.linear_join_impl; + context.enable_arrangement_size_logging = + compute_state.enable_arrangement_size_logging; for (id, (oks, errs)) in imported_sources.into_iter() { let bundle = crate::render::CollectionBundle::from_collections( @@ -500,7 +504,11 @@ where oks.stream().probe_notify_with(probes); compute_state.traces.set( idx_id, - TraceBundle::new(oks.trace(), errs.trace()).with_drop(needed_tokens), + TraceBundle::new( + oks.trace(self.enable_arrangement_size_logging), + errs.trace(self.enable_arrangement_size_logging), + ) + .with_drop(needed_tokens), ); } Some(ArrangementFlavor::Trace(gid, _, _)) => { @@ -572,7 +580,11 @@ where .mz_arrange("Arrange export iterative err"); compute_state.traces.set( idx_id, - TraceBundle::new(oks.trace(), errs.trace()).with_drop(needed_tokens), + TraceBundle::new( + oks.trace(self.enable_arrangement_size_logging), + errs.trace(self.enable_arrangement_size_logging), + ) + .with_drop(needed_tokens), ); } Some(ArrangementFlavor::Trace(gid, _, _)) => { @@ -691,6 +703,7 @@ where .mz_arrange::>("Arrange recursive err") .mz_reduce_abelian::<_, ErrSpine<_, _, _>>( "Distinct recursive err", + self.enable_arrangement_size_logging, move |_k, _s, t| t.push(((), 1)), ) .as_collection(|k, _| k.clone()); diff --git a/src/compute/src/render/reduce.rs b/src/compute/src/render/reduce.rs index 9421f00cbab5..39f6c8969f48 100644 --- a/src/compute/src/render/reduce.rs +++ b/src/compute/src/render/reduce.rs @@ -299,6 +299,7 @@ where .reduce_pair::<_, RowSpine<_, _, _, _>, _, ErrValSpine<_, _, _>>( "ReduceCollation", "ReduceCollation Errors", + self.enable_arrangement_size_logging, { let mut row_buf = Row::default(); move |_key, input, output| { @@ -430,6 +431,7 @@ where .reduce_pair::<_, RowSpine<_, _, _, _>, _, ErrValSpine<_, _, _>>( "DistinctBy", "DistinctByErrorCheck", + self.enable_arrangement_size_logging, |_key, _input, output| { // We're pushing an empty row here because the key is implicitly added by the // arrangement, and the permutation logic takes care of using the key part of the @@ -487,17 +489,21 @@ where } let output = differential_dataflow::collection::concatenate(&mut input.scope(), to_collect) .mz_arrange::>("Arranged ReduceFuseBasic input") - .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceFuseBasic", { - let mut row_buf = Row::default(); - move |_key, input, output| { - let mut row_packer = row_buf.packer(); - for ((_, row), _) in input.iter() { - let datum = row.unpack_first(); - row_packer.push(datum); + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>( + "ReduceFuseBasic", + self.enable_arrangement_size_logging, + { + let mut row_buf = Row::default(); + move |_key, input, output| { + let mut row_packer = row_buf.packer(); + for ((_, row), _) in input.iter() { + let datum = row.unpack_first(); + row_packer.push(datum); + } + output.push((row_buf.clone(), 1)); } - output.push((row_buf.clone(), 1)); - } - }); + }, + ); ( output, err_output.expect("expected to validate in at least one aggregate"), @@ -556,22 +562,26 @@ where } let arranged = partial.mz_arrange::>("Arranged ReduceInaccumulable"); - let oks = arranged.mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceInaccumulable", { - let mut row_buf = Row::default(); - move |_key, source, target| { - // We respect the multiplicity here (unlike in hierarchical aggregation) - // because we don't know that the aggregation method is not sensitive - // to the number of records. - let iter = source.iter().flat_map(|(v, w)| { - // Note that in the non-positive case, this is wrong, but harmless because - // our other reduction will produce an error. - let count = usize::try_from(*w).unwrap_or(0); - std::iter::repeat(v.iter().next().unwrap()).take(count) - }); - row_buf.packer().push(func.eval(iter, &RowArena::new())); - target.push((row_buf.clone(), 1)); - } - }); + let oks = arranged.mz_reduce_abelian::<_, RowSpine<_, _, _, _>>( + "ReduceInaccumulable", + self.enable_arrangement_size_logging, + { + let mut row_buf = Row::default(); + move |_key, source, target| { + // We respect the multiplicity here (unlike in hierarchical aggregation) + // because we don't know that the aggregation method is not sensitive + // to the number of records. + let iter = source.iter().flat_map(|(v, w)| { + // Note that in the non-positive case, this is wrong, but harmless because + // our other reduction will produce an error. + let count = usize::try_from(*w).unwrap_or(0); + std::iter::repeat(v.iter().next().unwrap()).take(count) + }); + row_buf.packer().push(func.eval(iter, &RowArena::new())); + target.push((row_buf.clone(), 1)); + } + }, + ); // Note that we would prefer to use `mz_timely_util::reduce::ReduceExt::reduce_pair` here, but // we then wouldn't be able to do this error check conditionally. See its documentation for the @@ -581,6 +591,7 @@ where let errs = arranged.mz_reduce_abelian::<_, ErrValSpine<_, _, _>>( "ReduceInaccumulable Error Check", + self.enable_arrangement_size_logging, move |_key, source, target| { // Negative counts would be surprising, but until we are 100% certain we won't // see them, we should report when we do. We may want to bake even more info @@ -618,6 +629,7 @@ where .mz_arrange::>("Arranged ReduceInaccumulable") .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>( "ReduceInaccumulable", + self.enable_arrangement_size_logging, move |_, source, t| { if let Some(err) = R::into_error() { for (value, count) in source.iter() { @@ -738,6 +750,7 @@ where let errs = arranged .mz_reduce_abelian::<_, ErrValSpine<_, _, _>>( "ReduceMinsMaxes Error Check", + self.enable_arrangement_size_logging, move |_key, source, target| { // Negative counts would be surprising, but until we are 100% certain we wont // see them, we should report when we do. We may want to bake even more info @@ -758,19 +771,23 @@ where err_output = Some(errs.leave_region()); } arranged - .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceMinsMaxes", { - let mut row_buf = Row::default(); - move |_key, source: &[(&Vec, Diff)], target: &mut Vec<(Row, Diff)>| { - let mut row_packer = row_buf.packer(); - for (aggr_index, func) in aggr_funcs.iter().enumerate() { - let iter = source - .iter() - .map(|(values, _cnt)| values[aggr_index].iter().next().unwrap()); - row_packer.push(func.eval(iter, &RowArena::new())); + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>( + "ReduceMinsMaxes", + self.enable_arrangement_size_logging, + { + let mut row_buf = Row::default(); + move |_key, source: &[(&Vec, Diff)], target: &mut Vec<(Row, Diff)>| { + let mut row_packer = row_buf.packer(); + for (aggr_index, func) in aggr_funcs.iter().enumerate() { + let iter = source.iter().map(|(values, _cnt)| { + values[aggr_index].iter().next().unwrap() + }); + row_packer.push(func.eval(iter, &RowArena::new())); + } + target.push((row_buf.clone(), 1)); } - target.push((row_buf.clone(), 1)); - } - }) + }, + ) .leave_region() }); (arranged_output, err_output) @@ -803,6 +820,7 @@ where arranged_input .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>( "Reduced Fallibly MinsMaxesHierarchical", + self.enable_arrangement_size_logging, move |key, source, target| { if let Some(err) = R::into_error() { // Should negative accumulations reach us, we should loudly complain. @@ -900,20 +918,24 @@ where let partial: KeyCollection<_, _, _> = partial.into(); let output = partial .mz_arrange::>>("ArrangeMonotonic") - .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("ReduceMonotonic", { - let mut row_buf = Row::default(); - move |_key, input, output| { - let mut row_packer = row_buf.packer(); - let accum = &input[0].1; - for monoid in accum.iter() { - use ReductionMonoid::*; - match monoid { - Min(row) | Max(row) => row_packer.extend(row.iter()), + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>( + "ReduceMonotonic", + self.enable_arrangement_size_logging, + { + let mut row_buf = Row::default(); + move |_key, input, output| { + let mut row_packer = row_buf.packer(); + let accum = &input[0].1; + for monoid in accum.iter() { + use ReductionMonoid::*; + match monoid { + Min(row) | Max(row) => row_packer.extend(row.iter()), + } } + output.push((row_buf.clone(), 1)); } - output.push((row_buf.clone(), 1)); - } - }); + }, + ); (output, errs) } @@ -1173,6 +1195,7 @@ where .mz_arrange::>("Arranged Accumulable") .mz_reduce_abelian::<_, RowKeySpine<_, _, _>>( "Reduced Accumulable", + self.enable_arrangement_size_logging, move |_k, _s, t| t.push(((), 1)), ) .as_collection(|k, _| k.clone()) @@ -1203,6 +1226,7 @@ where .reduce_pair::<_, RowSpine<_, _, _, _>, _, ErrValSpine<_, _, _>>( "ReduceAccumulable", "AccumulableErrorCheck", + self.enable_arrangement_size_logging, { let mut row_buf = Row::default(); move |_key, input, output| { diff --git a/src/compute/src/render/threshold.rs b/src/compute/src/render/threshold.rs index 232c96ae142a..d0af34fce966 100644 --- a/src/compute/src/render/threshold.rs +++ b/src/compute/src/render/threshold.rs @@ -29,6 +29,7 @@ use crate::typedefs::RowSpine; fn threshold_arrangement( arrangement: &R, name: &str, + enable_arrangement_size_logging: bool, logic: L, ) -> MzArranged>> where @@ -38,7 +39,7 @@ where R: MzReduce + ArrangementSize, L: Fn(&Diff) -> bool + 'static, { - arrangement.mz_reduce_abelian(name, move |_key, s, t| { + arrangement.mz_reduce_abelian(name, enable_arrangement_size_logging, move |_key, s, t| { for (record, count) in s.iter() { if logic(count) { t.push(((*record).clone(), *count)); @@ -54,6 +55,7 @@ where pub fn build_threshold_basic( input: CollectionBundle, key: Vec, + enable_arrangement_size_logging: bool, ) -> CollectionBundle where G: Scope, @@ -65,11 +67,21 @@ where .expect("Arrangement ensured to exist"); match arrangement { ArrangementFlavor::Local(oks, errs) => { - let oks = threshold_arrangement(&oks, "Threshold local", |count| *count > 0); + let oks = threshold_arrangement( + &oks, + "Threshold local", + enable_arrangement_size_logging, + |count| *count > 0, + ); CollectionBundle::from_expressions(key, ArrangementFlavor::Local(oks, errs)) } ArrangementFlavor::Trace(_, oks, errs) => { - let oks = threshold_arrangement(&oks, "Threshold trace", |count| *count > 0); + let oks = threshold_arrangement( + &oks, + "Threshold trace", + enable_arrangement_size_logging, + |count| *count > 0, + ); let errs: KeyCollection<_, _, _> = errs.as_collection(|k, _| k.clone()).into(); let errs = errs.mz_arrange("Arrange threshold basic err"); CollectionBundle::from_expressions(key, ArrangementFlavor::Local(oks, errs)) @@ -95,7 +107,7 @@ where // We do not need to apply the permutation here, // since threshold doesn't inspect the values, but only // their counts. - build_threshold_basic(input, key) + build_threshold_basic(input, key, self.enable_arrangement_size_logging) } } } diff --git a/src/compute/src/render/top_k.rs b/src/compute/src/render/top_k.rs index 80b83f4ebe4a..4945b79c05a5 100644 --- a/src/compute/src/render/top_k.rs +++ b/src/compute/src/render/top_k.rs @@ -277,7 +277,12 @@ where let input = collection.map(move |((key, hash), row)| ((key, hash % modulus), row)); let (oks, errs) = if validating { let stage = build_topk_negated_stage::>( - &input, order_key, offset, limit, arity, + &input, + order_key, + offset, + limit, + arity, + self.enable_arrangement_size_logging, ); let error_logger = self.error_logger(); @@ -293,7 +298,14 @@ where (oks, Some(errs)) } else { ( - build_topk_negated_stage::(&input, order_key, offset, limit, arity), + build_topk_negated_stage::( + &input, + order_key, + offset, + limit, + arity, + self.enable_arrangement_size_logging, + ), None, ) }; @@ -361,12 +373,16 @@ where let result = partial .map(|k| (k, ())) .mz_arrange::>("Arranged MonotonicTop1 partial") - .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("MonotonicTop1", { - move |_key, input, output| { - let accum: &monoids::Top1Monoid = &input[0].1; - output.push((accum.row.clone(), 1)); - } - }); + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>( + "MonotonicTop1", + self.enable_arrangement_size_logging, + { + move |_key, input, output| { + let accum: &monoids::Top1Monoid = &input[0].1; + output.push((accum.row.clone(), 1)); + } + }, + ); // TODO(#7331): Here we discard the arranged output. (result.as_collection(|_k, v| v.clone()), errs) } @@ -378,6 +394,7 @@ fn build_topk_negated_stage( offset: usize, limit: Option, arity: usize, + enable_arrangement_size_logging: bool, ) -> Collection where G: Scope, @@ -388,86 +405,90 @@ where // such that `input.concat(&negated_output.negate())` yields the correct TopK input .mz_arrange::>("Arranged TopK input") - .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>("Reduced TopK input", { - move |_key, source, target: &mut Vec<(R, Diff)>| { - if let Some(err) = R::into_error() { - for (row, diff) in source.iter() { - if diff.is_positive() { - continue; + .mz_reduce_abelian::<_, RowSpine<_, _, _, _>>( + "Reduced TopK input", + enable_arrangement_size_logging, + { + move |_key, source, target: &mut Vec<(R, Diff)>| { + if let Some(err) = R::into_error() { + for (row, diff) in source.iter() { + if diff.is_positive() { + continue; + } + target.push((err((*row).clone()), -1)); + return; } - target.push((err((*row).clone()), -1)); - return; } - } - // Determine if we must actually shrink the result set. - // TODO(benesch): avoid dangerous `as` conversion. - #[allow(clippy::as_conversions)] - let must_shrink = offset > 0 - || limit - .map(|l| source.iter().map(|(_, d)| *d).sum::() as usize > l) - .unwrap_or(false); - if !must_shrink { - return; - } - - // First go ahead and emit all records - for (row, diff) in source.iter() { - target.push((R::ok((*row).clone()), diff.clone())); - } - // local copies that may count down to zero. - let mut offset = offset; - let mut limit = limit; - - // The order in which we should produce rows. - let mut indexes = (0..source.len()).collect::>(); - // We decode the datums once, into a common buffer for efficiency. - // Each row should contain `arity` columns; we should check that. - let mut buffer = Vec::with_capacity(arity * source.len()); - for (index, row) in source.iter().enumerate() { - buffer.extend(row.0.iter()); - assert_eq!(buffer.len(), arity * (index + 1)); - } - let width = buffer.len() / source.len(); - - //todo: use arrangements or otherwise make the sort more performant? - indexes.sort_by(|left, right| { - let left = &buffer[left * width..][..width]; - let right = &buffer[right * width..][..width]; - // Note: source was originally ordered by the u8 array representation - // of rows, but left.cmp(right) uses Datum::cmp. - mz_expr::compare_columns(&order_key, left, right, || left.cmp(right)) - }); - - // We now need to lay out the data in order of `buffer`, but respecting - // the `offset` and `limit` constraints. - for index in indexes.into_iter() { - let (row, mut diff) = source[index]; - if !diff.is_positive() { - continue; - } - // If we are still skipping early records ... - if offset > 0 { - let to_skip = std::cmp::min(offset, usize::try_from(diff).unwrap()); - offset -= to_skip; - diff -= Diff::try_from(to_skip).unwrap(); - } - // We should produce at most `limit` records. + // Determine if we must actually shrink the result set. // TODO(benesch): avoid dangerous `as` conversion. #[allow(clippy::as_conversions)] - if let Some(limit) = &mut limit { - diff = std::cmp::min(diff, Diff::try_from(*limit).unwrap()); - *limit -= diff as usize; + let must_shrink = offset > 0 + || limit + .map(|l| source.iter().map(|(_, d)| *d).sum::() as usize > l) + .unwrap_or(false); + if !must_shrink { + return; } - // Output the indicated number of rows. - if diff > 0 { - // Emit retractions for the elements actually part of - // the set of TopK elements. - target.push((R::ok(row.clone()), -diff)); + + // First go ahead and emit all records + for (row, diff) in source.iter() { + target.push((R::ok((*row).clone()), diff.clone())); + } + // local copies that may count down to zero. + let mut offset = offset; + let mut limit = limit; + + // The order in which we should produce rows. + let mut indexes = (0..source.len()).collect::>(); + // We decode the datums once, into a common buffer for efficiency. + // Each row should contain `arity` columns; we should check that. + let mut buffer = Vec::with_capacity(arity * source.len()); + for (index, row) in source.iter().enumerate() { + buffer.extend(row.0.iter()); + assert_eq!(buffer.len(), arity * (index + 1)); + } + let width = buffer.len() / source.len(); + + //todo: use arrangements or otherwise make the sort more performant? + indexes.sort_by(|left, right| { + let left = &buffer[left * width..][..width]; + let right = &buffer[right * width..][..width]; + // Note: source was originally ordered by the u8 array representation + // of rows, but left.cmp(right) uses Datum::cmp. + mz_expr::compare_columns(&order_key, left, right, || left.cmp(right)) + }); + + // We now need to lay out the data in order of `buffer`, but respecting + // the `offset` and `limit` constraints. + for index in indexes.into_iter() { + let (row, mut diff) = source[index]; + if !diff.is_positive() { + continue; + } + // If we are still skipping early records ... + if offset > 0 { + let to_skip = std::cmp::min(offset, usize::try_from(diff).unwrap()); + offset -= to_skip; + diff -= Diff::try_from(to_skip).unwrap(); + } + // We should produce at most `limit` records. + // TODO(benesch): avoid dangerous `as` conversion. + #[allow(clippy::as_conversions)] + if let Some(limit) = &mut limit { + diff = std::cmp::min(diff, Diff::try_from(*limit).unwrap()); + *limit -= diff as usize; + } + // Output the indicated number of rows. + if diff > 0 { + // Emit retractions for the elements actually part of + // the set of TopK elements. + target.push((R::ok(row.clone()), -diff)); + } } } - } - }) + }, + ) .as_collection(|k, v| (k.clone(), v.clone())) } diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index e4988d19d87a..a4a83833d80f 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -414,6 +414,7 @@ impl<'w, A: Allocate + 'static> Worker<'w, A> { linear_join_impl: Default::default(), metrics: self.compute_metrics.clone(), tracing_handle: Arc::clone(&self.tracing_handle), + enable_arrangement_size_logging: Default::default(), }); } _ => (), diff --git a/src/sql/src/session/vars.rs b/src/sql/src/session/vars.rs index 32300d14569c..1fae462a3b90 100644 --- a/src/sql/src/session/vars.rs +++ b/src/sql/src/session/vars.rs @@ -1197,6 +1197,7 @@ feature_flags!( enable_try_parse_monotonic_iso8601_timestamp, "the try_parse_monotonic_iso8601_timestamp function" ), + (enable_arrangement_size_logging, "arrangement size logging") ); /// Represents the input to a variable. @@ -3706,6 +3707,7 @@ pub fn is_tracing_var(name: &str) -> bool { pub fn is_compute_config_var(name: &str) -> bool { name == MAX_RESULT_SIZE.name() || name == DATAFLOW_MAX_INFLIGHT_BYTES.name() + || name == ENABLE_ARRANGEMENT_SIZE_LOGGING.name() || name == ENABLE_MZ_JOIN_CORE.name() || is_persist_config_var(name) || is_tracing_var(name) diff --git a/test/testdrive/introspection-sources.td b/test/testdrive/introspection-sources.td index 2c51a4c09c8e..541dfa55f499 100644 --- a/test/testdrive/introspection-sources.td +++ b/test/testdrive/introspection-sources.td @@ -278,6 +278,9 @@ mv1_primary_idx mv1 > CREATE VIEW vv_arr AS SELECT sum(a) FROM t JOIN t2 ON t.a = t2.b; +$ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} +ALTER SYSTEM SET enable_arrangement_size_logging = true + > CREATE MATERIALIZED VIEW mv_arr AS SELECT * FROM vv_arr > CREATE DEFAULT INDEX ii_arr ON vv_arr From 57f0173f645e6c9f447249596103f7bb312bebaf Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Fri, 14 Jul 2023 13:40:11 +0200 Subject: [PATCH 06/10] Update documentation Signed-off-by: Moritz Hoffmann --- .../content/sql/system-catalog/mz_internal.md | 62 ++++++++++--------- .../autogenerated/mz_internal.slt | 34 ++++++++-- 2 files changed, 63 insertions(+), 33 deletions(-) diff --git a/doc/user/content/sql/system-catalog/mz_internal.md b/doc/user/content/sql/system-catalog/mz_internal.md index e8762570e96b..0b9df12dd620 100644 --- a/doc/user/content/sql/system-catalog/mz_internal.md +++ b/doc/user/content/sql/system-catalog/mz_internal.md @@ -375,16 +375,19 @@ The size, capacity, and allocations are an approximation, which may underestimat Specifically, reductions can use more memory than we show here. -| Field | Type | Meaning | -| -------------- |-------------| -------- | -| `operator_id` | [`uint8`] | The ID of the operator that created the arrangement. Corresponds to [`mz_dataflow_operators.id`](#mz_dataflow_operators). | -| `records` | [`numeric`] | The number of records in the arrangement. | -| `batches` | [`numeric`] | The number of batches in the arrangement. | -| `size` | [`bigint`] | The utilized size in bytes of the arrangement. | -| `capacity` | [`bigint`] | The capacity in bytes of the arrangement. Can be larger than the size. | -| `allocations` | [`bigint`] | The number of separate memory allocations backing the arrangement. | +| Field | Type | Meaning | +|---------------|-------------| -------- | +| `operator_id` | [`uint8`] | The ID of the operator that created the arrangement. Corresponds to [`mz_dataflow_operators.id`](#mz_dataflow_operators). | +| `records` | [`numeric`] | The number of records in the arrangement. | +| `batches` | [`numeric`] | The number of batches in the arrangement. | +| `size` | [`numeric`] | The utilized size in bytes of the arrangement. | +| `capacity` | [`numeric`] | The capacity in bytes of the arrangement. Can be larger than the size. | +| `allocations` | [`numeric`] | The number of separate memory allocations backing the arrangement. | + + + ### `mz_compute_delays_histogram` @@ -496,15 +499,15 @@ The `mz_dataflow_arrangement_sizes` view describes how many records and batches are contained in operators under each dataflow. -| Field | Type | Meaning | -|---------------|------------|------------------------------------------------------------------------------| -| `id` | [`bigint`] | The ID of the [dataflow]. Corresponds to [`mz_dataflows.id`](#mz_dataflows). | -| `name` | [`bigint`] | The name of the object (e.g., index) maintained by the dataflow. | -| `records` | [`bigint`] | The number of records in all arrangements in the dataflow. | -| `batches` | [`bigint`] | The number of batches in all arrangements in the dataflow. | -| `size` | [`bigint`] | The utilized size in bytes of the arrangements. | -| `capacity` | [`bigint`] | The capacity in bytes of the arrangements. Can be larger than the size. | -| `allocations` | [`bigint`] | The number of separate memory allocations backing the arrangements. | +| Field | Type | Meaning | +|---------------|-------------|------------------------------------------------------------------------------| +| `id` | [`uint8`] | The ID of the [dataflow]. Corresponds to [`mz_dataflows.id`](#mz_dataflows). | +| `name` | [`text`] | The name of the object (e.g., index) maintained by the dataflow. | +| `records` | [`numeric`] | The number of records in all arrangements in the dataflow. | +| `batches` | [`numeric`] | The number of batches in all arrangements in the dataflow. | +| `size` | [`numeric`] | The utilized size in bytes of the arrangements. | +| `capacity` | [`numeric`] | The capacity in bytes of the arrangements. Can be larger than the size. | +| `allocations` | [`numeric`] | The number of separate memory allocations backing the arrangements. | ### `mz_dataflow_channels` @@ -579,11 +582,14 @@ The `mz_dataflow_operator_parents` view describes how [dataflow] operators are n The `mz_dataflow_shutdown_durations_histogram` view describes a histogram of the time in nanoseconds required to fully shut down dropped [dataflows][dataflow]. - -| Field | Type | Meaning | -| -------------- | ------------ | -------- | -| `duration_ns` | [`bigint`] | The upper bound of the bucket in nanoseconds. | -| `count` | [`bigint`] | The (noncumulative) count of dataflows in this bucket. | + +| Field | Type | Meaning | +| -------------- |-------------| -------- | +| `duration_ns` | [`uint8`] | The upper bound of the bucket in nanoseconds. | +| `count` | [`numeric`] | The (noncumulative) count of dataflows in this bucket. | + + + ### `mz_message_counts` @@ -623,9 +629,9 @@ The `mz_records_per_dataflow` view describes the number of records in each [data | `id` | [`uint8`] | The ID of the dataflow. Corresponds to [`mz_dataflows.id`](#mz_dataflows). | | `name` | [`text`] | The internal name of the dataflow. | | `records` | [`numeric`] | The number of records in the dataflow. | -| `size` | [`bigint`] | The utilized size in bytes of the arrangements. | -| `capacity` | [`bigint`] | The capacity in bytes of the arrangements. Can be larger than the size. | -| `allocations` | [`bigint`] | The number of separate memory allocations backing the arrangements. | +| `size` | [`numeric`] | The utilized size in bytes of the arrangements. | +| `capacity` | [`numeric`] | The capacity in bytes of the arrangements. Can be larger than the size. | +| `allocations` | [`numeric`] | The number of separate memory allocations backing the arrangements. | @@ -640,9 +646,9 @@ The `mz_records_per_dataflow_operator` view describes the number of records in e | `name` | [`text`] | The internal name of the operator. | | `dataflow_id` | [`uint8`] | The ID of the dataflow. Corresponds to [`mz_dataflows.id`](#mz_dataflows). | | `records` | [`numeric`] | The number of records in the operator. | -| `size` | [`bigint`] | The utilized size in bytes of the arrangement. | -| `capacity` | [`bigint`] | The capacity in bytes of the arrangement. Can be larger than the size. | -| `allocations` | [`bigint`] | The number of separate memory allocations backing the arrangement. | +| `size` | [`numeric`] | The utilized size in bytes of the arrangement. | +| `capacity` | [`numeric`] | The capacity in bytes of the arrangement. Can be larger than the size. | +| `allocations` | [`numeric`] | The number of separate memory allocations backing the arrangement. | diff --git a/test/sqllogictest/autogenerated/mz_internal.slt b/test/sqllogictest/autogenerated/mz_internal.slt index d00d728919b9..a73a7bdf4229 100644 --- a/test/sqllogictest/autogenerated/mz_internal.slt +++ b/test/sqllogictest/autogenerated/mz_internal.slt @@ -219,6 +219,9 @@ SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object 1 operator_id uint8 2 records numeric 3 batches numeric +4 size numeric +5 capacity numeric +6 allocations numeric query ITT SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object = 'mz_compute_delays_histogram' ORDER BY position @@ -272,6 +275,17 @@ SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object 1 id uint8 2 address list +query ITT +SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object = 'mz_dataflow_arrangement_sizes' ORDER BY position +---- +1 id uint8 +2 name text +3 records numeric +4 batches numeric +5 size numeric +6 capacity numeric +7 allocations numeric + query ITT SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object = 'mz_dataflow_channels' ORDER BY position ---- @@ -311,12 +325,10 @@ SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object 2 parent_id uint8 query ITT -SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object = 'mz_dataflow_arrangement_sizes' ORDER BY position +SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object = 'mz_dataflow_shutdown_durations_histogram' ORDER BY position ---- -1 id uint8 -2 name text -3 records numeric -4 batches numeric +1 duration_ns uint8 +2 count numeric query ITT SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object = 'mz_message_counts' ORDER BY position @@ -337,6 +349,9 @@ SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object 1 id uint8 2 name text 3 records numeric +4 size numeric +5 capacity numeric +6 allocations numeric query ITT SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object = 'mz_records_per_dataflow_operator' ORDER BY position @@ -345,6 +360,9 @@ SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object 2 name text 3 dataflow_id uint8 4 records numeric +5 size numeric +6 capacity numeric +7 allocations numeric query ITT SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object = 'mz_scheduling_elapsed' ORDER BY position @@ -366,6 +384,9 @@ mz_active_peeks mz_active_peeks_per_worker mz_aggregates mz_arrangement_batches_raw +mz_arrangement_heap_allocations_raw +mz_arrangement_heap_capacity_raw +mz_arrangement_heap_size_raw mz_arrangement_records_raw mz_arrangement_sharing mz_arrangement_sharing_per_worker @@ -410,6 +431,9 @@ mz_dataflow_operator_reachability_per_worker mz_dataflow_operator_reachability_raw mz_dataflow_operators mz_dataflow_operators_per_worker +mz_dataflow_shutdown_durations_histogram +mz_dataflow_shutdown_durations_histogram_per_worker +mz_dataflow_shutdown_durations_histogram_raw mz_dataflows mz_dataflows_per_worker mz_kafka_sources From f4a80c24da853923c961e8795a86e17746d35a43 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Fri, 14 Jul 2023 14:10:48 +0200 Subject: [PATCH 07/10] Add safety argument to MzArranged::inner calls Signed-off-by: Moritz Hoffmann --- src/compute/src/extensions/reduce.rs | 1 + src/compute/src/render/join/delta_join.rs | 2 ++ src/compute/src/render/join/linear_join.rs | 1 + 3 files changed, 4 insertions(+) diff --git a/src/compute/src/extensions/reduce.rs b/src/compute/src/extensions/reduce.rs index 06c1875626fc..7c4708d4f806 100644 --- a/src/compute/src/extensions/reduce.rs +++ b/src/compute/src/extensions/reduce.rs @@ -68,6 +68,7 @@ where L: FnMut(&K, &[(&V, R)], &mut Vec<(T2::Val, T2::R)>) + 'static, Self: ArrangementSize, { + // Safety: `reduce_abelian` holds on to the trace. let inner = unsafe { self.inner(enable_arrangement_size_logging) }; // Allow access to `reduce_abelian` since we're within Mz's wrapper. #[allow(clippy::disallowed_methods)] diff --git a/src/compute/src/render/join/delta_join.rs b/src/compute/src/render/join/delta_join.rs index 24c4159fc0a0..b0a576f4fdd2 100644 --- a/src/compute/src/render/join/delta_join.rs +++ b/src/compute/src/render/join/delta_join.rs @@ -361,6 +361,7 @@ where if closure.could_error() { let (oks, errs2) = dogsdogsdogs::operators::half_join::half_join_internal_unsafe( &updates, + // Safety: `half_join` holds on to the trace. unsafe { trace.inner(enable_arrangement_size_logging) }, |time| time.step_back(), comparison, @@ -397,6 +398,7 @@ where } else { let oks = dogsdogsdogs::operators::half_join::half_join_internal_unsafe( &updates, + // Safety: `half_join` holds on to the trace. unsafe { trace.inner(enable_arrangement_size_logging) }, |time| time.step_back(), comparison, diff --git a/src/compute/src/render/join/linear_join.rs b/src/compute/src/render/join/linear_join.rs index 19ed61cdd46d..4bbd3f54a4f8 100644 --- a/src/compute/src/render/join/linear_join.rs +++ b/src/compute/src/render/join/linear_join.rs @@ -345,6 +345,7 @@ where let mut datums = DatumVec::new(); let mut row_builder = Row::default(); + // Safety: all `join_impl`s holds on to the trace. let prev_keyed = unsafe { prev_keyed.inner(enable_arrangement_size_logging) }; let next_input = unsafe { next_input.inner(enable_arrangement_size_logging) }; From 03e7e13bab188d3c98ee84b15fe422008c5a6692 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Fri, 14 Jul 2023 15:07:56 +0200 Subject: [PATCH 08/10] Fix testdrive Signed-off-by: Moritz Hoffmann --- test/testdrive/introspection-sources.td | 18 +++++++++--------- test/testdrive/mz-arrangement-sharing.td | 4 ++++ 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/test/testdrive/introspection-sources.td b/test/testdrive/introspection-sources.td index 541dfa55f499..e056df243697 100644 --- a/test/testdrive/introspection-sources.td +++ b/test/testdrive/introspection-sources.td @@ -295,8 +295,8 @@ true true true > CREATE DEFAULT INDEX ii_empty ON t3 -> SELECT records, batches, size, capacity, allocations FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_empty' -0 0 0 0 0 +> SELECT records, batches, size < 1000, capacity < 1000, allocations < 1000 FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_empty' +0 32 true true true # Tests that arrangement sizes are approximate @@ -304,17 +304,17 @@ true true true > CREATE INDEX ii_t4 ON t4(c) -> SELECT records, batches, size, capacity, allocations FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_t4' -0 0 0 0 0 +> SELECT records, batches, size < 1000, capacity < 1000, allocations < 1000 FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_t4' +0 32 true true true -> INSERT INTO t4 SELECT 1 +> INSERT INTO t4 SELECT generate_series(1, 10) -> SELECT records, batches, size > 96 AND size < 2*96, capacity > 96, allocations > 0 FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_t4' -1 1 true true true +> SELECT records, batches, size > 1000 AND size < 2000, capacity > 1000, allocations > 0 FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_t4' +10 32 true true true -> INSERT INTO t4 SELECT generate_series(1, 1000) +> INSERT INTO t4 SELECT generate_series(10, 1010) -> SELECT records >= 1000 AND records <= 1001, batches > 0, size > 96*1000 AND size < 2*96*1000, capacity > 96*1000, allocations > 0 FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_t4' +> SELECT records >= 1010 AND records <= 1011, batches > 0, size > 96*1010 AND size < 2*96*1010, capacity > 96*1010, allocations > 0 FROM mz_internal.mz_dataflow_arrangement_sizes WHERE name='ii_t4' true true true true true > DROP INDEX ii_t4 diff --git a/test/testdrive/mz-arrangement-sharing.td b/test/testdrive/mz-arrangement-sharing.td index d7a262551f67..7665f13adb04 100644 --- a/test/testdrive/mz-arrangement-sharing.td +++ b/test/testdrive/mz-arrangement-sharing.td @@ -804,6 +804,9 @@ ALTER SYSTEM SET log_filter = 'debug' > SELECT mdo.name FROM mz_internal.mz_arrangement_sharing mash JOIN mz_internal.mz_dataflow_operators mdo ON mash.operator_id = mdo.id ORDER BY mdo.name; "Arrange Timely(Reachability)" +"ArrangeByKey Compute(ArrangementHeapAllocations)" +"ArrangeByKey Compute(ArrangementHeapCapacity)" +"ArrangeByKey Compute(ArrangementHeapSize)" "ArrangeByKey Compute(DataflowCurrent)" "ArrangeByKey Compute(DataflowDependency)" "ArrangeByKey Compute(FrontierCurrent)" @@ -811,6 +814,7 @@ ALTER SYSTEM SET log_filter = 'debug' "ArrangeByKey Compute(ImportFrontierCurrent)" "ArrangeByKey Compute(PeekCurrent)" "ArrangeByKey Compute(PeekDuration)" +"ArrangeByKey Compute(ShutdownDuration)" "ArrangeByKey Differential(ArrangementBatches)" "ArrangeByKey Differential(ArrangementRecords)" "ArrangeByKey Differential(Sharing)" From 5dc6505fd1e31d91735697b074697a88244ca9dd Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Tue, 18 Jul 2023 11:11:19 +0200 Subject: [PATCH 09/10] Enable arrangement size logging in tests Signed-off-by: Moritz Hoffmann --- misc/python/materialize/mzcompose/services.py | 1 + 1 file changed, 1 insertion(+) diff --git a/misc/python/materialize/mzcompose/services.py b/misc/python/materialize/mzcompose/services.py index 154bad7d5688..f1b13c2bdc9e 100644 --- a/misc/python/materialize/mzcompose/services.py +++ b/misc/python/materialize/mzcompose/services.py @@ -49,6 +49,7 @@ "persist_stats_filter_enabled": "true", "persist_stats_collection_enabled": "true", "persist_stats_audit_percent": "100", + "enable_arrangement_size_logging": "true", "enable_ld_rbac_checks": "true", "enable_rbac_checks": "true", "enable_monotonic_oneshot_selects": "true", From 5db53dd94c118122843a26ed3aaa4988373b8da7 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Tue, 18 Jul 2023 14:40:06 +0200 Subject: [PATCH 10/10] Address review comments Signed-off-by: Moritz Hoffmann --- src/compute-client/src/protocol/command.rs | 4 +-- src/compute/src/extensions/arrange.rs | 41 +++++++++++++--------- src/compute/src/logging/compute.rs | 8 +++-- src/compute/src/logging/differential.rs | 8 +++-- src/compute/src/logging/initialize.rs | 6 ++-- src/compute/src/logging/reachability.rs | 10 +++--- src/compute/src/logging/timely.rs | 10 +++--- src/compute/src/render/join/delta_join.rs | 2 +- src/compute/src/render/join/linear_join.rs | 2 +- src/compute/src/render/mod.rs | 10 +++--- 10 files changed, 59 insertions(+), 42 deletions(-) diff --git a/src/compute-client/src/protocol/command.rs b/src/compute-client/src/protocol/command.rs index b7f42e53141d..f84b60e80a6d 100644 --- a/src/compute-client/src/protocol/command.rs +++ b/src/compute-client/src/protocol/command.rs @@ -360,14 +360,14 @@ pub struct ComputeParameters { pub max_result_size: Option, /// The maximum number of in-flight bytes emitted by persist_sources feeding dataflows. pub dataflow_max_inflight_bytes: Option, + /// Enable arrangement size logging + pub enable_arrangement_size_logging: Option, /// Whether rendering should use `mz_join_core` rather than DD's `JoinCore::join_core`. pub enable_mz_join_core: Option, /// Persist client configuration. pub persist: PersistParameters, /// Tracing configuration. pub tracing: TracingParameters, - /// Enable arrangement size logging - pub enable_arrangement_size_logging: Option, } impl ComputeParameters { diff --git a/src/compute/src/extensions/arrange.rs b/src/compute/src/extensions/arrange.rs index 7de11be08509..44fcad23e7e1 100644 --- a/src/compute/src/extensions/arrange.rs +++ b/src/compute/src/extensions/arrange.rs @@ -7,13 +7,14 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. +use std::cell::Cell; +use std::rc::Rc; + use differential_dataflow::difference::Semigroup; use differential_dataflow::lattice::Lattice; use differential_dataflow::operators::arrange::{Arrange, Arranged, TraceAgent}; use differential_dataflow::trace::{Batch, Trace, TraceReader}; use differential_dataflow::{Collection, Data, ExchangeData, Hashable}; -use std::cell::RefCell; -use std::rc::Rc; use timely::container::columnation::Columnation; use timely::dataflow::channels::pact::{ParallelizationContract, Pipeline}; use timely::dataflow::operators::Operator; @@ -200,12 +201,10 @@ where L: FnMut(&TraceAgent) -> (usize, usize, usize) + 'static, { let scope = arranged.stream.scope(); - let logger = if let Some(logger) = scope + let Some(logger) = scope .log_register() .get::("materialize/compute") - { - logger - } else { + else { return; }; let mut trace = arranged.trace.clone(); @@ -358,7 +357,7 @@ where arranged: Arranged, /// Shared boolean to indicate that someone is consuming the trace, and we've installed /// the arrangement size logging operator. - trace_consumed: Rc>, + trace_consumed: Rc>, } impl MzArranged @@ -374,7 +373,16 @@ where /// ## Safety /// /// This is unsafe because clients are required to use the trace, otherwise the arrangement size - /// logging infrastructure will cause a memory leak. + /// logging infrastructure will cause a memory leak. Using the trace means that the compaction + /// needs to be advanced, and the client must be interested in all batches until the dataflow is + /// shut down. + /// + /// The logging operator will hold on to a trace and inspect all batches, but allows compaction. + /// It'll cause Differential to maintain the trace until the trace is complete, which is why we + /// need the caller to be interested in all updates to the trace. + /// + /// We're using `unsafe` to indicate additional requirements to the caller that we cannot + /// express in Rust, but not to indicate memory safety. pub unsafe fn inner(&self, enable_arrangement_size_logging: bool) -> Arranged { self.consume_trace(enable_arrangement_size_logging); self.arranged.clone() @@ -383,22 +391,21 @@ where /// Obtain a handle to the trace. /// /// Similarly to `inner`, clients must use the trace, otherwise a memory leak can occur because - /// the arrangement size logging operator holds on the the trace. - #[must_use] - pub fn trace(&self, enable_arrangement_size_logging: bool) -> Tr { + /// the arrangement size logging operator holds on the the trace. See [`MzArranged::inner`] for + /// details. + pub unsafe fn trace(&self, enable_arrangement_size_logging: bool) -> Tr { self.consume_trace(enable_arrangement_size_logging); - self.arranged.trace.clone() } /// Indicate that something uses the trace, so we can install the arrangement size logging - /// infrastructure. This uses inner mutability to only ever install the operator once for each + /// infrastructure. This uses interior mutability to only ever install the operator once for each /// trace. fn consume_trace(&self, enable_arrangement_size_logging: bool) { - let mut trace_consumed = self.trace_consumed.borrow_mut(); - if enable_arrangement_size_logging && !*trace_consumed { + let trace_consumed = self.trace_consumed.get(); + if enable_arrangement_size_logging && !trace_consumed { self.log_arrangement_size(); - *trace_consumed = true; + self.trace_consumed.set(true); } } } @@ -490,7 +497,7 @@ where fn from(arranged: Arranged) -> Self { Self { arranged, - trace_consumed: Rc::new(RefCell::new(false)), + trace_consumed: Rc::new(Cell::new(false)), } } } diff --git a/src/compute/src/logging/compute.rs b/src/compute/src/logging/compute.rs index 0c13f82f50dc..1ac33ee43027 100644 --- a/src/compute/src/logging/compute.rs +++ b/src/compute/src/logging/compute.rs @@ -347,7 +347,7 @@ pub(super) fn construct( .collect::>(), variant.desc().arity(), ); - let trace = collection + let arranged = collection .map({ let mut row_buf = Row::default(); let mut datums = DatumVec::new(); @@ -360,8 +360,10 @@ pub(super) fn construct( (row_key, row_val) } }) - .mz_arrange::>(&format!("ArrangeByKey {:?}", variant)) - .trace(compute_state.enable_arrangement_size_logging); + .mz_arrange::>(&format!("ArrangeByKey {:?}", variant)); + // Safety: We're exporting the trace. + let trace = + unsafe { arranged.trace(compute_state.enable_arrangement_size_logging) }; traces.insert(variant.clone(), (trace, Rc::clone(&token))); } } diff --git a/src/compute/src/logging/differential.rs b/src/compute/src/logging/differential.rs index f08702ad1f95..1d91b82d9335 100644 --- a/src/compute/src/logging/differential.rs +++ b/src/compute/src/logging/differential.rs @@ -162,7 +162,7 @@ pub(super) fn construct( .collect::>(), variant.desc().arity(), ); - let trace = collection + let arranged = collection .map({ let mut row_buf = Row::default(); let mut datums = DatumVec::new(); @@ -175,8 +175,10 @@ pub(super) fn construct( (row_key, row_val) } }) - .mz_arrange::>(&format!("ArrangeByKey {:?}", variant)) - .trace(compute_state.enable_arrangement_size_logging); + .mz_arrange::>(&format!("ArrangeByKey {:?}", variant)); + // Safety: We're exporting the trace. + let trace = + unsafe { arranged.trace(compute_state.enable_arrangement_size_logging) }; traces.insert(variant.clone(), (trace, Rc::clone(&token))); } } diff --git a/src/compute/src/logging/initialize.rs b/src/compute/src/logging/initialize.rs index c8cc6b1a1669..dced08f03b96 100644 --- a/src/compute/src/logging/initialize.rs +++ b/src/compute/src/logging/initialize.rs @@ -127,9 +127,9 @@ impl LoggingContext<'_, A> { .dataflow_named("Dataflow: logging errors", |scope| { let collection: KeyCollection<_, DataflowError, Diff> = Collection::empty(scope).into(); - collection - .mz_arrange("Arrange logging err") - .trace(self.compute_state.enable_arrangement_size_logging) + let arranged = collection.mz_arrange("Arrange logging err"); + // Safety: We're exporting the trace. + unsafe { arranged.trace(self.compute_state.enable_arrangement_size_logging) } }); traces diff --git a/src/compute/src/logging/reachability.rs b/src/compute/src/logging/reachability.rs index fe83f965070b..91af1cf96dee 100644 --- a/src/compute/src/logging/reachability.rs +++ b/src/compute/src/logging/reachability.rs @@ -14,7 +14,6 @@ use std::collections::BTreeMap; use std::convert::TryInto; use std::rc::Rc; -use crate::compute_state::ComputeState; use differential_dataflow::AsCollection; use mz_compute_client::logging::LoggingConfig; use mz_expr::{permutation_for_arrangement, MirScalarExpr}; @@ -27,6 +26,7 @@ use timely::communication::Allocate; use timely::dataflow::channels::pact::Exchange; use timely::dataflow::operators::Filter; +use crate::compute_state::ComputeState; use crate::extensions::arrange::MzArrange; use crate::logging::{EventQueue, LogVariant, TimelyLog}; use crate::typedefs::{KeysValsHandle, RowSpine}; @@ -149,9 +149,11 @@ pub(super) fn construct( }, ); - let trace = updates - .mz_arrange::>(&format!("Arrange {:?}", variant)) - .trace(compute_state.enable_arrangement_size_logging); + let arranged = + updates.mz_arrange::>(&format!("Arrange {:?}", variant)); + // Safety: We're exporting the trace. + let trace = + unsafe { arranged.trace(compute_state.enable_arrangement_size_logging) }; result.insert(variant.clone(), (trace, Rc::clone(&token))); } } diff --git a/src/compute/src/logging/timely.rs b/src/compute/src/logging/timely.rs index 17cf1273ce22..3435ab25025c 100644 --- a/src/compute/src/logging/timely.rs +++ b/src/compute/src/logging/timely.rs @@ -15,7 +15,6 @@ use std::collections::BTreeMap; use std::rc::Rc; use std::time::Duration; -use crate::compute_state::ComputeState; use differential_dataflow::collection::AsCollection; use mz_compute_client::logging::LoggingConfig; use mz_expr::{permutation_for_arrangement, MirScalarExpr}; @@ -36,6 +35,7 @@ use timely::logging::{ }; use tracing::error; +use crate::compute_state::ComputeState; use crate::extensions::arrange::MzArrange; use crate::logging::compute::ComputeEvent; use crate::logging::{EventQueue, LogVariant, SharedLoggingState, TimelyLog}; @@ -276,7 +276,7 @@ pub(super) fn construct( .collect::>(), variant.desc().arity(), ); - let trace = collection + let arranged = collection .map({ let mut row_buf = Row::default(); let mut datums = DatumVec::new(); @@ -289,8 +289,10 @@ pub(super) fn construct( (row_key, row_val) } }) - .mz_arrange::>(&format!("ArrangeByKey {:?}", variant)) - .trace(compute_state.enable_arrangement_size_logging); + .mz_arrange::>(&format!("ArrangeByKey {:?}", variant)); + // Safety: We're exporting the trace. + let trace = + unsafe { arranged.trace(compute_state.enable_arrangement_size_logging) }; traces.insert(variant.clone(), (trace, Rc::clone(&token))); } } diff --git a/src/compute/src/render/join/delta_join.rs b/src/compute/src/render/join/delta_join.rs index b0a576f4fdd2..cd0d5601c884 100644 --- a/src/compute/src/render/join/delta_join.rs +++ b/src/compute/src/render/join/delta_join.rs @@ -15,7 +15,6 @@ use std::collections::{BTreeMap, BTreeSet}; -use crate::extensions::arrange::{ArrangementSize, MzArranged}; use differential_dataflow::trace::{BatchReader, Cursor, TraceReader}; use differential_dataflow::{AsCollection, Collection}; use mz_compute_client::plan::join::delta_join::{DeltaJoinPlan, DeltaPathPlan, DeltaStagePlan}; @@ -30,6 +29,7 @@ use timely::dataflow::Scope; use timely::progress::timestamp::Refines; use timely::progress::Antichain; +use crate::extensions::arrange::{ArrangementSize, MzArranged}; use crate::render::context::{ArrangementFlavor, CollectionBundle, Context, ShutdownToken}; use crate::render::RenderTimestamp; diff --git a/src/compute/src/render/join/linear_join.rs b/src/compute/src/render/join/linear_join.rs index 4bbd3f54a4f8..c84ed0ed50a1 100644 --- a/src/compute/src/render/join/linear_join.rs +++ b/src/compute/src/render/join/linear_join.rs @@ -345,7 +345,7 @@ where let mut datums = DatumVec::new(); let mut row_builder = Row::default(); - // Safety: all `join_impl`s holds on to the trace. + // Safety: all `join_impl`s hold on to the trace. let prev_keyed = unsafe { prev_keyed.inner(enable_arrangement_size_logging) }; let next_input = unsafe { next_input.inner(enable_arrangement_size_logging) }; diff --git a/src/compute/src/render/mod.rs b/src/compute/src/render/mod.rs index 4e05a5bef7b4..5ea7fd29859b 100644 --- a/src/compute/src/render/mod.rs +++ b/src/compute/src/render/mod.rs @@ -502,11 +502,12 @@ where Some(ArrangementFlavor::Local(oks, errs)) => { // Set up probes to notify on index frontier advancement. oks.stream().probe_notify_with(probes); + // Safety: We're exporting the traces. compute_state.traces.set( idx_id, TraceBundle::new( - oks.trace(self.enable_arrangement_size_logging), - errs.trace(self.enable_arrangement_size_logging), + unsafe { oks.trace(self.enable_arrangement_size_logging) }, + unsafe { errs.trace(self.enable_arrangement_size_logging) }, ) .with_drop(needed_tokens), ); @@ -578,11 +579,12 @@ where .as_collection(|k, v| (k.clone(), v.clone())) .leave() .mz_arrange("Arrange export iterative err"); + // Safety: We're exporting the traces. compute_state.traces.set( idx_id, TraceBundle::new( - oks.trace(self.enable_arrangement_size_logging), - errs.trace(self.enable_arrangement_size_logging), + unsafe { oks.trace(self.enable_arrangement_size_logging) }, + unsafe { errs.trace(self.enable_arrangement_size_logging) }, ) .with_drop(needed_tokens), );