1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
// 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.

//! Renders the statistics collection of the [`MySqlSourceConnection`] ingestion dataflow.

use std::cell::{Cell, RefCell};

use futures::StreamExt;
use timely::dataflow::operators::Map;
use timely::dataflow::{Scope, Stream};
use timely::progress::Antichain;

use mz_mysql_util::query_sys_var;
use mz_ore::future::InTask;
use mz_storage_types::sources::mysql::{gtid_set_frontier, GtidPartition, GtidState};
use mz_storage_types::sources::MySqlSourceConnection;
use mz_timely_util::builder_async::{OperatorBuilder as AsyncOperatorBuilder, PressOnDropButton};

use crate::source::types::{Probe, ProgressStatisticsUpdate};
use crate::source::RawSourceCreationConfig;

use super::{ReplicationError, TransientError};

static STATISTICS: &str = "statistics";

/// Renders the statistics dataflow.
pub(crate) fn render<G: Scope<Timestamp = GtidPartition>>(
    scope: G,
    config: RawSourceCreationConfig,
    connection: MySqlSourceConnection,
    resume_uppers: impl futures::Stream<Item = Antichain<GtidPartition>> + 'static,
) -> (
    Stream<G, ProgressStatisticsUpdate>,
    Stream<G, ReplicationError>,
    Stream<G, Probe<GtidPartition>>,
    PressOnDropButton,
) {
    let op_name = format!("MySqlStatistics({})", config.id);
    let mut builder = AsyncOperatorBuilder::new(op_name, scope);

    let (stats_output, stats_stream) = builder.new_output();
    let (probe_output, probe_stream) = builder.new_output();

    // TODO: Add additional metrics

    let (button, transient_errors) = builder.build_fallible::<TransientError, _>(move |caps| {
        Box::pin(async move {
            let worker_id = config.worker_id;
            let [stats_cap, probe_cap]: &mut [_; 2] = caps.try_into().unwrap();

            // Only run the replication reader on the worker responsible for it.
            if !config.responsible_for(STATISTICS) {
                // Emit 0, to mark this worker as having started up correctly.
                stats_output.give(
                    &stats_cap[0],
                    ProgressStatisticsUpdate::SteadyState {
                        offset_known: 0,
                        offset_committed: 0,
                    },
                );
                return Ok(());
            }

            let connection_config = connection
                .connection
                .config(
                    &config.config.connection_context.secrets_reader,
                    &config.config,
                    InTask::Yes,
                )
                .await?;

            let mut stats_conn = connection_config
                .connect(
                    &format!("timely-{worker_id} MySQL replication statistics"),
                    &config.config.connection_context.ssh_tunnel_manager,
                )
                .await?;

            tokio::pin!(resume_uppers);

            let prev_offset_known = Cell::new(None);
            let prev_offset_committed = Cell::new(None);
            let stats_output = RefCell::new(stats_output);

            let mut interval = tokio::time::interval(
                mz_storage_types::dyncfgs::MYSQL_OFFSET_KNOWN_INTERVAL
                    .get(config.config.config_set()),
            );
            let probe_loop = async {
                loop {
                    interval.tick().await;

                    let probe_ts =
                        mz_repr::Timestamp::try_from((config.now_fn)()).expect("must fit");
                    let gtid_executed =
                        query_sys_var(&mut stats_conn, "global.gtid_executed").await?;
                    // We don't translate this into a definite error like in snapshotting, but we
                    // will restart the source.
                    let upstream_frontier =
                        gtid_set_frontier(&gtid_executed).map_err(TransientError::from)?;

                    let offset_known = aggregate_mysql_frontier(&upstream_frontier);
                    if let Some(offset_committed) = prev_offset_committed.get() {
                        stats_output.borrow_mut().give(
                            &stats_cap[0],
                            ProgressStatisticsUpdate::SteadyState {
                                offset_known,
                                offset_committed,
                            },
                        );
                    }
                    prev_offset_known.set(Some(offset_known));

                    probe_output.give(
                        &probe_cap[0],
                        Probe {
                            probe_ts,
                            upstream_frontier,
                        },
                    );
                }
            };
            let commit_loop = async {
                while let Some(committed_frontier) = resume_uppers.next().await {
                    let offset_committed = aggregate_mysql_frontier(&committed_frontier);
                    if let Some(offset_known) = prev_offset_known.get() {
                        stats_output.borrow_mut().give(
                            &stats_cap[0],
                            ProgressStatisticsUpdate::SteadyState {
                                offset_known,
                                offset_committed,
                            },
                        );
                    }
                    prev_offset_committed.set(Some(offset_committed));
                }
            };

            futures::future::join(probe_loop, commit_loop).await.0
        })
    });

    (
        stats_stream,
        transient_errors.map(ReplicationError::from),
        probe_stream,
        button.press_on_drop(),
    )
}

/// Aggregate a mysql frontier into single number representing the
/// _number of transactions_ it represents.
fn aggregate_mysql_frontier(frontier: &Antichain<GtidPartition>) -> u64 {
    let mut progress_stat = 0;
    for ts in frontier.iter() {
        if let Some(_uuid) = ts.interval().singleton() {
            // We assume source id's don't disappear once they appear.
            let ts = match ts.timestamp() {
                GtidState::Absent => 0,
                // Txid's in mysql start at 1, so we subtract 1 from the _frontier_
                // to get the _number of transactions_.
                GtidState::Active(id) => id.get().saturating_sub(1),
            };
            progress_stat += ts;
        }
    }
    progress_stat
}