Skip to content

Commit 65e5dc1

Browse files
committed
Stream injected compute events
Pass compute events through a stream instead of injecting them into the compute event log. Signed-off-by: Moritz Hoffmann <mh@materialize.com>
1 parent aea5c7a commit 65e5dc1

File tree

5 files changed

+101
-81
lines changed

5 files changed

+101
-81
lines changed

src/compute/src/logging.rs

Lines changed: 14 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ use std::marker::PhantomData;
2121
use std::rc::Rc;
2222
use std::time::Duration;
2323

24-
use ::timely::container::ContainerBuilder;
24+
use ::timely::container::{CapacityContainerBuilder, ContainerBuilder};
2525
use ::timely::dataflow::channels::pact::Pipeline;
2626
use ::timely::dataflow::channels::pushers::buffer::Session;
2727
use ::timely::dataflow::channels::pushers::{Counter, Tee};
@@ -36,13 +36,26 @@ use mz_compute_client::logging::{ComputeLog, DifferentialLog, LogVariant, Timely
3636
use mz_expr::{permutation_for_arrangement, MirScalarExpr};
3737
use mz_repr::{Datum, Diff, Row, RowPacker, RowRef, Timestamp};
3838
use mz_timely_util::activator::RcActivator;
39+
use mz_timely_util::containers::ColumnBuilder;
3940
use mz_timely_util::operator::consolidate_pact;
4041

4142
use crate::logging::compute::Logger as ComputeLogger;
4243
use crate::typedefs::RowRowAgent;
4344

4445
pub use crate::logging::initialize::initialize;
4546

47+
/// An update of value `D` at a time and with a diff.
48+
pub(super) type Update<D> = (D, Timestamp, Diff);
49+
/// A pusher for containers `C`.
50+
pub(super) type Pusher<C> = Counter<Timestamp, C, Tee<Timestamp, C>>;
51+
/// An output session for the specified container builder.
52+
pub(super) type OutputSession<'a, CB> =
53+
Session<'a, Timestamp, CB, Pusher<<CB as ContainerBuilder>::Container>>;
54+
/// An output session for vector-based containers of updates `D`, using a capacity container builder.
55+
pub(super) type OutputSessionVec<'a, D> = OutputSession<'a, CapacityContainerBuilder<Vec<D>>>;
56+
/// An output session for columnar containers of updates `D`, using a column builder.
57+
pub(super) type OutputSessionColumnar<'a, D> = OutputSession<'a, ColumnBuilder<D>>;
58+
4659
/// Logs events as a timely stream, with progress statements.
4760
struct BatchLogger<C, P>
4861
where
@@ -216,10 +229,6 @@ struct LogCollection {
216229
token: Rc<dyn Any>,
217230
}
218231

219-
pub(super) type Pusher<C> = Counter<Timestamp, C, Tee<Timestamp, C>>;
220-
pub(super) type OutputSession<'a, CB> =
221-
Session<'a, Timestamp, CB, Pusher<<CB as ContainerBuilder>::Container>>;
222-
223232
/// A single-purpose function to consolidate and pack updates for log collection.
224233
///
225234
/// The function first consolidates worker-local updates using the [`Pipeline`] pact, then converts

src/compute/src/logging/compute.rs

Lines changed: 24 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -24,22 +24,20 @@ use mz_ore::cast::CastFrom;
2424
use mz_repr::{Datum, Diff, GlobalId, Timestamp};
2525
use mz_timely_util::containers::{Column, ColumnBuilder, ProvidedBuilder};
2626
use mz_timely_util::replay::MzReplay;
27-
use timely::container::CapacityContainerBuilder;
2827
use timely::dataflow::channels::pact::Pipeline;
29-
use timely::dataflow::channels::pushers::buffer::Session;
30-
use timely::dataflow::channels::pushers::{Counter, Tee};
3128
use timely::dataflow::operators::core::Map;
3229
use timely::dataflow::operators::generic::builder_rc::OperatorBuilder;
33-
use timely::dataflow::operators::Operator;
34-
use timely::dataflow::{Scope, Stream};
30+
use timely::dataflow::operators::{Concatenate, Enter, Operator};
31+
use timely::dataflow::{Scope, Stream, StreamCore};
3532
use timely::scheduling::Scheduler;
3633
use timely::{Container, Data};
3734
use tracing::error;
3835
use uuid::Uuid;
3936

4037
use crate::extensions::arrange::MzArrange;
4138
use crate::logging::{
42-
ComputeLog, EventQueue, LogCollection, LogVariant, PermutedRowPacker, SharedLoggingState,
39+
ComputeLog, EventQueue, LogCollection, LogVariant, OutputSessionColumnar, OutputSessionVec,
40+
PermutedRowPacker, SharedLoggingState, Update,
4341
};
4442
use crate::row_spine::{RowRowBatcher, RowRowBuilder};
4543
use crate::typedefs::RowRowSpine;
@@ -299,6 +297,7 @@ pub(super) fn construct<A: Scheduler + 'static, S: Scope<Timestamp = Timestamp>>
299297
scheduler: A,
300298
config: &mz_compute_client::logging::LoggingConfig,
301299
event_queue: EventQueue<Column<(Duration, ComputeEvent)>>,
300+
compute_event_stream: impl IntoIterator<Item = StreamCore<S, Column<(Duration, ComputeEvent)>>>,
302301
shared_state: Rc<RefCell<SharedLoggingState>>,
303302
) -> BTreeMap<LogVariant, LogCollection> {
304303
let logging_interval_ms = std::cmp::max(1, config.interval.as_millis());
@@ -319,6 +318,12 @@ pub(super) fn construct<A: Scheduler + 'static, S: Scope<Timestamp = Timestamp>>
319318
},
320319
);
321320

