mz_storage/source/sql_server/progress.rs
1// Copyright Materialize, Inc. and contributors. All rights reserved.
2//
3// Use of this software is governed by the Business Source License
4// included in the LICENSE file.
5//
6// As of the Change Date specified in that file, in accordance with
7// the Business Source License, use of this software will be governed
8// by the Apache License, Version 2.0.
9
10//! A "non-critical" operator that tracks the progress of a [`SqlServerSourceConnection`].
11//!
12//! The operator does the following:
13//!
14//! * At some cadence [`OFFSET_KNOWN_INTERVAL`] will probe the source for the max
15//! [`Lsn`], emit the upstream known offset, and update `SourceStatistics`.
16//! * Listen to a provided [`futures::Stream`] of resume uppers, which represents
17//! the durably committed upper for _all_ of the subsources/exports associated
18//! with this source. As the source makes progress this operator does two
19//! things:
20//! 1. If [`CDC_CLEANUP_CHANGE_TABLE`] is enabled, will delete entries from
21//! the upstream change table that we've already ingested.
22//! 2. Update `SourceStatistics` to notify listeners of a new
23//! "committed LSN".
24//!
25//! [`SqlServerSourceConnection`]: mz_storage_types::sources::SqlServerSourceConnection
26
27use std::collections::{BTreeMap, BTreeSet};
28
29use futures::StreamExt;
30use mz_ore::future::InTask;
31use mz_repr::GlobalId;
32use mz_sql_server_util::cdc::Lsn;
33use mz_sql_server_util::inspect::get_latest_restore_history_id;
34use mz_storage_types::connections::SqlServerConnectionDetails;
35use mz_storage_types::sources::SqlServerSourceExtras;
36use mz_storage_types::sources::sql_server::{
37 CDC_CLEANUP_CHANGE_TABLE, CDC_CLEANUP_CHANGE_TABLE_MAX_DELETES, OFFSET_KNOWN_INTERVAL,
38};
39use mz_timely_util::builder_async::{OperatorBuilder as AsyncOperatorBuilder, PressOnDropButton};
40use timely::container::CapacityContainerBuilder;
41use timely::dataflow::operators::Map;
42use timely::dataflow::{Scope, Stream as TimelyStream};
43use timely::progress::Antichain;
44
45use crate::source::sql_server::{ReplicationError, SourceOutputInfo, TransientError};
46use crate::source::types::Probe;
47use crate::source::{RawSourceCreationConfig, probe};
48
49/// Used as a partition ID to determine the worker that is responsible for
50/// handling progress.
51static PROGRESS_WORKER: &str = "progress";
52
53pub(crate) fn render<G: Scope<Timestamp = Lsn>>(
54 scope: G,
55 config: RawSourceCreationConfig,
56 connection: SqlServerConnectionDetails,
57 outputs: BTreeMap<GlobalId, SourceOutputInfo>,
58 committed_uppers: impl futures::Stream<Item = Antichain<Lsn>> + 'static,
59 extras: SqlServerSourceExtras,
60) -> (
61 TimelyStream<G, ReplicationError>,
62 TimelyStream<G, Probe<Lsn>>,
63 PressOnDropButton,
64) {
65 let op_name = format!("SqlServerProgress({})", config.id);
66 let mut builder = AsyncOperatorBuilder::new(op_name, scope);
67
68 let (probe_output, probe_stream) = builder.new_output::<CapacityContainerBuilder<_>>();
69
70 let (button, transient_errors) = builder.build_fallible::<TransientError, _>(move |caps| {
71 Box::pin(async move {
72 let [probe_cap]: &mut [_; 1] = caps.try_into().unwrap();
73
74 let emit_probe = |cap, probe: Probe<Lsn>| {
75 probe_output.give(cap, probe);
76 };
77
78 // Only a single worker is responsible for processing progress.
79 if !config.responsible_for(PROGRESS_WORKER) {
80 // Emit 0 to mark this worker as having started up correctly.
81 for stat in config.statistics.values() {
82 stat.set_offset_known(0);
83 stat.set_offset_committed(0);
84 }
85 }
86 let conn_config = connection
87 .resolve_config(
88 &config.config.connection_context.secrets_reader,
89 &config.config,
90 InTask::Yes,
91 )
92 .await?;
93 let mut client = mz_sql_server_util::Client::connect(conn_config).await?;
94
95
96 // Terminate the progress probes if a restore has happened. Replication operator will
97 // emit a definite error at the max LSN, but we also have to terminate the RLU probes
98 // to ensure that the error propogates to downstream consumers, otherwise it will
99 // wait in reclock as the server LSN will always be less than the LSN of the definite
100 // error.
101 let current_restore_history_id = get_latest_restore_history_id(&mut client).await?;
102 if current_restore_history_id != extras.restore_history_id {
103 tracing::warn!("Restore detected, exiting");
104 return Ok(());
105 }
106
107
108 let probe_interval = OFFSET_KNOWN_INTERVAL.handle(config.config.config_set());
109 let mut probe_ticker = probe::Ticker::new(|| probe_interval.get(), config.now_fn);
110
111 // Offset that is measured from the upstream SQL Server instance. Tracked to detect an offset that moves backwards.
112 let mut prev_offset_known: Option<Lsn> = None;
113
114 // This stream of "resume uppers" tracks all of the Lsn's that we have durably
115 // committed for all subsources/exports and thus we can notify the upstream that the
116 // change tables can be cleaned up.
117 let mut committed_uppers = std::pin::pin!(committed_uppers);
118 let cleanup_change_table = CDC_CLEANUP_CHANGE_TABLE.handle(config.config.config_set());
119 let cleanup_max_deletes = CDC_CLEANUP_CHANGE_TABLE_MAX_DELETES.handle(config.config.config_set());
120 let capture_instances: BTreeSet<_> = outputs.into_values().map(|info| info.capture_instance).collect();
121
122 loop {
123 tokio::select! {
124 probe_ts = probe_ticker.tick() => {
125 let max_lsn: Lsn = mz_sql_server_util::inspect::get_max_lsn(&mut client).await?;
126 // We have to return max_lsn + 1 in the probe so that the downstream consumers of
127 // the probe view the actual max lsn as fully committed and all data at that LSN
128 // as no longer subject to change. If we don't increment the LSN before emitting
129 // the probe then data will not be queryable in the tables produced by the Source.
130 let known_lsn = max_lsn.increment();
131 for stat in config.statistics.values() {
132 stat.set_offset_known(known_lsn.abbreviate());
133 }
134
135
136 // The DB should never go backwards, but it's good to know if it does.
137 let prev_known_lsn = match prev_offset_known {
138 None => {
139 prev_offset_known = Some(known_lsn);
140 known_lsn
141 },
142 Some(prev) => prev,
143 };
144 if known_lsn < prev_known_lsn {
145 mz_ore::soft_panic_or_log!(
146 "upstream SQL Server went backwards in time, current LSN: {known_lsn}, last known {prev_known_lsn}",
147 );
148 continue;
149 }
150 let probe = Probe { probe_ts, upstream_frontier: Antichain::from_elem(known_lsn) };
151 emit_probe(&probe_cap[0], probe);
152 prev_offset_known = Some(known_lsn);
153 },
154 Some(committed_upper) = committed_uppers.next() => {
155 let Some(committed_upper) = committed_upper.as_option() else {
156 // It's possible that the source has been dropped, in which case this can
157 // observe an empty upper. This operator should continue to loop until
158 // the drop dataflow propagates.
159 continue;
160 };
161
162 // If enabled, tell the upstream SQL Server instance to
163 // cleanup the underlying change table.
164 if cleanup_change_table.get() {
165 for instance in &capture_instances {
166 // TODO(sql_server3): The number of rows that got cleaned
167 // up should be present in informational notices sent back
168 // from the upstream, but the tiberius crate does not
169 // expose these.
170 let cleanup_result = mz_sql_server_util::inspect::cleanup_change_table(
171 &mut client,
172 instance,
173 committed_upper,
174 cleanup_max_deletes.get(),
175 ).await;
176 // TODO(sql_server2): Track this in a more user observable way.
177 if let Err(err) = cleanup_result {
178 tracing::warn!(?err, %instance, "cleanup of change table failed!");
179 }
180 }
181 }
182 for stat in config.statistics.values() {
183 stat.set_offset_committed(committed_upper.abbreviate());
184 }
185 }
186 };
187 }
188 })
189 });
190
191 let error_stream = transient_errors.map(ReplicationError::Transient);
192
193 (error_stream, probe_stream, button.press_on_drop())
194}