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 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.
use std::convert::Infallible;
use std::time::Duration;
use differential_dataflow::{AsCollection, Collection};
use futures::StreamExt;
use mz_repr::{Diff, Row};
use mz_storage_types::sources::load_generator::{
Event, Generator, KeyValueLoadGenerator, LoadGenerator, LoadGeneratorSourceConnection,
};
use mz_storage_types::sources::{MzOffset, SourceTimestamp};
use mz_timely_util::builder_async::{OperatorBuilder as AsyncOperatorBuilder, PressOnDropButton};
use timely::dataflow::operators::ToStream;
use timely::dataflow::{Scope, Stream};
use timely::progress::Antichain;
use crate::healthcheck::{HealthStatusMessage, HealthStatusUpdate, StatusNamespace};
use crate::source::types::{ProgressStatisticsUpdate, SourceRender};
use crate::source::{RawSourceCreationConfig, SourceMessage, SourceReaderError};
mod auction;
mod counter;
mod datums;
mod key_value;
mod marketing;
mod tpch;
pub use auction::Auction;
pub use counter::Counter;
pub use datums::Datums;
pub use tpch::Tpch;
use self::marketing::Marketing;
enum GeneratorKind {
Simple {
generator: Box<dyn Generator>,
tick_micros: Option<u64>,
// Load generators cannot be rendered until all of their exports are
// present.
//
// TODO(#26765): can this limitation be removed?
required_exports: usize,
},
KeyValue(KeyValueLoadGenerator),
}
impl GeneratorKind {
fn new(g: &LoadGenerator, tick_micros: Option<u64>) -> Self {
let required_exports = g.views().len() + 1;
match g {
LoadGenerator::Auction => GeneratorKind::Simple {
generator: Box::new(Auction {}),
tick_micros,
required_exports,
},
LoadGenerator::Counter { max_cardinality } => GeneratorKind::Simple {
generator: Box::new(Counter {
max_cardinality: max_cardinality.clone(),
}),
tick_micros,
required_exports,
},
LoadGenerator::Datums => GeneratorKind::Simple {
generator: Box::new(Datums {}),
tick_micros,
required_exports,
},
LoadGenerator::Marketing => GeneratorKind::Simple {
generator: Box::new(Marketing {}),
tick_micros,
required_exports,
},
LoadGenerator::Tpch {
count_supplier,
count_part,
count_customer,
count_orders,
count_clerk,
} => GeneratorKind::Simple {
generator: Box::new(Tpch {
count_supplier: *count_supplier,
count_part: *count_part,
count_customer: *count_customer,
count_orders: *count_orders,
count_clerk: *count_clerk,
// The default tick behavior 1s. For tpch we want to disable ticking
// completely.
tick: Duration::from_micros(tick_micros.unwrap_or(0)),
}),
tick_micros,
required_exports,
},
LoadGenerator::KeyValue(kv) => {
mz_ore::soft_assert_eq_or_log!(
required_exports,
1,
"KeyValue generators should not have any additional views"
);
GeneratorKind::KeyValue(kv.clone())
}
}
}
fn render<G: Scope<Timestamp = MzOffset>>(
self,
scope: &mut G,
config: RawSourceCreationConfig,
committed_uppers: impl futures::Stream<Item = Antichain<MzOffset>> + 'static,
start_signal: impl std::future::Future<Output = ()> + 'static,
) -> (
Collection<G, (usize, Result<SourceMessage, SourceReaderError>), Diff>,
Option<Stream<G, Infallible>>,
Stream<G, HealthStatusMessage>,
Stream<G, ProgressStatisticsUpdate>,
Vec<PressOnDropButton>,
) {
match self {
GeneratorKind::Simple {
tick_micros,
generator,
required_exports,
} => render_simple_generator(
generator,
tick_micros,
scope,
config,
committed_uppers,
required_exports,
),
GeneratorKind::KeyValue(kv) => {
key_value::render(kv, scope, config, committed_uppers, start_signal)
}
}
}
}
impl SourceRender for LoadGeneratorSourceConnection {
type Time = MzOffset;
const STATUS_NAMESPACE: StatusNamespace = StatusNamespace::Generator;
fn render<G: Scope<Timestamp = MzOffset>>(
self,
scope: &mut G,
config: RawSourceCreationConfig,
committed_uppers: impl futures::Stream<Item = Antichain<MzOffset>> + 'static,
start_signal: impl std::future::Future<Output = ()> + 'static,
) -> (
Collection<G, (usize, Result<SourceMessage, SourceReaderError>), Diff>,
Option<Stream<G, Infallible>>,
Stream<G, HealthStatusMessage>,
Stream<G, ProgressStatisticsUpdate>,
Vec<PressOnDropButton>,
) {
let generator_kind = GeneratorKind::new(&self.load_generator, self.tick_micros);
generator_kind.render(scope, config, committed_uppers, start_signal)
}
}
fn render_simple_generator<G: Scope<Timestamp = MzOffset>>(
generator: Box<dyn Generator>,
tick_micros: Option<u64>,
scope: &mut G,
config: RawSourceCreationConfig,
committed_uppers: impl futures::Stream<Item = Antichain<MzOffset>> + 'static,
required_exports: usize,
) -> (
Collection<G, (usize, Result<SourceMessage, SourceReaderError>), Diff>,
Option<Stream<G, Infallible>>,
Stream<G, HealthStatusMessage>,
Stream<G, ProgressStatisticsUpdate>,
Vec<PressOnDropButton>,
) {
let mut builder = AsyncOperatorBuilder::new(config.name.clone(), scope.clone());
let (mut data_output, stream) = builder.new_output();
let (mut stats_output, stats_stream) = builder.new_output();
let button = builder.build(move |caps| async move {
// Do not run the load generator until we have all of our source
// exports. Waiting here is fine because we know that their creation and
// scheduling of this dataflow is imminent.
//
// TODO(#26765): can this limitation be removed?
if required_exports != config.source_exports.len() {
std::future::pending().await
}
let [mut cap, stats_cap]: [_; 2] = caps.try_into().unwrap();
if !config.responsible_for(()) {
// Emit 0, to mark this worker as having started up correctly.
stats_output
.give(
&stats_cap,
ProgressStatisticsUpdate::SteadyState {
offset_known: 0,
offset_committed: 0,
},
)
.await;
return;
}
let resume_upper = Antichain::from_iter(
config.source_resume_uppers[&config.id]
.iter()
.map(MzOffset::decode_row),
);
let Some(resume_offset) = resume_upper.into_option() else {
return;
};
let mut rows = generator.by_seed(mz_ore::now::SYSTEM_TIME.clone(), None, resume_offset);
let tick = Duration::from_micros(tick_micros.unwrap_or(1_000_000));
let mut committed_uppers = std::pin::pin!(committed_uppers);
// If we are just starting up, report 0 as our `offset_committed`.
let mut offset_committed = if resume_offset.offset == 0 {
Some(0)
} else {
None
};
while let Some((output, event)) = rows.next() {
match event {
Event::Message(offset, (value, diff)) => {
let message = (
output,
Ok(SourceMessage {
key: Row::default(),
value,
metadata: Row::default(),
}),
);
// Some generators always reproduce their TVC from the beginning which can
// generate a significant amount of data that will overwhelm the dataflow.
// Since those are not required downstream we eagerly ignore them here.
if resume_offset <= offset {
data_output.give(&cap, (message, offset, diff)).await;
}
}
Event::Progress(Some(offset)) => {
cap.downgrade(&offset);
// We only sleep if we have surpassed the resume offset so that we can
// quickly go over any historical updates that a generator might choose to
// emit.
// TODO(petrosagg): Remove the sleep below and make generators return an
// async stream so that they can drive the rate of production directly
if resume_offset < offset {
let mut sleep = std::pin::pin!(tokio::time::sleep(tick));
loop {
tokio::select! {
_ = &mut sleep => {
break;
}
Some(frontier) = committed_uppers.next() => {
if let Some(offset) = frontier.as_option() {
// Offset N means we have committed N offsets (offsets are
// 0-indexed)
offset_committed = Some(offset.offset);
}
}
}
}
// TODO(guswynn): generators have various definitions of "snapshot", so
// we are not going to implement snapshot progress statistics for them
// right now, but will come back to it.
if let Some(offset_committed) = offset_committed {
stats_output
.give(
&stats_cap,
ProgressStatisticsUpdate::SteadyState {
// technically we could have _known_ a larger offset
// than the one that has been committed, but we can
// never recover that known amount on restart, so we
// just advance these in lock step.
offset_known: offset_committed,
offset_committed,
},
)
.await;
}
}
}
Event::Progress(None) => return,
}
}
});
let status = [HealthStatusMessage {
index: 0,
namespace: StatusNamespace::Generator,
update: HealthStatusUpdate::running(),
}]
.to_stream(scope);
(
stream.as_collection(),
None,
status,
stats_stream,
vec![button.press_on_drop()],
)
}