321+
let logs = compute_event_stream
322+
.into_iter()
323+
.map(|stream| stream.enter(scope))
324+
.chain(std::iter::once(logs));
325+
let logs = scope.concatenate(logs);
326+
322327
// Build a demux operator that splits the replayed event stream up into the separate
323328
// logging streams.
324329
let mut demux = OperatorBuilder::new("Compute Logging Demux".to_string(), scope.clone());
@@ -663,34 +668,21 @@ struct ArrangementSizeState {
663668
count: isize,
664669
}
665670

666-
/// An update of value `D` at a time and with a diff.
667-
type Update<D> = (D, Timestamp, Diff);
668-
/// A pusher for updates of value `D` for vector-based containers.
669-
type Pusher<D> = Counter<Timestamp, Vec<Update<D>>, Tee<Timestamp, Vec<Update<D>>>>;
670-
/// A pusher for updates of value `D` for columnar containers.
671-
type PusherColumnar<D> = Counter<Timestamp, Column<Update<D>>, Tee<Timestamp, Column<Update<D>>>>;
672-
/// An output session for vector-based containers of updates `D`, using a capacity container builder.
673-
type OutputSession<'a, D> =
674-
Session<'a, Timestamp, CapacityContainerBuilder<Vec<Update<D>>>, Pusher<D>>;
675-
/// An output session for columnar containers of updates `D`, using a column builder.
676-
type OutputSessionColumnar<'a, D> =
677-
Session<'a, Timestamp, ColumnBuilder<Update<D>>, PusherColumnar<D>>;
678-
679671
/// Bundled output sessions used by the demux operator.
680672
struct DemuxOutput<'a> {
681-
export: OutputSession<'a, ExportDatum>,
682-
frontier: OutputSession<'a, FrontierDatum>,
683-
import_frontier: OutputSession<'a, ImportFrontierDatum>,
684-
peek: OutputSession<'a, PeekDatum>,
685-
peek_duration: OutputSession<'a, PeekDurationDatum>,
686-
shutdown_duration: OutputSession<'a, u128>,
687-
arrangement_heap_size: OutputSession<'a, ArrangementHeapDatum>,
688-
arrangement_heap_capacity: OutputSession<'a, ArrangementHeapDatum>,
689-
arrangement_heap_allocations: OutputSession<'a, ArrangementHeapDatum>,
690-
hydration_time: OutputSession<'a, HydrationTimeDatum>,
691-
error_count: OutputSession<'a, ErrorCountDatum>,
692-
lir_mapping: OutputSessionColumnar<'a, LirMappingDatum>,
693-
dataflow_global_ids: OutputSession<'a, DataflowGlobalDatum>,
673+
export: OutputSessionVec<'a, Update<ExportDatum>>,
674+
frontier: OutputSessionVec<'a, Update<FrontierDatum>>,
675+
import_frontier: OutputSessionVec<'a, Update<ImportFrontierDatum>>,
676+
peek: OutputSessionVec<'a, Update<PeekDatum>>,
677+
peek_duration: OutputSessionVec<'a, Update<PeekDurationDatum>>,
678+
shutdown_duration: OutputSessionVec<'a, Update<u128>>,
679+
arrangement_heap_size: OutputSessionVec<'a, Update<ArrangementHeapDatum>>,
680+
arrangement_heap_capacity: OutputSessionVec<'a, Update<ArrangementHeapDatum>>,
681+
arrangement_heap_allocations: OutputSessionVec<'a, Update<ArrangementHeapDatum>>,
682+
hydration_time: OutputSessionVec<'a, Update<HydrationTimeDatum>>,
683+
error_count: OutputSessionVec<'a, Update<ErrorCountDatum>>,
684+
lir_mapping: OutputSessionColumnar<'a, Update<LirMappingDatum>>,
685+
dataflow_global_ids: OutputSessionVec<'a, Update<DataflowGlobalDatum>>,
694686
}
695687

