Skip to content

Commit 087dba3

Browse files
Partition storage and compute dataflows (reissue) (MaterializeInc#10531)
* Partition storage and compute dataflows (MaterializeInc#10519) * Partition storage and compute dataflows * Improve abstractions and comments * tidying comments * dataflow: Remove mut requirement for RcActivator The functions use inner mutability and don't require mutable access to the object. * dataflow: Event-based replay for sources Signed-off-by: Moritz Hoffmann <mh@materialize.com> * Wrap only push half with activator Co-authored-by: Frank McSherry <fmcsherry@me.com>
1 parent 1183d06 commit 087dba3

File tree

7 files changed

+195
-43
lines changed

7 files changed

+195
-43
lines changed

src/dataflow/src/activator.rs

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -47,20 +47,20 @@ impl RcActivator {
4747
}
4848

4949
/// Register an additional [Activator] with this [RcActivator].
50-
pub fn register(&mut self, activator: Activator) {
50+
pub fn register(&self, activator: Activator) {
5151
self.inner.borrow_mut().register(activator)
5252
}
5353

5454
/// Activate all contained activators.
5555
///
5656
/// The implementation is free to ignore activations and only release them once a sufficient
5757
/// volume has been accumulated.
58-
pub fn activate(&mut self) {
58+
pub fn activate(&self) {
5959
self.inner.borrow_mut().activate()
6060
}
6161

6262
/// Acknowledge the activation, which enables new activations to be scheduled.
63-
pub fn ack(&mut self) {
63+
pub fn ack(&self) {
6464
self.inner.borrow_mut().ack()
6565
}
6666
}

src/dataflow/src/event.rs

Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,38 @@
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+
//! Traits and types for describing captured timely dataflow streams.
11+
//!
12+
//! This is roughly based on [timely::dataflow::operators::capture::event].
13+
14+
use crate::activator::RcActivator;
15+
use timely::dataflow::operators::capture::{EventCore, EventPusherCore};
16+
17+
/// An event pusher wrapper that activates targets on push.
18+
#[derive(Clone, Debug)]
19+
pub struct ActivatedEventPusher<E> {
20+
pub inner: E,
21+
pub activator: RcActivator,
22+
}
23+
24+
impl<E> ActivatedEventPusher<E> {
25+
/// Create a new activated event link wrapper.
26+
///
27+
/// * inner: A wrapped event pusher/iterator.
28+
pub fn new(inner: E, activator: RcActivator) -> Self {
29+
Self { inner, activator }
30+
}
31+
}
32+
33+
impl<T, D, E: EventPusherCore<T, D>> EventPusherCore<T, D> for ActivatedEventPusher<E> {
34+
fn push(&mut self, event: EventCore<T, D>) {
35+
self.inner.push(event);
36+
self.activator.activate();
37+
}
38+
}

src/dataflow/src/lib.rs

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@
1414
mod activator;
1515
mod arrangement;
1616
mod decode;
17+
mod event;
1718
mod metrics;
1819
mod operator;
1920
mod render;

src/dataflow/src/render/mod.rs

Lines changed: 115 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -102,18 +102,23 @@
102102
103103
use std::any::Any;
104104
use std::cell::RefCell;
105-
use std::collections::{HashMap, HashSet};
105+
use std::collections::{BTreeMap, HashMap, HashSet};
106106
use std::rc::Rc;
107107
use std::rc::Weak;
108+
use std::time::Duration;
108109

109110
use differential_dataflow::AsCollection;
110111
use timely::communication::Allocate;
112+
use timely::dataflow::operators::capture::EventLink;
111113
use timely::dataflow::operators::to_stream::ToStream;
114+
use timely::dataflow::operators::Capture;
112115
use timely::dataflow::scopes::Child;
113116
use timely::dataflow::Scope;
117+
114118
use timely::progress::Antichain;
115119
use timely::worker::Worker as TimelyWorker;
116120

121+
use crate::activator::RcActivator;
117122
use dataflow_types::*;
118123
use expr::{GlobalId, Id};
119124
use itertools::Itertools;
@@ -123,10 +128,12 @@ use persist::client::RuntimeClient;
123128
use repr::{Row, Timestamp};
124129

125130
use crate::arrangement::manager::{TraceBundle, TraceManager};
131+
use crate::event::ActivatedEventPusher;
126132
use crate::metrics::Metrics;
127133
use crate::render::context::CollectionBundle;
128134
use crate::render::context::{ArrangementFlavor, Context};
129135
use crate::render::sources::PersistedSourceManager;
136+
use crate::replay::MzReplay;
130137
use crate::server::LocalInput;
131138
use crate::sink::SinkBaseMetrics;
132139
use crate::source::metrics::SourceBaseMetrics;
@@ -205,28 +212,47 @@ pub struct RelevantTokens {
205212
pub cdc_tokens: HashMap<GlobalId, Rc<dyn Any>>,
206213
}
207214

208-
/// Build a dataflow from a description.
209-
pub fn build_dataflow<A: Allocate>(
215+
/// Information about each source that must be communicated between storage and compute layers.
216+
pub struct SourceBoundary {
217+
/// Captured `row` updates representing a differential collection.
218+
pub ok:
219+
ActivatedEventPusher<Rc<EventLink<repr::Timestamp, (Row, repr::Timestamp, repr::Diff)>>>,
220+
/// Captured error updates representing a differential collection.
221+
pub err: ActivatedEventPusher<
222+
Rc<EventLink<repr::Timestamp, (DataflowError, repr::Timestamp, repr::Diff)>>,
223+
>,
224+
/// A token that should be dropped to terminate the source.
225+
pub token: Rc<Option<crate::source::SourceToken>>,
226+
/// Additional tokens that should be dropped to terminate the source.
227+
pub additional_tokens: Vec<Rc<dyn std::any::Any>>,
228+
}
229+
230+
/// Assemble the "storage" side of a dataflow, i.e. the sources.
231+
///
232+
/// This method creates a new dataflow to host the implementations of sources for the `dataflow`
233+
/// argument, and returns assets for each source that can import the results into a new dataflow.
234+
pub fn build_storage_dataflow<A: Allocate>(
210235
timely_worker: &mut TimelyWorker<A>,
211-
compute_state: &mut ComputeState,
212236
storage_state: &mut StorageState,
213-
dataflow: DataflowDescription<plan::Plan>,
237+
dataflow: &DataflowDescription<plan::Plan>,
214238
now: NowFn,
215239
source_metrics: &SourceBaseMetrics,
216-
sink_metrics: &SinkBaseMetrics,
217-
) {
240+
) -> BTreeMap<GlobalId, SourceBoundary> {
218241
let worker_logging = timely_worker.log_register().get("timely");
219242
let name = format!("Dataflow: {}", &dataflow.debug_name);
220243
let materialized_logging = timely_worker.log_register().get("materialized");
221244

245+
let mut results = BTreeMap::new();
246+
222247
timely_worker.dataflow_core(&name, worker_logging, Box::new(()), |_, scope| {
223248
// The scope.clone() occurs to allow import in the region.
224249
// We build a region here to establish a pattern of a scope inside the dataflow,
225250
// so that other similar uses (e.g. with iterative scopes) do not require weird
226251
// alternate type signatures.
227252
scope.clone().region_named(&name, |region| {
228-
let mut context = Context::for_dataflow(&dataflow, scope.addr().into_element());
229-
let mut tokens = RelevantTokens::default();
253+
let as_of = dataflow.as_of.clone().unwrap();
254+
let dataflow_id = scope.addr().into_element();
255+
let debug_name = format!("{}-sources", dataflow.debug_name);
230256

231257
assert!(
232258
!dataflow
@@ -241,11 +267,11 @@ pub fn build_dataflow<A: Allocate>(
241267

242268
// Import declared sources into the rendering context.
243269
for (src_id, source) in &dataflow.source_imports {
244-
let (collection_bundle, (source_token, additional_tokens)) =
270+
let ((ok, err), (source_token, additional_tokens)) =
245271
crate::render::sources::import_source(
246-
&context.debug_name,
247-
context.dataflow_id,
248-
&context.as_of_frontier,
272+
&debug_name,
273+
dataflow_id,
274+
&as_of,
249275
source.clone(),
250276
storage_state,
251277
region,
@@ -255,17 +281,89 @@ pub fn build_dataflow<A: Allocate>(
255281
source_metrics,
256282
);
257283

284+
let ok_activator = RcActivator::new(format!("{debug_name}-ok"), 1);
285+
let err_activator = RcActivator::new(format!("{debug_name}-err"), 1);
286+
287+
let ok_handle =
288+
ActivatedEventPusher::new(Rc::new(EventLink::new()), ok_activator.clone());
289+
let err_handle =
290+
ActivatedEventPusher::new(Rc::new(EventLink::new()), err_activator.clone());
291+
292+
results.insert(
293+
*src_id,
294+
SourceBoundary {
295+
ok: ActivatedEventPusher::<_>::clone(&ok_handle),
296+
err: ActivatedEventPusher::<_>::clone(&err_handle),
297+
token: source_token,
298+
additional_tokens,
299+
},
300+
);
301+
302+
ok.inner.capture_into(ok_handle);
303+
err.inner.capture_into(err_handle);
304+
}
305+
})
306+
});
307+
308+
results
309+
}
310+
311+
/// Assemble the "compute" side of a dataflow, i.e. all but the sources.
312+
///
313+
/// This method imports sources from provided assets, and then builds the remaining
314+
/// dataflow using "compute-local" assets like shared arrangements, and producing
315+
/// both arrangements and sinks.
316+
pub fn build_compute_dataflow<A: Allocate>(
317+
timely_worker: &mut TimelyWorker<A>,
318+
compute_state: &mut ComputeState,
319+
sources: BTreeMap<GlobalId, SourceBoundary>,
320+
dataflow: DataflowDescription<plan::Plan>,
321+
sink_metrics: &SinkBaseMetrics,
322+
) {
323+
let worker_logging = timely_worker.log_register().get("timely");
324+
let name = format!("Dataflow: {}", &dataflow.debug_name);
325+
326+
timely_worker.dataflow_core(&name, worker_logging, Box::new(()), |_, scope| {
327+
// The scope.clone() occurs to allow import in the region.
328+
// We build a region here to establish a pattern of a scope inside the dataflow,
329+
// so that other similar uses (e.g. with iterative scopes) do not require weird
330+
// alternate type signatures.
331+
scope.clone().region_named(&name, |region| {
332+
let mut context = Context::for_dataflow(&dataflow, scope.addr().into_element());
333+
let mut tokens = RelevantTokens::default();
334+
335+
// Import declared sources into the rendering context.
336+
for (source_id, source) in sources.into_iter() {
258337
// Associate collection bundle with the source identifier.
259-
context.insert_id(Id::Global(*src_id), collection_bundle);
338+
let ok = Some(source.ok.inner)
339+
.mz_replay(
340+
region,
341+
&format!("{name}-{source_id}"),
342+
Duration::MAX,
343+
source.ok.activator,
344+
)
345+
.as_collection();
346+
let err = Some(source.err.inner)
347+
.mz_replay(
348+
region,
349+
&format!("{name}-{source_id}-err"),
350+
Duration::MAX,
351+
source.err.activator,
352+
)
353+
.as_collection();
354+
context.insert_id(
355+
Id::Global(source_id),
356+
CollectionBundle::from_collections(ok, err),
357+
);
260358

261359
// Associate returned tokens with the source identifier.
262-
let prior = tokens.source_tokens.insert(*src_id, source_token);
360+
let prior = tokens.source_tokens.insert(source_id, source.token);
263361
assert!(prior.is_none());
264362
tokens
265363
.additional_tokens
266-
.entry(*src_id)
364+
.entry(source_id)
267365
.or_insert_with(Vec::new)
268-
.extend(additional_tokens);
366+
.extend(source.additional_tokens);
269367
}
270368

271369
// Import declared indexes into the rendering context.

src/dataflow/src/render/sources.rs

Lines changed: 9 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -69,7 +69,7 @@ pub(crate) fn import_table<G>(
6969
persisted_name: Option<String>,
7070
) -> (
7171
LocalInput,
72-
crate::render::CollectionBundle<G, Row, Timestamp>,
72+
(Collection<G, Row>, Collection<G, DataflowError>),
7373
)
7474
where
7575
G: Scope<Timestamp = Timestamp>,
@@ -124,10 +124,7 @@ where
124124
})
125125
.as_collection();
126126

127-
let collection_bundle =
128-
crate::render::CollectionBundle::from_collections(ok_collection, err_collection);
129-
130-
(local_input, collection_bundle)
127+
(local_input, (ok_collection, err_collection))
131128
}
132129

133130
/// Constructs a `CollectionBundle` and tokens from source arguments.
@@ -150,7 +147,7 @@ pub(crate) fn import_source<G>(
150147
now: NowFn,
151148
base_metrics: &SourceBaseMetrics,
152149
) -> (
153-
crate::render::CollectionBundle<G, Row, Timestamp>,
150+
(Collection<G, Row>, Collection<G, DataflowError>),
154151
(
155152
Rc<Option<crate::source::SourceToken>>,
156153
Vec<Rc<dyn std::any::Any>>,
@@ -180,12 +177,12 @@ where
180177
// Create a new local input (exposed as TABLEs to users). Data is inserted
181178
// via Command::Insert commands.
182179
SourceConnector::Local { persisted_name, .. } => {
183-
let (local_input, collection_bundle) =
180+
let (local_input, (ok, err)) =
184181
import_table(as_of_frontier, storage_state, scope, persisted_name);
185182
storage_state.local_inputs.insert(src_id, local_input);
186183

187184
// TODO(mcsherry): Local tables are a special non-source we should relocate.
188-
(collection_bundle, (Rc::new(None), Vec::new()))
185+
((ok, err), (Rc::new(None), Vec::new()))
189186
}
190187

191188
SourceConnector::External {
@@ -632,10 +629,6 @@ where
632629
use differential_dataflow::operators::consolidate::ConsolidateStream;
633630
collection = collection.consolidate_stream();
634631

635-
// Introduce the stream by name, as an unarranged collection.
636-
let collection_bundle =
637-
crate::render::CollectionBundle::from_collections(collection, err_collection);
638-
639632
let source_token = Rc::new(capability);
640633

641634
// We also need to keep track of this mapping globally to activate sources
@@ -647,7 +640,10 @@ where
647640
.push(Rc::downgrade(&source_token));
648641

649642
// Return the source token for capability manipulation, and any additional tokens.
650-
(collection_bundle, (source_token, additional_tokens))
643+
(
644+
(collection, err_collection),
645+
(source_token, additional_tokens),
646+
)
651647
}
652648
}
653649
}

src/dataflow/src/replay.rs

Lines changed: 16 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -43,12 +43,19 @@ where
4343
I: IntoIterator,
4444
<I as IntoIterator>::Item: EventIterator<T, D> + 'static,
4545
{
46+
/// Replay a collection of [EventIterator]s into a Timely stream.
47+
///
48+
/// * `scope`: The [Scope] to replay into.
49+
/// * `name`: Human-readable debug name of the Timely operator.
50+
/// * `period`: Reschedule the operator once the period has elapsed.
51+
/// Provide [Duration::MAX] to disable periodic scheduling.
52+
/// * `rc_activator`: An activator to trigger the operator.
4653
fn mz_replay<S: Scope<Timestamp = T>>(
4754
self,
4855
scope: &mut S,
4956
name: &str,
5057
period: Duration,
51-
mut rc_activator: RcActivator,
58+
rc_activator: RcActivator,
5259
) -> Stream<S, D> {
5360
let name = format!("Replay {}", name);
5461
let mut builder = OperatorBuilder::new(name, scope.clone());
@@ -68,9 +75,15 @@ where
6875

6976
builder.build(move |progress| {
7077
rc_activator.ack();
71-
if last_active + period <= Instant::now() || !started {
78+
if last_active
79+
.checked_add(period)
80+
.map_or(false, |next_active| next_active <= Instant::now())
81+
|| !started
82+
{
7283
last_active = Instant::now();
73-
activator.activate_after(period);
84+
if period < Duration::MAX {
85+
activator.activate_after(period);
86+
}
7487
}
7588

7689
if !started {

0 commit comments

Comments
 (0)