MaterializeInc / materialize

The Cloud Operational Data Store: use SQL to transform, deliver, and act on fast-changing data.
https://materialize.com
Other
5.71k stars 466 forks source link

controller gets confused when replicas are dropped in a multi-replica cluster #28166

Closed def- closed 4 weeks ago

def- commented 1 month ago

What version of Materialize are you using?

708f88f6cb1459f0c3f85753386a7941c845a78f

What is the issue?

I found this while looking into https://github.com/MaterializeInc/materialize/issues/23582 by just running bin/mzcompose --find parallel-workload down && bin/mzcompose --find parallel-workload run default. I don't think it explains all the stuck queries, but at least a subset:

materialize=> select * from mz_internal.mz_source_statuses where id = 'u163';
  id  |     name     | type  |   last_status_change_at    | status | error |                             details
------+--------------+-------+----------------------------+--------+-------+-----------------------------------------------------------------
 u163 | kafka_table9 | kafka | 2024-07-11 08:05:28.811+00 | paused |       | {"hints":["There is currently no replica running this source"]}
(1 row)
materialize=> explain timestamp for SELECT * FROM "db-pw-1720684941-0"."s-2".kafka_table9;
                                 Timestamp
---------------------------------------------------------------------------
                 query timestamp: 1720692697370 (2024-07-11 10:11:37.370) +
           oracle read timestamp: 1720692697370 (2024-07-11 10:11:37.370) +
 largest not in advance of upper:             0 (1970-01-01 00:00:00.000) +
                           upper:[            0 (1970-01-01 00:00:00.000)]+
                           since:[            0 (1970-01-01 00:00:00.000)]+
         can respond immediately: false                                   +
                        timeline: Some(EpochMilliseconds)                 +
               session wall time: 1720692698030 (2024-07-11 10:11:38.030) +
                                                                          +
 source db-pw-1720684941-0.s-2.kafka_table9 (u163, storage):              +
                   read frontier:[            0 (1970-01-01 00:00:00.000)]+
                  write frontier:[            0 (1970-01-01 00:00:00.000)]+

(1 row)

But the cluster reports to be in a good state and has a replica:

materialize=> select * from mz_clusters where Name = 'cluster-3';
 id |   name    | owner_id |        privileges         | managed | size | replication_factor | disk | availability_zones | introspection_debugging | introspection_interval
----+-----------+----------+---------------------------+---------+------+--------------------+------+--------------------+-------------------------+------------------------
 u5 | cluster-3 | u1       | {s2=U/u1,u1=UC/u1,=UC/u1} | f       |      |                    |      |                    |                         |
(1 row)
materialize=> select * from mz_cluster_replicas where cluster_id = 'u5';
 id | name | cluster_id | size | availability_zone | owner_id | disk
----+------+------------+------+-------------------+----------+------
 u6 | r-1  | u5         | 2    |                   | u1       | t
(1 row)

In truth creating new sources on the cluster will have the same problem:

materialize=> create source counter in cluster "cluster-3" from load generator counter;
CREATE SOURCE
materialize=> select * from counter;
^CCancel request sent
ERROR:  canceling statement due to user request

All queries that ran: parallel-workload-queries.log There were a bunch of attempts to drop the cluster and replica, which should have failed since it had sources:

[worker_10] DROP CLUSTER "cluster-3"; [HTTP]
[worker_8] DROP CLUSTER REPLICA "cluster-3"."r-1"; [HTTP]
[worker_8] DROP CLUSTER REPLICA "cluster-3"."r-2"; [WS]

But since the replica is still shown as existing the drop somehow didn't complete fully. Logs: services.zip

parallel-workload-materialized-1     | environmentd: 2024-07-09T15:55:44.735998Z  INFO mz_orchestrator_process: terminating orphaned process for cluster-u5-replica-u6-gen-0 with PID 1941

I'll try to create a smaller reproducer.

def- commented 1 month ago

This is actually much easier, doesn't even require concurrency:

drop cluster c cascade;
create cluster c REPLICAS(r1 (size = '2'), r2 (size = '2'));
drop cluster replica c.r1;
create source counter in cluster c from load generator counter;
SELECT count(*) from counter;

Hangs already. Why does r2 not work anymore after dropping r1?

materialize=> select * from mz_clusters where name = 'c';
 id  | name | owner_id |     privileges     | managed | size | replication_factor | disk | availability_zones | introspection_debugging | introspection_interval
-----+------+----------+--------------------+---------+------+--------------------+------+--------------------+-------------------------+------------------------
 u10 | c    | s1       | {s1=UC/s1,s2=U/s1} | f       |      |                    |      |                    |                         |
(1 row)
materialize=> select * from mz_cluster_replicas where cluster_id = 'u10';
 id  | name | cluster_id | size | availability_zone | owner_id | disk
-----+------+------------+------+-------------------+----------+------
 u18 | r2   | u10        | 2    |                   | s1       | f
(1 row)
materialize=> select * from mz_internal.mz_source_statuses;
 id  |       name       |      type      |   last_status_change_at    | status  | error |                             details
-----+------------------+----------------+----------------------------+---------+-------+-----------------------------------------------------------------
 u14 | counter_progress | progress       |                            | running |       |
 u15 | counter          | load-generator | 2024-07-11 10:33:18.778+00 | paused  |       | {"hints":["There is currently no replica running this source"]}
(2 rows)

Logs:

environmentd: 2024-07-11T10:33:16.996805Z  INFO coord::handle_message{kind="command-execute"}:message_command:coord::handle_execute{session="68f4f7f9-c9f8-4e0b-bc6f-d821a7ca2ad5"}:coord::handle_execute_inner{stmt="CREATE CLUSTER c (REPLICAS = (r1 (SIZE = '2'), r2 (SIZE = '2')))"}:coord::catalog_transact:coord::catalog_transact_conn:coord::catalog_transact_inner:catalog::transact:catalog::transact_inner:transact_op: mz_adapter::catalog::transact: create cluster c
environmentd: 2024-07-11T10:33:17.361211Z  INFO mz_compute_client::controller::replica: starting replica task replica=User(17)
environmentd: 2024-07-11T10:33:17.361272Z  INFO coord::handle_message{kind="command-execute"}:message_command:coord::handle_execute{session="68f4f7f9-c9f8-4e0b-bc6f-d821a7ca2ad5"}:coord::handle_execute_inner{stmt="CREATE CLUSTER c (REPLICAS = (r1 (SIZE = '2'), r2 (SIZE = '2')))"}: mz_adapter::coord::introspection: installing introspection subscribe id=t107 replica_id=u17 type_=ComputeErrorCounts
environmentd: 2024-07-11T10:33:17.361343Z  INFO mz_orchestrator_process: launching cluster-u10-replica-u17-gen-0-0 via /usr/local/bin/clusterd --storage-controller-listen-addr=/tmp/1a380ce41d0b07d4f943a66a2d61bcbe9a4711b4 --compute-controller-listen-addr=/tmp/1c79659f469530488f8703a82923f658779f9a1f --internal-http-listen-addr=/tmp/e6e2c69c0194b8eb01739bd838681f86a68d0fda --opentelemetry-resource=cluster_id=u10 --opentelemetry-resource=replica_id=u17 --persist-pubsub-url=http://localhost:6879 --persist-txn-tables=eager --environment-id=docker-container-f6159cb4-1c6c-4f25-a5e7-4c7635633d43-0 --secrets-reader=local-file --secrets-reader-local-file-dir=/mzdata/secrets --startup-log-filter=info --log-format=text --log-prefix=cluster-u10-replica-u17-gen-0...
environmentd: 2024-07-11T10:33:17.361548Z  INFO mz_compute_client::controller::replica: starting replica task replica=User(18)
environmentd: 2024-07-11T10:33:17.361596Z  INFO coord::handle_message{kind="command-execute"}:message_command:coord::handle_execute{session="68f4f7f9-c9f8-4e0b-bc6f-d821a7ca2ad5"}:coord::handle_execute_inner{stmt="CREATE CLUSTER c (REPLICAS = (r1 (SIZE = '2'), r2 (SIZE = '2')))"}: mz_adapter::coord::introspection: installing introspection subscribe id=t109 replica_id=u18 type_=ComputeErrorCounts
environmentd: 2024-07-11T10:33:17.361656Z  INFO mz_orchestrator_process: launching cluster-u10-replica-u18-gen-0-0 via /usr/local/bin/clusterd --storage-controller-listen-addr=/tmp/a4e2575c53c41e710608bb7714e670742416b19a --compute-controller-listen-addr=/tmp/b9c9b50c3ab63d2849717cf590113c89eb821d73 --internal-http-listen-addr=/tmp/427e9e7ee1b41534647fc4c712ae17a5e5e0f6f4 --opentelemetry-resource=cluster_id=u10 --opentelemetry-resource=replica_id=u18 --persist-pubsub-url=http://localhost:6879 --persist-txn-tables=eager --environment-id=docker-container-f6159cb4-1c6c-4f25-a5e7-4c7635633d43-0 --secrets-reader=local-file --secrets-reader-local-file-dir=/mzdata/secrets --startup-log-filter=info --log-format=text --log-prefix=cluster-u10-replica-u18-gen-0...
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.390930Z  INFO clusterd: booting os.os_type=Ubuntu os.version=22.04 os.bitness=64-bit build.version="0.107.2" build.sha="12c99b30eed467f5112beab6a39d8a47f2175c7c" build.time="2024-07-07T18:37:01Z" cpus.logical=16 cpus.physical=8 cpu0.brand="AMD Ryzen 7 7700 8-Core Processor" cpu0.frequency=5282 memory.total=134686658560 memory.used=15079116800 memory.limit=<unknown> swap.total=0 swap.used=0 swap.limit=<unknown> tracing.max_level=info
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.391130Z  INFO clusterd: serving internal HTTP server on /tmp/e6e2c69c0194b8eb01739bd838681f86a68d0fda
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.391462Z  INFO clusterd: booting os.os_type=Ubuntu os.version=22.04 os.bitness=64-bit build.version="0.107.2" build.sha="12c99b30eed467f5112beab6a39d8a47f2175c7c" build.time="2024-07-07T18:37:01Z" cpus.logical=16 cpus.physical=8 cpu0.brand="AMD Ryzen 7 7700 8-Core Processor" cpu0.frequency=5282 memory.total=134686658560 memory.used=15079116800 memory.limit=<unknown> swap.total=0 swap.used=0 swap.limit=<unknown> tracing.max_level=info
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.391610Z  INFO clusterd: serving internal HTTP server on /tmp/427e9e7ee1b41534647fc4c712ae17a5e5e0f6f4
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.394290Z  INFO clusterd: listening for storage controller connections on /tmp/1a380ce41d0b07d4f943a66a2d61bcbe9a4711b4
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.394308Z  INFO mz_service::grpc: Starting to listen on /tmp/1a380ce41d0b07d4f943a66a2d61bcbe9a4711b4
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.394382Z  INFO clusterd: listening for compute controller connections on /tmp/1c79659f469530488f8703a82923f658779f9a1f
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.394388Z  INFO mz_service::grpc: Starting to listen on /tmp/1c79659f469530488f8703a82923f658779f9a1f
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.395904Z  INFO clusterd: listening for storage controller connections on /tmp/a4e2575c53c41e710608bb7714e670742416b19a
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.395916Z  INFO mz_service::grpc: Starting to listen on /tmp/a4e2575c53c41e710608bb7714e670742416b19a
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.395964Z  INFO clusterd: listening for compute controller connections on /tmp/b9c9b50c3ab63d2849717cf590113c89eb821d73
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.395982Z  INFO mz_service::grpc: Starting to listen on /tmp/b9c9b50c3ab63d2849717cf590113c89eb821d73
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.487520Z  INFO mz_service::grpc: GrpcServer: remote client connected
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.487519Z  INFO mz_service::grpc: GrpcServer: remote client connected
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.487519Z  INFO mz_service::grpc: GrpcServer: remote client connected
environmentd: 2024-07-11T10:33:17.487645Z  INFO mz_service::grpc: GrpcClient /tmp/a4e2575c53c41e710608bb7714e670742416b19a: connected
environmentd: 2024-07-11T10:33:17.487645Z  INFO mz_service::grpc: GrpcClient /tmp/1c79659f469530488f8703a82923f658779f9a1f: connected
environmentd: 2024-07-11T10:33:17.487645Z  INFO mz_service::grpc: GrpcClient /tmp/b9c9b50c3ab63d2849717cf590113c89eb821d73: connected
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.487827Z  INFO mz_cluster::server: Building timely container with config TimelyConfig { workers: 2, process: 0, addresses: ["/tmp/16d6aaf7659a684aed60f19644b2f900bc72a331"], arrangement_exert_proportionality: 16 }
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.487838Z  INFO mz_cluster::server: Building timely container with config TimelyConfig { workers: 2, process: 0, addresses: ["/tmp/941f58cabfa7ec8a2f02c474b6135eb660891a84"], arrangement_exert_proportionality: 16 }
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.487851Z  INFO mz_cluster::communication: initializing network for timely instance, with 1 processes for epoch number (2, 1) process=0 addresses=["/tmp/16d6aaf7659a684aed60f19644b2f900bc72a331"]
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.487858Z  INFO mz_cluster::communication: initializing network for timely instance, with 1 processes for epoch number (2, 1) process=0 addresses=["/tmp/941f58cabfa7ec8a2f02c474b6135eb660891a84"]
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.488222Z  INFO mz_cluster::communication: successfully initialized network process=0
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.488234Z  INFO mz_cluster::server: Building timely container with config TimelyConfig { workers: 2, process: 0, addresses: ["/tmp/d64d6a5bf362a654fb011ac1210a197f186cec69"], arrangement_exert_proportionality: 1337 }
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.488247Z  INFO mz_cluster::communication: initializing network for timely instance, with 1 processes for epoch number (2, 1) process=0 addresses=["/tmp/d64d6a5bf362a654fb011ac1210a197f186cec69"]
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.488293Z  INFO mz_cluster::communication: successfully initialized network process=0
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.488558Z  INFO mz_cluster::communication: successfully initialized network process=0
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.488944Z  INFO mz_compute::compute_state: disabling lgalloc
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.488946Z  INFO mz_compute::compute_state: disabling lgalloc
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.488950Z  INFO mz_compute::compute_state: using chunked stack: false
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.488950Z  INFO mz_compute::compute_state: using chunked stack: false
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.488982Z  INFO mz_compute::compute_state: disabling lgalloc
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.488983Z  INFO mz_compute::compute_state: disabling lgalloc
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.488988Z  INFO mz_compute::compute_state: using chunked stack: false
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.488987Z  INFO mz_compute::compute_state: using chunked stack: false
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.489168Z  INFO mz_storage::storage_state: Applying configuration update: StorageParameters { pg_source_connect_timeout: Some(30s), pg_source_tcp_keepalives_retries: Some(5), pg_source_tcp_keepalives_idle: Some(10s), pg_source_tcp_keepalives_interval: Some(10s), pg_source_tcp_user_timeout: Some(60s), pg_source_tcp_configure_server: false, pg_source_snapshot_statement_timeout: 0ns, pg_source_wal_sender_timeout: None, mysql_source_timeouts: TimeoutConfig { snapshot_max_execution_time: Some(0ns), snapshot_lock_wait_timeout: Some(3600s), tcp_keepalive: Some(60s) }, keep_n_source_status_history_entries: 5, keep_n_sink_status_history_entries: 5, keep_n_privatelink_status_history_entries: 5, upsert_rocksdb_tuning_config: RocksDBTuningParameters { compaction_style: Level, optimize_compaction_memtable_budget: 178956970, level_compaction_dynamic_level_bytes: true, universal_compaction_target_ratio: 200, parallelism: None, compression_type: Lz4, bottommost_compression_type: Lz4, batch_size: 20480, retry_max_duration: 1s, stats_log_interval_seconds: 600, stats_persist_interval_seconds: 600, point_lookup_block_cache_size_mb: None, shrink_buffers_by_ratio: 0, write_buffer_manager_memory_bytes: None, write_buffer_manager_memory_fraction: None, write_buffer_manager_allow_stall: false }, finalize_shards: true, tracing: TracingParameters { log_filter: Some(info), opentelemetry_filter: Some(info), log_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })], opentelemetry_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("h2"), level: LevelFilter::OFF }), SerializableDirective(Directive { in_span: None, fields: [], target: Some("hyper"), level: LevelFilter::OFF })], sentry_filters: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })] }, upsert_auto_spill_config: UpsertAutoSpillConfig { allow_spilling_to_disk: false, spill_to_disk_threshold_bytes: 89478484 }, storage_dataflow_max_inflight_bytes_config: StorageMaxInflightBytesConfig { max_inflight_bytes_default: Some(268435456), max_inflight_bytes_cluster_size_fraction: Some(0.0025), disk_only: true }, grpc_client: GrpcClientParameters { connect_timeout: Some(5s), http2_keep_alive_interval: Some(3s), http2_keep_alive_timeout: Some(5s) }, shrink_upsert_unused_buffers_by_ratio: 0, record_namespaced_errors: true, ssh_timeout_config: SshTimeoutConfig { check_interval: 30s, connect_timeout: 30s, keepalives_idle: 10s }, kafka_timeout_config: TimeoutConfig { keepalive: true, socket_timeout: 60s, transaction_timeout: 60s, socket_connection_setup_timeout: 30s, fetch_metadata_timeout: 10s, progress_record_fetch_timeout: 90s, default_metadata_fetch_interval: 60s }, statistics_interval: 60s, statistics_collection_interval: 10s, pg_snapshot_config: PgSourceSnapshotConfig { collect_strict_count: true, fallback_to_strict_count: true, wait_for_count: true }, user_storage_managed_collections_batch_duration: 1s, dyncfg_updates: ConfigUpdates { updates: {"allow_user_sessions": ProtoConfigVal { val: Some(Bool(true)) }, "compute_dataflow_max_inflight_bytes": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_dataflow_max_inflight_bytes_cc": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_hydration_concurrency": ProtoConfigVal { val: Some(Usize(4)) }, "compute_persist_sink_obey_read_only": ProtoConfigVal { val: Some(Bool(true)) }, "controller_past_generation_replica_cleanup_retry_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "copy_to_s3_arrow_builder_buffer_ratio": ProtoConfigVal { val: Some(Usize(150)) }, "copy_to_s3_multipart_part_size_bytes": ProtoConfigVal { val: Some(Usize(8388608)) }, "copy_to_s3_parquet_row_group_file_ratio": ProtoConfigVal { val: Some(Usize(20)) }, "crdb_connect_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 5, nanos: 0 })) }, "crdb_tcp_user_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "enable_0dt_deployment": ProtoConfigVal { val: Some(Bool(false)) }, "enable_columnation_lgalloc": ProtoConfigVal { val: Some(Bool(false)) }, "enable_compute_chunked_stack": ProtoConfigVal { val: Some(Bool(false)) }, "enable_introspection_subscribes": ProtoConfigVal { val: Some(Bool(true)) }, "enable_lgalloc_eager_reclamation": ProtoConfigVal { val: Some(Bool(true)) }, "enable_mz_join_core": ProtoConfigVal { val: Some(Bool(true)) }, "enable_statement_lifecycle_logging": ProtoConfigVal { val: Some(Bool(false)) }, "kafka_client_id_enrichment_rules": ProtoConfigVal { val: Some(Json("[]")) }, "kafka_poll_max_wait": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_background_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_slow_clear_bytes": ProtoConfigVal { val: Some(Usize(33554432)) }, "linear_join_yielding": ProtoConfigVal { val: Some(String("work:1000000,time:100")) }, "mysql_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "mysql_replication_heartbeat_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "persist_batch_columnar_format": ProtoConfigVal { val: Some(String("row")) }, "persist_batch_delete_enabled": ProtoConfigVal { val: Some(Bool(false)) }, "persist_batch_record_part_format": ProtoConfigVal { val: Some(Bool(false)) }, "persist_blob_cache_mem_limit_bytes": ProtoConfigVal { val: Some(Usize(1048576)) }, "persist_blob_target_size": ProtoConfigVal { val: Some(Usize(134217728)) }, "persist_claim_unclaimed_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_compaction_minimum_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 90, nanos: 0 })) }, "persist_consensus_connection_pool_ttl": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "persist_consensus_connection_pool_ttl_stagger": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 6, nanos: 0 })) }, "persist_enable_arrow_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_arrow_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_enable_s3_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_s3_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_fetch_semaphore_cost_adjustment": ProtoConfigVal { val: Some(F64(1.2)) }, "persist_fetch_semaphore_permit_adjustment": ProtoConfigVal { val: Some(F64(1.0)) }, "persist_inline_writes_single_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_inline_writes_total_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_next_listen_batch_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "persist_next_listen_batch_retryer_fixed_sleep": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 0 })) }, "persist_next_listen_batch_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 200000000 })) }, "persist_next_listen_batch_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_optimize_ignored_data_decode": ProtoConfigVal { val: Some(Bool(true)) }, "persist_optimize_ignored_data_fetch": ProtoConfigVal { val: Some(Bool(true)) }, "persist_part_decode_format": ProtoConfigVal { val: Some(String("row")) }, "persist_pubsub_client_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_pubsub_push_diff_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_reader_lease_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 900, nanos: 0 })) }, "persist_record_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_rollup_threshold": ProtoConfigVal { val: Some(Usize(128)) }, "persist_roundtrip_spine": ProtoConfigVal { val: Some(Bool(false)) }, "persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(0)) }, "persist_split_old_runs": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_audit_percent": ProtoConfigVal { val: Some(Usize(0)) }, "persist_stats_budget_bytes": ProtoConfigVal { val: Some(Usize(1024)) }, "persist_stats_collection_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_filter_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_untrimmable_columns_equals": ProtoConfigVal { val: Some(String("err,ts,receivedat,createdat,_fivetran_deleted")) }, "persist_stats_untrimmable_columns_prefix": ProtoConfigVal { val: Some(String("last_")) }, "persist_stats_untrimmable_columns_suffix": ProtoConfigVal { val: Some(String("timestamp,time,_at,_tstamp")) }, "persist_txns_data_shard_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 16, nanos: 0 })) }, "persist_txns_data_shard_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 24000000 })) }, "persist_txns_data_shard_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_use_critical_since_catalog": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_snapshot": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_source": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_txn": ProtoConfigVal { val: Some(Bool(true)) }, "persist_write_diffs_sum": ProtoConfigVal { val: Some(Bool(true)) }, "pg_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "plan_insights_notice fast_path_clusters_optimize_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 10000000 })) }, "postgres_fetch_slot_resume_lsn_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 500000000 })) }, "storage_cluster_shutdown_grace_period": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 600, nanos: 0 })) }, "storage_dataflow_delay_sources_past_rehydration": ProtoConfigVal { val: Some(Bool(true)) }, "storage_downgrade_since_during_finalization": ProtoConfigVal { val: Some(Bool(true)) }, "storage_enforce_external_addresses": ProtoConfigVal { val: Some(Bool(false)) }, "storage_persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(1024)) }, "storage_rocksdb_cleanup_tries": ProtoConfigVal { val: Some(Usize(5)) }, "storage_rocksdb_use_merge_operator": ProtoConfigVal { val: Some(Bool(false)) }, "storage_source_decode_fuel": ProtoConfigVal { val: Some(Usize(1000000)) }, "storage_upsert_max_snapshot_batch_buffering": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "storage_upsert_prevent_snapshot_buffering": ProtoConfigVal { val: Some(Bool(true)) }, "use_global_txn_cache_source": ProtoConfigVal { val: Some(Bool(true)) }} } }
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.489168Z  INFO mz_storage::storage_state: Applying configuration update: StorageParameters { pg_source_connect_timeout: Some(30s), pg_source_tcp_keepalives_retries: Some(5), pg_source_tcp_keepalives_idle: Some(10s), pg_source_tcp_keepalives_interval: Some(10s), pg_source_tcp_user_timeout: Some(60s), pg_source_tcp_configure_server: false, pg_source_snapshot_statement_timeout: 0ns, pg_source_wal_sender_timeout: None, mysql_source_timeouts: TimeoutConfig { snapshot_max_execution_time: Some(0ns), snapshot_lock_wait_timeout: Some(3600s), tcp_keepalive: Some(60s) }, keep_n_source_status_history_entries: 5, keep_n_sink_status_history_entries: 5, keep_n_privatelink_status_history_entries: 5, upsert_rocksdb_tuning_config: RocksDBTuningParameters { compaction_style: Level, optimize_compaction_memtable_budget: 178956970, level_compaction_dynamic_level_bytes: true, universal_compaction_target_ratio: 200, parallelism: None, compression_type: Lz4, bottommost_compression_type: Lz4, batch_size: 20480, retry_max_duration: 1s, stats_log_interval_seconds: 600, stats_persist_interval_seconds: 600, point_lookup_block_cache_size_mb: None, shrink_buffers_by_ratio: 0, write_buffer_manager_memory_bytes: None, write_buffer_manager_memory_fraction: None, write_buffer_manager_allow_stall: false }, finalize_shards: true, tracing: TracingParameters { log_filter: Some(info), opentelemetry_filter: Some(info), log_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })], opentelemetry_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("h2"), level: LevelFilter::OFF }), SerializableDirective(Directive { in_span: None, fields: [], target: Some("hyper"), level: LevelFilter::OFF })], sentry_filters: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })] }, upsert_auto_spill_config: UpsertAutoSpillConfig { allow_spilling_to_disk: false, spill_to_disk_threshold_bytes: 89478484 }, storage_dataflow_max_inflight_bytes_config: StorageMaxInflightBytesConfig { max_inflight_bytes_default: Some(268435456), max_inflight_bytes_cluster_size_fraction: Some(0.0025), disk_only: true }, grpc_client: GrpcClientParameters { connect_timeout: Some(5s), http2_keep_alive_interval: Some(3s), http2_keep_alive_timeout: Some(5s) }, shrink_upsert_unused_buffers_by_ratio: 0, record_namespaced_errors: true, ssh_timeout_config: SshTimeoutConfig { check_interval: 30s, connect_timeout: 30s, keepalives_idle: 10s }, kafka_timeout_config: TimeoutConfig { keepalive: true, socket_timeout: 60s, transaction_timeout: 60s, socket_connection_setup_timeout: 30s, fetch_metadata_timeout: 10s, progress_record_fetch_timeout: 90s, default_metadata_fetch_interval: 60s }, statistics_interval: 60s, statistics_collection_interval: 10s, pg_snapshot_config: PgSourceSnapshotConfig { collect_strict_count: true, fallback_to_strict_count: true, wait_for_count: true }, user_storage_managed_collections_batch_duration: 1s, dyncfg_updates: ConfigUpdates { updates: {"allow_user_sessions": ProtoConfigVal { val: Some(Bool(true)) }, "compute_dataflow_max_inflight_bytes": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_dataflow_max_inflight_bytes_cc": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_hydration_concurrency": ProtoConfigVal { val: Some(Usize(4)) }, "compute_persist_sink_obey_read_only": ProtoConfigVal { val: Some(Bool(true)) }, "controller_past_generation_replica_cleanup_retry_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "copy_to_s3_arrow_builder_buffer_ratio": ProtoConfigVal { val: Some(Usize(150)) }, "copy_to_s3_multipart_part_size_bytes": ProtoConfigVal { val: Some(Usize(8388608)) }, "copy_to_s3_parquet_row_group_file_ratio": ProtoConfigVal { val: Some(Usize(20)) }, "crdb_connect_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 5, nanos: 0 })) }, "crdb_tcp_user_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "enable_0dt_deployment": ProtoConfigVal { val: Some(Bool(false)) }, "enable_columnation_lgalloc": ProtoConfigVal { val: Some(Bool(false)) }, "enable_compute_chunked_stack": ProtoConfigVal { val: Some(Bool(false)) }, "enable_introspection_subscribes": ProtoConfigVal { val: Some(Bool(true)) }, "enable_lgalloc_eager_reclamation": ProtoConfigVal { val: Some(Bool(true)) }, "enable_mz_join_core": ProtoConfigVal { val: Some(Bool(true)) }, "enable_statement_lifecycle_logging": ProtoConfigVal { val: Some(Bool(false)) }, "kafka_client_id_enrichment_rules": ProtoConfigVal { val: Some(Json("[]")) }, "kafka_poll_max_wait": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_background_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_slow_clear_bytes": ProtoConfigVal { val: Some(Usize(33554432)) }, "linear_join_yielding": ProtoConfigVal { val: Some(String("work:1000000,time:100")) }, "mysql_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "mysql_replication_heartbeat_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "persist_batch_columnar_format": ProtoConfigVal { val: Some(String("row")) }, "persist_batch_delete_enabled": ProtoConfigVal { val: Some(Bool(false)) }, "persist_batch_record_part_format": ProtoConfigVal { val: Some(Bool(false)) }, "persist_blob_cache_mem_limit_bytes": ProtoConfigVal { val: Some(Usize(1048576)) }, "persist_blob_target_size": ProtoConfigVal { val: Some(Usize(134217728)) }, "persist_claim_unclaimed_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_compaction_minimum_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 90, nanos: 0 })) }, "persist_consensus_connection_pool_ttl": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "persist_consensus_connection_pool_ttl_stagger": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 6, nanos: 0 })) }, "persist_enable_arrow_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_arrow_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_enable_s3_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_s3_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_fetch_semaphore_cost_adjustment": ProtoConfigVal { val: Some(F64(1.2)) }, "persist_fetch_semaphore_permit_adjustment": ProtoConfigVal { val: Some(F64(1.0)) }, "persist_inline_writes_single_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_inline_writes_total_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_next_listen_batch_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "persist_next_listen_batch_retryer_fixed_sleep": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 0 })) }, "persist_next_listen_batch_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 200000000 })) }, "persist_next_listen_batch_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_optimize_ignored_data_decode": ProtoConfigVal { val: Some(Bool(true)) }, "persist_optimize_ignored_data_fetch": ProtoConfigVal { val: Some(Bool(true)) }, "persist_part_decode_format": ProtoConfigVal { val: Some(String("row")) }, "persist_pubsub_client_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_pubsub_push_diff_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_reader_lease_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 900, nanos: 0 })) }, "persist_record_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_rollup_threshold": ProtoConfigVal { val: Some(Usize(128)) }, "persist_roundtrip_spine": ProtoConfigVal { val: Some(Bool(false)) }, "persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(0)) }, "persist_split_old_runs": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_audit_percent": ProtoConfigVal { val: Some(Usize(0)) }, "persist_stats_budget_bytes": ProtoConfigVal { val: Some(Usize(1024)) }, "persist_stats_collection_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_filter_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_untrimmable_columns_equals": ProtoConfigVal { val: Some(String("err,ts,receivedat,createdat,_fivetran_deleted")) }, "persist_stats_untrimmable_columns_prefix": ProtoConfigVal { val: Some(String("last_")) }, "persist_stats_untrimmable_columns_suffix": ProtoConfigVal { val: Some(String("timestamp,time,_at,_tstamp")) }, "persist_txns_data_shard_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 16, nanos: 0 })) }, "persist_txns_data_shard_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 24000000 })) }, "persist_txns_data_shard_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_use_critical_since_catalog": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_snapshot": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_source": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_txn": ProtoConfigVal { val: Some(Bool(true)) }, "persist_write_diffs_sum": ProtoConfigVal { val: Some(Bool(true)) }, "pg_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "plan_insights_notice fast_path_clusters_optimize_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 10000000 })) }, "postgres_fetch_slot_resume_lsn_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 500000000 })) }, "storage_cluster_shutdown_grace_period": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 600, nanos: 0 })) }, "storage_dataflow_delay_sources_past_rehydration": ProtoConfigVal { val: Some(Bool(true)) }, "storage_downgrade_since_during_finalization": ProtoConfigVal { val: Some(Bool(true)) }, "storage_enforce_external_addresses": ProtoConfigVal { val: Some(Bool(false)) }, "storage_persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(1024)) }, "storage_rocksdb_cleanup_tries": ProtoConfigVal { val: Some(Usize(5)) }, "storage_rocksdb_use_merge_operator": ProtoConfigVal { val: Some(Bool(false)) }, "storage_source_decode_fuel": ProtoConfigVal { val: Some(Usize(1000000)) }, "storage_upsert_max_snapshot_batch_buffering": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "storage_upsert_prevent_snapshot_buffering": ProtoConfigVal { val: Some(Bool(true)) }, "use_global_txn_cache_source": ProtoConfigVal { val: Some(Bool(true)) }} } }
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.489273Z  INFO mz_persist_client::rpc: Connecting to Persist PubSub: http://localhost:6879
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.489719Z  INFO mz_persist_client::rpc: Connected to Persist PubSub: http://localhost:6879
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.489799Z  INFO mz_compute::compute_state: Applying configuration update: ComputeParameters { max_result_size: Some(1073741824), tracing: TracingParameters { log_filter: Some(info), opentelemetry_filter: Some(info), log_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })], opentelemetry_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("h2"), level: LevelFilter::OFF }), SerializableDirective(Directive { in_span: None, fields: [], target: Some("hyper"), level: LevelFilter::OFF })], sentry_filters: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })] }, grpc_client: GrpcClientParameters { connect_timeout: Some(5s), http2_keep_alive_interval: Some(3s), http2_keep_alive_timeout: Some(5s) }, dyncfg_updates: ConfigUpdates { updates: {"allow_user_sessions": ProtoConfigVal { val: Some(Bool(true)) }, "compute_dataflow_max_inflight_bytes": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_dataflow_max_inflight_bytes_cc": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_hydration_concurrency": ProtoConfigVal { val: Some(Usize(4)) }, "compute_persist_sink_obey_read_only": ProtoConfigVal { val: Some(Bool(true)) }, "controller_past_generation_replica_cleanup_retry_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "copy_to_s3_arrow_builder_buffer_ratio": ProtoConfigVal { val: Some(Usize(150)) }, "copy_to_s3_multipart_part_size_bytes": ProtoConfigVal { val: Some(Usize(8388608)) }, "copy_to_s3_parquet_row_group_file_ratio": ProtoConfigVal { val: Some(Usize(20)) }, "crdb_connect_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 5, nanos: 0 })) }, "crdb_tcp_user_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "enable_0dt_deployment": ProtoConfigVal { val: Some(Bool(false)) }, "enable_columnation_lgalloc": ProtoConfigVal { val: Some(Bool(false)) }, "enable_compute_chunked_stack": ProtoConfigVal { val: Some(Bool(false)) }, "enable_introspection_subscribes": ProtoConfigVal { val: Some(Bool(true)) }, "enable_lgalloc_eager_reclamation": ProtoConfigVal { val: Some(Bool(true)) }, "enable_mz_join_core": ProtoConfigVal { val: Some(Bool(true)) }, "enable_statement_lifecycle_logging": ProtoConfigVal { val: Some(Bool(false)) }, "kafka_client_id_enrichment_rules": ProtoConfigVal { val: Some(Json("[]")) }, "kafka_poll_max_wait": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_background_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_slow_clear_bytes": ProtoConfigVal { val: Some(Usize(33554432)) }, "linear_join_yielding": ProtoConfigVal { val: Some(String("work:1000000,time:100")) }, "mysql_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "mysql_replication_heartbeat_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "persist_batch_columnar_format": ProtoConfigVal { val: Some(String("row")) }, "persist_batch_delete_enabled": ProtoConfigVal { val: Some(Bool(false)) }, "persist_batch_record_part_format": ProtoConfigVal { val: Some(Bool(false)) }, "persist_blob_cache_mem_limit_bytes": ProtoConfigVal { val: Some(Usize(1048576)) }, "persist_blob_target_size": ProtoConfigVal { val: Some(Usize(134217728)) }, "persist_claim_unclaimed_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_compaction_minimum_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 90, nanos: 0 })) }, "persist_consensus_connection_pool_ttl": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "persist_consensus_connection_pool_ttl_stagger": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 6, nanos: 0 })) }, "persist_enable_arrow_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_arrow_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_enable_s3_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_s3_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_fetch_semaphore_cost_adjustment": ProtoConfigVal { val: Some(F64(1.2)) }, "persist_fetch_semaphore_permit_adjustment": ProtoConfigVal { val: Some(F64(1.0)) }, "persist_inline_writes_single_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_inline_writes_total_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_next_listen_batch_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "persist_next_listen_batch_retryer_fixed_sleep": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 0 })) }, "persist_next_listen_batch_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 200000000 })) }, "persist_next_listen_batch_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_optimize_ignored_data_decode": ProtoConfigVal { val: Some(Bool(true)) }, "persist_optimize_ignored_data_fetch": ProtoConfigVal { val: Some(Bool(true)) }, "persist_part_decode_format": ProtoConfigVal { val: Some(String("row")) }, "persist_pubsub_client_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_pubsub_push_diff_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_reader_lease_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 900, nanos: 0 })) }, "persist_record_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_rollup_threshold": ProtoConfigVal { val: Some(Usize(128)) }, "persist_roundtrip_spine": ProtoConfigVal { val: Some(Bool(false)) }, "persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(0)) }, "persist_split_old_runs": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_audit_percent": ProtoConfigVal { val: Some(Usize(0)) }, "persist_stats_budget_bytes": ProtoConfigVal { val: Some(Usize(1024)) }, "persist_stats_collection_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_filter_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_untrimmable_columns_equals": ProtoConfigVal { val: Some(String("err,ts,receivedat,createdat,_fivetran_deleted")) }, "persist_stats_untrimmable_columns_prefix": ProtoConfigVal { val: Some(String("last_")) }, "persist_stats_untrimmable_columns_suffix": ProtoConfigVal { val: Some(String("timestamp,time,_at,_tstamp")) }, "persist_txns_data_shard_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 16, nanos: 0 })) }, "persist_txns_data_shard_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 24000000 })) }, "persist_txns_data_shard_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_use_critical_since_catalog": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_snapshot": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_source": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_txn": ProtoConfigVal { val: Some(Bool(true)) }, "persist_write_diffs_sum": ProtoConfigVal { val: Some(Bool(true)) }, "pg_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "plan_insights_notice fast_path_clusters_optimize_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 10000000 })) }, "postgres_fetch_slot_resume_lsn_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 500000000 })) }, "storage_cluster_shutdown_grace_period": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 600, nanos: 0 })) }, "storage_dataflow_delay_sources_past_rehydration": ProtoConfigVal { val: Some(Bool(true)) }, "storage_downgrade_since_during_finalization": ProtoConfigVal { val: Some(Bool(true)) }, "storage_enforce_external_addresses": ProtoConfigVal { val: Some(Bool(false)) }, "storage_persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(1024)) }, "storage_rocksdb_cleanup_tries": ProtoConfigVal { val: Some(Usize(5)) }, "storage_rocksdb_use_merge_operator": ProtoConfigVal { val: Some(Bool(false)) }, "storage_source_decode_fuel": ProtoConfigVal { val: Some(Usize(1000000)) }, "storage_upsert_max_snapshot_batch_buffering": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "storage_upsert_prevent_snapshot_buffering": ProtoConfigVal { val: Some(Bool(true)) }, "use_global_txn_cache_source": ProtoConfigVal { val: Some(Bool(true)) }} } }
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.489795Z  INFO mz_compute::compute_state: Applying configuration update: ComputeParameters { max_result_size: Some(1073741824), tracing: TracingParameters { log_filter: Some(info), opentelemetry_filter: Some(info), log_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })], opentelemetry_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("h2"), level: LevelFilter::OFF }), SerializableDirective(Directive { in_span: None, fields: [], target: Some("hyper"), level: LevelFilter::OFF })], sentry_filters: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })] }, grpc_client: GrpcClientParameters { connect_timeout: Some(5s), http2_keep_alive_interval: Some(3s), http2_keep_alive_timeout: Some(5s) }, dyncfg_updates: ConfigUpdates { updates: {"allow_user_sessions": ProtoConfigVal { val: Some(Bool(true)) }, "compute_dataflow_max_inflight_bytes": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_dataflow_max_inflight_bytes_cc": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_hydration_concurrency": ProtoConfigVal { val: Some(Usize(4)) }, "compute_persist_sink_obey_read_only": ProtoConfigVal { val: Some(Bool(true)) }, "controller_past_generation_replica_cleanup_retry_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "copy_to_s3_arrow_builder_buffer_ratio": ProtoConfigVal { val: Some(Usize(150)) }, "copy_to_s3_multipart_part_size_bytes": ProtoConfigVal { val: Some(Usize(8388608)) }, "copy_to_s3_parquet_row_group_file_ratio": ProtoConfigVal { val: Some(Usize(20)) }, "crdb_connect_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 5, nanos: 0 })) }, "crdb_tcp_user_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "enable_0dt_deployment": ProtoConfigVal { val: Some(Bool(false)) }, "enable_columnation_lgalloc": ProtoConfigVal { val: Some(Bool(false)) }, "enable_compute_chunked_stack": ProtoConfigVal { val: Some(Bool(false)) }, "enable_introspection_subscribes": ProtoConfigVal { val: Some(Bool(true)) }, "enable_lgalloc_eager_reclamation": ProtoConfigVal { val: Some(Bool(true)) }, "enable_mz_join_core": ProtoConfigVal { val: Some(Bool(true)) }, "enable_statement_lifecycle_logging": ProtoConfigVal { val: Some(Bool(false)) }, "kafka_client_id_enrichment_rules": ProtoConfigVal { val: Some(Json("[]")) }, "kafka_poll_max_wait": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_background_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_slow_clear_bytes": ProtoConfigVal { val: Some(Usize(33554432)) }, "linear_join_yielding": ProtoConfigVal { val: Some(String("work:1000000,time:100")) }, "mysql_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "mysql_replication_heartbeat_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "persist_batch_columnar_format": ProtoConfigVal { val: Some(String("row")) }, "persist_batch_delete_enabled": ProtoConfigVal { val: Some(Bool(false)) }, "persist_batch_record_part_format": ProtoConfigVal { val: Some(Bool(false)) }, "persist_blob_cache_mem_limit_bytes": ProtoConfigVal { val: Some(Usize(1048576)) }, "persist_blob_target_size": ProtoConfigVal { val: Some(Usize(134217728)) }, "persist_claim_unclaimed_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_compaction_minimum_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 90, nanos: 0 })) }, "persist_consensus_connection_pool_ttl": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "persist_consensus_connection_pool_ttl_stagger": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 6, nanos: 0 })) }, "persist_enable_arrow_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_arrow_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_enable_s3_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_s3_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_fetch_semaphore_cost_adjustment": ProtoConfigVal { val: Some(F64(1.2)) }, "persist_fetch_semaphore_permit_adjustment": ProtoConfigVal { val: Some(F64(1.0)) }, "persist_inline_writes_single_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_inline_writes_total_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_next_listen_batch_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "persist_next_listen_batch_retryer_fixed_sleep": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 0 })) }, "persist_next_listen_batch_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 200000000 })) }, "persist_next_listen_batch_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_optimize_ignored_data_decode": ProtoConfigVal { val: Some(Bool(true)) }, "persist_optimize_ignored_data_fetch": ProtoConfigVal { val: Some(Bool(true)) }, "persist_part_decode_format": ProtoConfigVal { val: Some(String("row")) }, "persist_pubsub_client_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_pubsub_push_diff_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_reader_lease_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 900, nanos: 0 })) }, "persist_record_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_rollup_threshold": ProtoConfigVal { val: Some(Usize(128)) }, "persist_roundtrip_spine": ProtoConfigVal { val: Some(Bool(false)) }, "persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(0)) }, "persist_split_old_runs": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_audit_percent": ProtoConfigVal { val: Some(Usize(0)) }, "persist_stats_budget_bytes": ProtoConfigVal { val: Some(Usize(1024)) }, "persist_stats_collection_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_filter_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_untrimmable_columns_equals": ProtoConfigVal { val: Some(String("err,ts,receivedat,createdat,_fivetran_deleted")) }, "persist_stats_untrimmable_columns_prefix": ProtoConfigVal { val: Some(String("last_")) }, "persist_stats_untrimmable_columns_suffix": ProtoConfigVal { val: Some(String("timestamp,time,_at,_tstamp")) }, "persist_txns_data_shard_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 16, nanos: 0 })) }, "persist_txns_data_shard_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 24000000 })) }, "persist_txns_data_shard_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_use_critical_since_catalog": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_snapshot": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_source": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_txn": ProtoConfigVal { val: Some(Bool(true)) }, "persist_write_diffs_sum": ProtoConfigVal { val: Some(Bool(true)) }, "pg_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "plan_insights_notice fast_path_clusters_optimize_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 10000000 })) }, "postgres_fetch_slot_resume_lsn_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 500000000 })) }, "storage_cluster_shutdown_grace_period": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 600, nanos: 0 })) }, "storage_dataflow_delay_sources_past_rehydration": ProtoConfigVal { val: Some(Bool(true)) }, "storage_downgrade_since_during_finalization": ProtoConfigVal { val: Some(Bool(true)) }, "storage_enforce_external_addresses": ProtoConfigVal { val: Some(Bool(false)) }, "storage_persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(1024)) }, "storage_rocksdb_cleanup_tries": ProtoConfigVal { val: Some(Usize(5)) }, "storage_rocksdb_use_merge_operator": ProtoConfigVal { val: Some(Bool(false)) }, "storage_source_decode_fuel": ProtoConfigVal { val: Some(Usize(1000000)) }, "storage_upsert_max_snapshot_batch_buffering": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "storage_upsert_prevent_snapshot_buffering": ProtoConfigVal { val: Some(Bool(true)) }, "use_global_txn_cache_source": ProtoConfigVal { val: Some(Bool(true)) }} } }
environmentd: 2024-07-11T10:33:17.489846Z  INFO persist::rpc::server: mz_persist_client::rpc: Received Persist PubSub connection from: "f50feabed3a9"
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.489859Z  INFO mz_compute::compute_state: disabling lgalloc
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.489861Z  INFO mz_compute::compute_state: using chunked stack: false
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.489816Z  INFO mz_compute::compute_state: Applying configuration update: ComputeParameters { max_result_size: Some(1073741824), tracing: TracingParameters { log_filter: Some(info), opentelemetry_filter: Some(info), log_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })], opentelemetry_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("h2"), level: LevelFilter::OFF }), SerializableDirective(Directive { in_span: None, fields: [], target: Some("hyper"), level: LevelFilter::OFF })], sentry_filters: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })] }, grpc_client: GrpcClientParameters { connect_timeout: Some(5s), http2_keep_alive_interval: Some(3s), http2_keep_alive_timeout: Some(5s) }, dyncfg_updates: ConfigUpdates { updates: {"allow_user_sessions": ProtoConfigVal { val: Some(Bool(true)) }, "compute_dataflow_max_inflight_bytes": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_dataflow_max_inflight_bytes_cc": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_hydration_concurrency": ProtoConfigVal { val: Some(Usize(4)) }, "compute_persist_sink_obey_read_only": ProtoConfigVal { val: Some(Bool(true)) }, "controller_past_generation_replica_cleanup_retry_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "copy_to_s3_arrow_builder_buffer_ratio": ProtoConfigVal { val: Some(Usize(150)) }, "copy_to_s3_multipart_part_size_bytes": ProtoConfigVal { val: Some(Usize(8388608)) }, "copy_to_s3_parquet_row_group_file_ratio": ProtoConfigVal { val: Some(Usize(20)) }, "crdb_connect_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 5, nanos: 0 })) }, "crdb_tcp_user_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "enable_0dt_deployment": ProtoConfigVal { val: Some(Bool(false)) }, "enable_columnation_lgalloc": ProtoConfigVal { val: Some(Bool(false)) }, "enable_compute_chunked_stack": ProtoConfigVal { val: Some(Bool(false)) }, "enable_introspection_subscribes": ProtoConfigVal { val: Some(Bool(true)) }, "enable_lgalloc_eager_reclamation": ProtoConfigVal { val: Some(Bool(true)) }, "enable_mz_join_core": ProtoConfigVal { val: Some(Bool(true)) }, "enable_statement_lifecycle_logging": ProtoConfigVal { val: Some(Bool(false)) }, "kafka_client_id_enrichment_rules": ProtoConfigVal { val: Some(Json("[]")) }, "kafka_poll_max_wait": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_background_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_slow_clear_bytes": ProtoConfigVal { val: Some(Usize(33554432)) }, "linear_join_yielding": ProtoConfigVal { val: Some(String("work:1000000,time:100")) }, "mysql_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "mysql_replication_heartbeat_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "persist_batch_columnar_format": ProtoConfigVal { val: Some(String("row")) }, "persist_batch_delete_enabled": ProtoConfigVal { val: Some(Bool(false)) }, "persist_batch_record_part_format": ProtoConfigVal { val: Some(Bool(false)) }, "persist_blob_cache_mem_limit_bytes": ProtoConfigVal { val: Some(Usize(1048576)) }, "persist_blob_target_size": ProtoConfigVal { val: Some(Usize(134217728)) }, "persist_claim_unclaimed_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_compaction_minimum_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 90, nanos: 0 })) }, "persist_consensus_connection_pool_ttl": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "persist_consensus_connection_pool_ttl_stagger": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 6, nanos: 0 })) }, "persist_enable_arrow_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_arrow_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_enable_s3_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_s3_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_fetch_semaphore_cost_adjustment": ProtoConfigVal { val: Some(F64(1.2)) }, "persist_fetch_semaphore_permit_adjustment": ProtoConfigVal { val: Some(F64(1.0)) }, "persist_inline_writes_single_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_inline_writes_total_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_next_listen_batch_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "persist_next_listen_batch_retryer_fixed_sleep": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 0 })) }, "persist_next_listen_batch_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 200000000 })) }, "persist_next_listen_batch_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_optimize_ignored_data_decode": ProtoConfigVal { val: Some(Bool(true)) }, "persist_optimize_ignored_data_fetch": ProtoConfigVal { val: Some(Bool(true)) }, "persist_part_decode_format": ProtoConfigVal { val: Some(String("row")) }, "persist_pubsub_client_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_pubsub_push_diff_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_reader_lease_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 900, nanos: 0 })) }, "persist_record_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_rollup_threshold": ProtoConfigVal { val: Some(Usize(128)) }, "persist_roundtrip_spine": ProtoConfigVal { val: Some(Bool(false)) }, "persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(0)) }, "persist_split_old_runs": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_audit_percent": ProtoConfigVal { val: Some(Usize(0)) }, "persist_stats_budget_bytes": ProtoConfigVal { val: Some(Usize(1024)) }, "persist_stats_collection_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_filter_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_untrimmable_columns_equals": ProtoConfigVal { val: Some(String("err,ts,receivedat,createdat,_fivetran_deleted")) }, "persist_stats_untrimmable_columns_prefix": ProtoConfigVal { val: Some(String("last_")) }, "persist_stats_untrimmable_columns_suffix": ProtoConfigVal { val: Some(String("timestamp,time,_at,_tstamp")) }, "persist_txns_data_shard_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 16, nanos: 0 })) }, "persist_txns_data_shard_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 24000000 })) }, "persist_txns_data_shard_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_use_critical_since_catalog": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_snapshot": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_source": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_txn": ProtoConfigVal { val: Some(Bool(true)) }, "persist_write_diffs_sum": ProtoConfigVal { val: Some(Bool(true)) }, "pg_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "plan_insights_notice fast_path_clusters_optimize_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 10000000 })) }, "postgres_fetch_slot_resume_lsn_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 500000000 })) }, "storage_cluster_shutdown_grace_period": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 600, nanos: 0 })) }, "storage_dataflow_delay_sources_past_rehydration": ProtoConfigVal { val: Some(Bool(true)) }, "storage_downgrade_since_during_finalization": ProtoConfigVal { val: Some(Bool(true)) }, "storage_enforce_external_addresses": ProtoConfigVal { val: Some(Bool(false)) }, "storage_persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(1024)) }, "storage_rocksdb_cleanup_tries": ProtoConfigVal { val: Some(Usize(5)) }, "storage_rocksdb_use_merge_operator": ProtoConfigVal { val: Some(Bool(false)) }, "storage_source_decode_fuel": ProtoConfigVal { val: Some(Usize(1000000)) }, "storage_upsert_max_snapshot_batch_buffering": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "storage_upsert_prevent_snapshot_buffering": ProtoConfigVal { val: Some(Bool(true)) }, "use_global_txn_cache_source": ProtoConfigVal { val: Some(Bool(true)) }} } }
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.489866Z  INFO mz_compute::compute_state: disabling lgalloc
cluster-u10-replica-u18-gen-0: 2024-07-11T10:33:17.489868Z  INFO mz_compute::compute_state: using chunked stack: false
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.489823Z  INFO mz_compute::compute_state: Applying configuration update: ComputeParameters { max_result_size: Some(1073741824), tracing: TracingParameters { log_filter: Some(info), opentelemetry_filter: Some(info), log_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })], opentelemetry_filter_defaults: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("h2"), level: LevelFilter::OFF }), SerializableDirective(Directive { in_span: None, fields: [], target: Some("hyper"), level: LevelFilter::OFF })], sentry_filters: [SerializableDirective(Directive { in_span: None, fields: [], target: Some("kube_client::client::builder"), level: LevelFilter::OFF })] }, grpc_client: GrpcClientParameters { connect_timeout: Some(5s), http2_keep_alive_interval: Some(3s), http2_keep_alive_timeout: Some(5s) }, dyncfg_updates: ConfigUpdates { updates: {"allow_user_sessions": ProtoConfigVal { val: Some(Bool(true)) }, "compute_dataflow_max_inflight_bytes": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_dataflow_max_inflight_bytes_cc": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "compute_hydration_concurrency": ProtoConfigVal { val: Some(Usize(4)) }, "compute_persist_sink_obey_read_only": ProtoConfigVal { val: Some(Bool(true)) }, "controller_past_generation_replica_cleanup_retry_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "copy_to_s3_arrow_builder_buffer_ratio": ProtoConfigVal { val: Some(Usize(150)) }, "copy_to_s3_multipart_part_size_bytes": ProtoConfigVal { val: Some(Usize(8388608)) }, "copy_to_s3_parquet_row_group_file_ratio": ProtoConfigVal { val: Some(Usize(20)) }, "crdb_connect_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 5, nanos: 0 })) }, "crdb_tcp_user_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "enable_0dt_deployment": ProtoConfigVal { val: Some(Bool(false)) }, "enable_columnation_lgalloc": ProtoConfigVal { val: Some(Bool(false)) }, "enable_compute_chunked_stack": ProtoConfigVal { val: Some(Bool(false)) }, "enable_introspection_subscribes": ProtoConfigVal { val: Some(Bool(true)) }, "enable_lgalloc_eager_reclamation": ProtoConfigVal { val: Some(Bool(true)) }, "enable_mz_join_core": ProtoConfigVal { val: Some(Bool(true)) }, "enable_statement_lifecycle_logging": ProtoConfigVal { val: Some(Bool(false)) }, "kafka_client_id_enrichment_rules": ProtoConfigVal { val: Some(Json("[]")) }, "kafka_poll_max_wait": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_background_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "lgalloc_slow_clear_bytes": ProtoConfigVal { val: Some(Usize(33554432)) }, "linear_join_yielding": ProtoConfigVal { val: Some(String("work:1000000,time:100")) }, "mysql_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "mysql_replication_heartbeat_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 30, nanos: 0 })) }, "persist_batch_columnar_format": ProtoConfigVal { val: Some(String("row")) }, "persist_batch_delete_enabled": ProtoConfigVal { val: Some(Bool(false)) }, "persist_batch_record_part_format": ProtoConfigVal { val: Some(Bool(false)) }, "persist_blob_cache_mem_limit_bytes": ProtoConfigVal { val: Some(Usize(1048576)) }, "persist_blob_target_size": ProtoConfigVal { val: Some(Usize(134217728)) }, "persist_claim_unclaimed_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_compaction_minimum_timeout": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 90, nanos: 0 })) }, "persist_consensus_connection_pool_ttl": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 300, nanos: 0 })) }, "persist_consensus_connection_pool_ttl_stagger": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 6, nanos: 0 })) }, "persist_enable_arrow_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_arrow_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_enable_s3_lgalloc_cc_sizes": ProtoConfigVal { val: Some(Bool(true)) }, "persist_enable_s3_lgalloc_noncc_sizes": ProtoConfigVal { val: Some(Bool(false)) }, "persist_fetch_semaphore_cost_adjustment": ProtoConfigVal { val: Some(F64(1.2)) }, "persist_fetch_semaphore_permit_adjustment": ProtoConfigVal { val: Some(F64(1.0)) }, "persist_inline_writes_single_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_inline_writes_total_max_bytes": ProtoConfigVal { val: Some(Usize(0)) }, "persist_next_listen_batch_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 0 })) }, "persist_next_listen_batch_retryer_fixed_sleep": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 0 })) }, "persist_next_listen_batch_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 200000000 })) }, "persist_next_listen_batch_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_optimize_ignored_data_decode": ProtoConfigVal { val: Some(Bool(true)) }, "persist_optimize_ignored_data_fetch": ProtoConfigVal { val: Some(Bool(true)) }, "persist_part_decode_format": ProtoConfigVal { val: Some(String("row")) }, "persist_pubsub_client_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_pubsub_push_diff_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_reader_lease_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 900, nanos: 0 })) }, "persist_record_compactions": ProtoConfigVal { val: Some(Bool(false)) }, "persist_rollup_threshold": ProtoConfigVal { val: Some(Usize(128)) }, "persist_roundtrip_spine": ProtoConfigVal { val: Some(Bool(false)) }, "persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(0)) }, "persist_split_old_runs": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_audit_percent": ProtoConfigVal { val: Some(Usize(0)) }, "persist_stats_budget_bytes": ProtoConfigVal { val: Some(Usize(1024)) }, "persist_stats_collection_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_filter_enabled": ProtoConfigVal { val: Some(Bool(true)) }, "persist_stats_untrimmable_columns_equals": ProtoConfigVal { val: Some(String("err,ts,receivedat,createdat,_fivetran_deleted")) }, "persist_stats_untrimmable_columns_prefix": ProtoConfigVal { val: Some(String("last_")) }, "persist_stats_untrimmable_columns_suffix": ProtoConfigVal { val: Some(String("timestamp,time,_at,_tstamp")) }, "persist_txns_data_shard_retryer_clamp": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 16, nanos: 0 })) }, "persist_txns_data_shard_retryer_initial_backoff": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 1, nanos: 24000000 })) }, "persist_txns_data_shard_retryer_multiplier": ProtoConfigVal { val: Some(U32(2)) }, "persist_use_critical_since_catalog": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_snapshot": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_source": ProtoConfigVal { val: Some(Bool(false)) }, "persist_use_critical_since_txn": ProtoConfigVal { val: Some(Bool(true)) }, "persist_write_diffs_sum": ProtoConfigVal { val: Some(Bool(true)) }, "pg_offset_known_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 10, nanos: 0 })) }, "plan_insights_notice fast_path_clusters_optimize_duration": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 10000000 })) }, "postgres_fetch_slot_resume_lsn_interval": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 0, nanos: 500000000 })) }, "storage_cluster_shutdown_grace_period": ProtoConfigVal { val: Some(Duration(ProtoDuration { secs: 600, nanos: 0 })) }, "storage_dataflow_delay_sources_past_rehydration": ProtoConfigVal { val: Some(Bool(true)) }, "storage_downgrade_since_during_finalization": ProtoConfigVal { val: Some(Bool(true)) }, "storage_enforce_external_addresses": ProtoConfigVal { val: Some(Bool(false)) }, "storage_persist_sink_minimum_batch_updates": ProtoConfigVal { val: Some(Usize(1024)) }, "storage_rocksdb_cleanup_tries": ProtoConfigVal { val: Some(Usize(5)) }, "storage_rocksdb_use_merge_operator": ProtoConfigVal { val: Some(Bool(false)) }, "storage_source_decode_fuel": ProtoConfigVal { val: Some(Usize(1000000)) }, "storage_upsert_max_snapshot_batch_buffering": ProtoConfigVal { val: Some(OptUsize(ProtoOptionU64 { val: None })) }, "storage_upsert_prevent_snapshot_buffering": ProtoConfigVal { val: Some(Bool(true)) }, "use_global_txn_cache_source": ProtoConfigVal { val: Some(Bool(true)) }} } }
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.489904Z  INFO mz_compute::compute_state: disabling lgalloc
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.489906Z  INFO mz_compute::compute_state: disabling lgalloc
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.489908Z  INFO mz_compute::compute_state: using chunked stack: false
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.489909Z  INFO mz_compute::compute_state: using chunked stack: false
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.489925Z  INFO mz_persist_client::rpc: Connecting to Persist PubSub: http://localhost:6879
cluster-u10-replica-u17-gen-0: 2024-07-11T10:33:17.490315Z  INFO mz_persist_client::rpc: Connected to Persist PubSub: http://localhost:6879
environmentd: 2024-07-11T10:33:17.490404Z  INFO persist::rpc::server: mz_persist_client::rpc: Received Persist PubSub connection from: "f50feabed3a9"
environmentd: 2024-07-11T10:33:18.050336Z  INFO coord::handle_message{kind="command-execute"}:message_command:coord::handle_execute{session="68f4f7f9-c9f8-4e0b-bc6f-d821a7ca2ad5"}:coord::handle_execute_inner{stmt="DROP CLUSTER REPLICA c.r1"}:sequence_drop_objects:coord::catalog_transact:coord::catalog_transact_conn:coord::catalog_transact_inner:coord::catalog_transact_with::finalize: mz_adapter::coord::introspection: dropping introspection subscribe id=t107 replica_id=u17 type_=ComputeErrorCounts
environmentd: 2024-07-11T10:33:18.050439Z  INFO mz_compute_client::controller::replica: stopped replica task replica=User(17)
environmentd: 2024-07-11T10:33:18.050947Z  INFO mz_orchestrator_process: terminating orphaned process for cluster-u10-replica-u17-gen-0 with PID 1845
environmentd: 2024-07-11T10:33:18.054162Z  WARN persist::rpc::server:connection{caller_id="f50feabed3a9"}: mz_persist_client::rpc: pubsub connection err: status: Unknown, message: "h2 protocol error: error reading a body from connection: stream closed because of a broken pipe", details: [], metadata: MetadataMap { headers: {} }
environmentd: 2024-07-11T10:33:18.054171Z  INFO persist::rpc::server:connection{caller_id="f50feabed3a9"}: mz_persist_client::rpc: Persist PubSub connection ended: "f50feabed3a9"
environmentd: 2024-07-11T10:33:18.499823Z  INFO coord::handle_message{kind="purified_statement_ready"}:sequence_create_source:coord::catalog_transact_with_side_effects:coord::catalog_transact_inner:catalog::transact:catalog::transact_inner:transact_op: mz_adapter::catalog::transact: create source materialize.public.counter_progress (u14)
environmentd: 2024-07-11T10:33:18.500086Z  INFO coord::handle_message{kind="purified_statement_ready"}:sequence_create_source:coord::catalog_transact_with_side_effects:coord::catalog_transact_inner:catalog::transact:catalog::transact_inner:transact_op: mz_adapter::catalog::transact: create source materialize.public.counter (u15)
environmentd: 2024-07-11T10:33:18.778313Z  INFO coord::handle_message{kind="purified_statement_ready"}:sequence_create_source:coord::catalog_transact_with_side_effects:coord::catalog_transact_with_side_effects::side_effects_fut:storage::create_collections: mz_storage_controller: starting ingestion id=u15
cluster-s1-replica-s1-gen-0: 2024-07-11T10:33:20.888092Z  INFO mz_persist_client::internal::machine: snapshot saa0337ad-b3e6-4441-a357-a8d95bb71907 as of [1720693998583] not yet available for v5 upper [0]
environmentd: 2024-07-11T10:33:21.325694Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }
environmentd: 2024-07-11T10:33:26.276494Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }
environmentd: 2024-07-11T10:33:31.228086Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }
environmentd: 2024-07-11T10:33:36.259445Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }
environmentd: 2024-07-11T10:33:41.261935Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }
environmentd: 2024-07-11T10:33:46.271684Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }
environmentd: 2024-07-11T10:33:51.228058Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }
environmentd: 2024-07-11T10:33:56.273360Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }
environmentd: 2024-07-11T10:34:01.270646Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }
environmentd: 2024-07-11T10:34:06.270803Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }
environmentd: 2024-07-11T10:34:11.259540Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }
environmentd: 2024-07-11T10:34:16.254856Z  WARN mz_storage_client::storage_collections: error during finalization of shard s4ee9a640-4dd1-4441-9290-bfd3fd27ca5e: FinalizationError { since: Antichain { elements: [0] }, upper: Antichain { elements: [] } }

This is not a regression, happens also in old versions. Maybe this is not supposed to work? I'll disable dropping cluster replicas for now in parallel-workload.

teskje commented 1 month ago

The issue is that the Controller's handling of storage replicas becomes confused. Storage only supports single-replica clusters, but the top-level Controller supports multi-replica clusters because it also wraps compute, and it presents a multi-replica API to its clients.

The storage-related logic performed by the Controller when replicas are added/removed is less smart than it needs to be:

So there are two reasons why the storage controller can end up in a disconnected state even when the cluster has replicas installed.