696688
#[derive(Clone)]

src/compute/src/logging/differential.rs

Lines changed: 29 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -21,24 +21,30 @@ use differential_dataflow::logging::{
2121
use mz_ore::cast::CastFrom;
2222
use mz_repr::{Datum, Diff, Timestamp};
2323
use mz_timely_util::containers::{
24-
columnar_exchange, Col2ValBatcher, ColumnBuilder, ProvidedBuilder,
24+
columnar_exchange, Col2ValBatcher, Column, ColumnBuilder, ProvidedBuilder,
2525
};
2626
use mz_timely_util::replay::MzReplay;
2727
use timely::dataflow::channels::pact::{ExchangeCore, Pipeline};
2828
use timely::dataflow::channels::pushers::buffer::Session;
2929
use timely::dataflow::channels::pushers::{Counter, Tee};
3030
use timely::dataflow::operators::generic::builder_rc::OperatorBuilder;
31-
use timely::dataflow::{Scope, Stream};
31+
use timely::dataflow::operators::Leave;
32+
use timely::dataflow::{Scope, Stream, StreamCore};
3233

3334
use crate::extensions::arrange::MzArrangeCore;
3435
use crate::logging::compute::{ArrangementHeapSizeOperatorDrop, ComputeEvent};
3536
use crate::logging::{
3637
consolidate_and_pack, DifferentialLog, EventQueue, LogCollection, LogVariant,
37-
SharedLoggingState,
38+
OutputSessionColumnar, SharedLoggingState,
3839
};
3940
use crate::row_spine::RowRowBuilder;
4041
use crate::typedefs::{KeyBatcher, RowRowSpine};
4142

43+
pub(super) struct Return<S: Scope> {
44+
pub collections: BTreeMap<LogVariant, LogCollection>,
45+
pub compute_events: StreamCore<S, Column<(Duration, ComputeEvent)>>,
46+
}
47+
4248
/// Constructs the logging dataflow for differential logs.
4349
///
4450
/// Params
@@ -50,7 +56,7 @@ pub(super) fn construct<S: Scope<Timestamp = Timestamp>>(
5056
config: &mz_compute_client::logging::LoggingConfig,
5157
event_queue: EventQueue<Vec<(Duration, DifferentialEvent)>>,
5258
shared_state: Rc<RefCell<SharedLoggingState>>,
53-
) -> BTreeMap<LogVariant, LogCollection> {
59+
) -> Return<S> {
5460
let logging_interval_ms = std::cmp::max(1, config.interval.as_millis());
5561

5662
scope.scoped("Dataflow: differential logging", move |scope| {
@@ -82,6 +88,7 @@ pub(super) fn construct<S: Scope<Timestamp = Timestamp>>(
8288
let (mut batcher_size_out, batcher_size) = demux.new_output();
8389
let (mut batcher_capacity_out, batcher_capacity) = demux.new_output();
8490
let (mut batcher_allocations_out, batcher_allocations) = demux.new_output();
91+
let (mut compute_events_out, compute_events) = demux.new_output();
8592

8693
let mut demux_state = Default::default();
8794
demux.build(move |_capability| {
@@ -93,6 +100,7 @@ pub(super) fn construct<S: Scope<Timestamp = Timestamp>>(
93100
let mut batcher_size = batcher_size_out.activate();
94101
let mut batcher_capacity = batcher_capacity_out.activate();
95102
let mut batcher_allocations = batcher_allocations_out.activate();
103+
let mut compute_events_out = compute_events_out.activate();
96104

97105
input.for_each(|cap, data| {
98106
let mut output_buffers = DemuxOutput {
@@ -103,6 +111,7 @@ pub(super) fn construct<S: Scope<Timestamp = Timestamp>>(
103111
batcher_size: batcher_size.session_with_builder(&cap),
104112
batcher_capacity: batcher_capacity.session_with_builder(&cap),
105113
batcher_allocations: batcher_allocations.session_with_builder(&cap),
114+
compute_events: compute_events_out.session_with_builder(&cap),
106115
};
107116

108117
for (time, event) in data.drain(..) {
@@ -157,7 +166,7 @@ pub(super) fn construct<S: Scope<Timestamp = Timestamp>>(
157166
];
158167

159168
// Build the output arrangements.
160-
let mut result = BTreeMap::new();
169+
let mut collections = BTreeMap::new();
161170
for (variant, collection) in logs {
162171
let variant = LogVariant::Differential(variant);
163172
if config.index_logs.contains_key(&variant) {
@@ -171,11 +180,11 @@ pub(super) fn construct<S: Scope<Timestamp = Timestamp>>(
171180
trace,
172181
token: Rc::clone(&token),
173182
};
174-
result.insert(variant, collection);
183+
collections.insert(variant, collection);
175184
}
176185
}
177186

178-
result
187+
Return { collections, compute_events: compute_events.leave() }
179188
})
180189
}
181190

@@ -193,6 +202,7 @@ struct DemuxOutput<'a> {
193202
batcher_size: OutputSession<'a, (usize, ())>,
194203
batcher_capacity: OutputSession<'a, (usize, ())>,
195204
batcher_allocations: OutputSession<'a, (usize, ())>,
205+
compute_events: OutputSessionColumnar<'a, (Duration, ComputeEvent)>,
196206
}
197207

198208
/// State maintained by the demux operator.
@@ -284,17 +294,18 @@ impl DemuxHandler<'_, '_> {
284294
debug_assert_ne!(diff, 0);
285295
self.output.sharing.give(((operator_id, ()), ts, diff));
286296

287-
if let Some(logger) = &mut self.shared_state.compute_logger {
288-
let sharing = self.state.sharing.entry(operator_id).or_default();
289-
*sharing = (i64::try_from(*sharing).expect("must fit") + diff)
290-
.try_into()
291-
.expect("under/overflow");
292-
if *sharing == 0 {
293-
self.state.sharing.remove(&operator_id);
294-
logger.log(&ComputeEvent::ArrangementHeapSizeOperatorDrop(
295-
ArrangementHeapSizeOperatorDrop { operator_id },
296-
));
297-
}
297+
let sharing = self.state.sharing.entry(operator_id).or_default();
298+
*sharing = (i64::try_from(*sharing).expect("must fit") + diff)
299+
.try_into()
300+
.expect("under/overflow");
301+
if *sharing == 0 {
302+
self.state.sharing.remove(&operator_id);
303+
self.output.compute_events.give(&(
304+
self.time,
305+
ComputeEvent::ArrangementHeapSizeOperatorDrop(ArrangementHeapSizeOperatorDrop {
306+
operator_id,
307+
}),
308+
));
298309
}
299310
}
300311

src/compute/src/logging/initialize.rs

Lines changed: 12 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -102,28 +102,32 @@ impl<A: Allocate + 'static> LoggingContext<'_, A> {
102102
let dataflow_index = self.worker.next_dataflow_index();
103103
self.worker.dataflow_named("Dataflow: logging", |scope| {
104104
let mut collections = BTreeMap::new();
105-
collections.extend(super::timely::construct(
106-
scope.clone(),
107-
self.config,
108-
self.t_event_queue.clone(),
109-
Rc::clone(&self.shared_state),
110-
));
105+
let super::timely::Return {
106+
collections: timely_collections,
107+
compute_events: compute_events_timely,
108+
} = super::timely::construct(scope.clone(), self.config, self.t_event_queue.clone());
109+
collections.extend(timely_collections);
111110
collections.extend(super::reachability::construct(
112111
scope.clone(),
113112
self.config,
114113
self.r_event_queue.clone(),
115114
));
116-
collections.extend(super::differential::construct(
115+
let super::differential::Return {
116+
collections: differential_collections,
117+
compute_events: compute_events_differential,
118+
} = super::differential::construct(
117119
scope.clone(),
118120
self.config,
119121
self.d_event_queue.clone(),
120122
Rc::clone(&self.shared_state),
121-
));
123+
);
124+
collections.extend(differential_collections);
122125
collections.extend(super::compute::construct(
123126
scope.clone(),
124127
scope.parent.clone(),
125128
self.config,
126129
self.c_event_queue.clone(),
130+
[compute_events_timely, compute_events_differential],
127131
Rc::clone(&self.shared_state),
128132
));
129133

0 commit comments

Comments
 (0)