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 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860 861 862 863 864 865 866 867
// 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.
//! Types to build async operators with general shapes.
use std::cell::{Cell, RefCell};
use std::collections::VecDeque;
use std::future::Future;
use std::pin::Pin;
use std::rc::Rc;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::Arc;
use std::task::{ready, Context, Poll, Waker};
use futures_util::task::ArcWake;
use futures_util::Stream;
use timely::communication::{Pull, Push};
use timely::container::columnation::Columnation;
use timely::container::{CapacityContainerBuilder, ContainerBuilder, PushInto};
use timely::dataflow::channels::pact::ParallelizationContract;
use timely::dataflow::channels::pushers::Tee;
use timely::dataflow::channels::Message;
use timely::dataflow::operators::generic::builder_rc::OperatorBuilder as OperatorBuilderRc;
use timely::dataflow::operators::generic::{
InputHandleCore, OperatorInfo, OutputHandleCore, OutputWrapper,
};
use timely::dataflow::operators::{Capability, CapabilitySet, InputCapability};
use timely::dataflow::{Scope, StreamCore};
use timely::progress::{Antichain, Timestamp};
use timely::scheduling::{Activator, SyncActivator};
use timely::{Bincode, Container, PartialOrder};
use crate::containers::stack::{AccountedStackBuilder, StackWrapper};
/// Builds async operators with generic shape.
pub struct OperatorBuilder<G: Scope> {
builder: OperatorBuilderRc<G>,
/// The activator for this operator
activator: Activator,
/// The waker set up to activate this timely operator when woken
operator_waker: Arc<TimelyWaker>,
/// The currently known upper frontier of each of the input handles.
input_frontiers: Vec<Antichain<G::Timestamp>>,
/// Input queues for each of the declared inputs of the operator.
input_queues: Vec<Box<dyn InputQueue<G::Timestamp>>>,
/// Holds type erased closures that flush an output handle when called. These handles will be
/// automatically drained when the operator is scheduled after the logic future has been polled
output_flushes: Vec<Box<dyn FnMut()>>,
/// A handle to check whether all workers have pressed the shutdown button.
shutdown_handle: ButtonHandle,
/// A button to coordinate shutdown of this operator among workers.
shutdown_button: Button,
}
/// A helper trait abstracting over an input handle. It facilitates keeping around type erased
/// handles for each of the operator inputs.
trait InputQueue<T: Timestamp> {
/// Accepts all available input into local queues.
fn accept_input(&mut self);
/// Drains all available input and empties the local queue.
fn drain_input(&mut self);
/// Registers a frontier notification to be delivered.
fn notify_progress(&mut self, upper: Antichain<T>);
}
impl<T, D, C, P> InputQueue<T> for InputHandleQueue<T, D, C, P>
where
T: Timestamp,
D: Container,
C: InputConnection<T> + 'static,
P: Pull<Message<T, D>> + 'static,
{
fn accept_input(&mut self) {
let mut queue = self.queue.borrow_mut();
let mut new_data = false;
while let Some((cap, data)) = self.handle.next() {
new_data = true;
let cap = self.connection.accept(cap);
queue.push_back(Event::Data(cap, std::mem::take(data)));
}
if new_data {
if let Some(waker) = self.waker.take() {
waker.wake();
}
}
}
fn drain_input(&mut self) {
self.queue.borrow_mut().clear();
self.handle.for_each(|_, _| {});
}
fn notify_progress(&mut self, upper: Antichain<T>) {
let mut queue = self.queue.borrow_mut();
// It's beneficial to consolidate two consecutive progress statements into one if the
// operator hasn't seen the previous progress yet. This also avoids accumulation of
// progress statements in the queue if the operator only conditionally checks this input.
match queue.back_mut() {
Some(&mut Event::Progress(ref mut prev_upper)) => *prev_upper = upper,
_ => queue.push_back(Event::Progress(upper)),
}
if let Some(waker) = self.waker.take() {
waker.wake();
}
}
}
struct InputHandleQueue<
T: Timestamp,
D: Container,
C: InputConnection<T>,
P: Pull<Message<T, D>> + 'static,
> {
queue: Rc<RefCell<VecDeque<Event<T, C::Capability, D>>>>,
waker: Rc<Cell<Option<Waker>>>,
connection: C,
handle: InputHandleCore<T, D, P>,
}
/// An async Waker that activates a specific operator when woken and marks the task as ready
struct TimelyWaker {
activator: SyncActivator,
active: AtomicBool,
task_ready: AtomicBool,
}
impl ArcWake for TimelyWaker {
fn wake_by_ref(arc_self: &Arc<Self>) {
arc_self.task_ready.store(true, Ordering::SeqCst);
// Only activate the timely operator if it's not already active to avoid an infinite loop
if !arc_self.active.load(Ordering::SeqCst) {
// We don't have any guarantees about how long the Waker will be held for and so we
// must be prepared for the receiving end to have hung up when we finally do get woken.
// This can happen if by the time the waker is called the receiving timely worker has
// been shutdown. For this reason we ignore the activation error.
let _ = arc_self.activator.activate();
}
}
}
/// Async handle to an operator's input stream
pub struct AsyncInputHandle<T: Timestamp, D: Container, C: InputConnection<T>> {
queue: Rc<RefCell<VecDeque<Event<T, C::Capability, D>>>>,
waker: Rc<Cell<Option<Waker>>>,
/// Whether this handle has finished producing data
done: bool,
}
impl<T: Timestamp, D: Container, C: InputConnection<T>> AsyncInputHandle<T, D, C> {
pub fn next_sync(&mut self) -> Option<Event<T, C::Capability, D>> {
let mut queue = self.queue.borrow_mut();
match queue.pop_front()? {
Event::Data(cap, data) => Some(Event::Data(cap, data)),
Event::Progress(frontier) => {
self.done = frontier.is_empty();
Some(Event::Progress(frontier))
}
}
}
/// Waits for the handle to have data. After this function returns it is guaranteed that at
/// least one call to `next_sync` will be `Some(_)`.
pub async fn ready(&self) {
std::future::poll_fn(|cx| self.poll_ready(cx)).await
}
fn poll_ready(&self, cx: &Context<'_>) -> Poll<()> {
if self.queue.borrow().is_empty() {
self.waker.set(Some(cx.waker().clone()));
Poll::Pending
} else {
Poll::Ready(())
}
}
}
impl<T: Timestamp, D: Container, C: InputConnection<T>> Stream for AsyncInputHandle<T, D, C> {
type Item = Event<T, C::Capability, D>;
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
if self.done {
return Poll::Ready(None);
}
ready!(self.poll_ready(cx));
Poll::Ready(self.next_sync())
}
fn size_hint(&self) -> (usize, Option<usize>) {
(self.queue.borrow().len(), None)
}
}
/// An event of an input stream
#[derive(Debug)]
pub enum Event<T: Timestamp, C, D> {
/// A data event
Data(C, D),
/// A progress event
Progress(Antichain<T>),
}
pub struct AsyncOutputHandle<
T: Timestamp,
CB: ContainerBuilder,
P: Push<Message<T, CB::Container>> + 'static,
> {
// The field order is important here as the handle is borrowing from the wrapper. See also the
// safety argument in the constructor
handle: Rc<RefCell<OutputHandleCore<'static, T, CB, P>>>,
wrapper: Rc<Pin<Box<OutputWrapper<T, CB, P>>>>,
index: usize,
}
impl<T, C, P> AsyncOutputHandle<T, CapacityContainerBuilder<C>, P>
where
T: Timestamp,
C: Container + Clone + 'static,
P: Push<Message<T, C>> + 'static,
{
#[inline]
pub fn give_container(&self, cap: &Capability<T>, container: &mut C) {
let mut handle = self.handle.borrow_mut();
handle.session_with_builder(cap).give_container(container);
}
}
impl<T, CB, P> AsyncOutputHandle<T, CB, P>
where
T: Timestamp,
CB: ContainerBuilder,
P: Push<Message<T, CB::Container>> + 'static,
{
fn new(wrapper: OutputWrapper<T, CB, P>, index: usize) -> Self {
let mut wrapper = Rc::new(Box::pin(wrapper));
// SAFETY:
// get_unchecked_mut is safe because we are not moving the wrapper
//
// transmute is safe because:
// * We're erasing the lifetime but we guarantee through field order that the handle will
// be dropped before the wrapper, thus manually enforcing the lifetime.
// * We never touch wrapper again after this point
let handle = unsafe {
let handle = Rc::get_mut(&mut wrapper)
.unwrap()
.as_mut()
.get_unchecked_mut()
.activate();
std::mem::transmute::<OutputHandleCore<'_, T, CB, P>, OutputHandleCore<'static, T, CB, P>>(
handle,
)
};
Self {
wrapper,
handle: Rc::new(RefCell::new(handle)),
index,
}
}
fn cease(&self) {
self.handle.borrow_mut().cease()
}
}
impl<T, C, P> AsyncOutputHandle<T, CapacityContainerBuilder<C>, P>
where
T: Timestamp,
C: Container + Clone + 'static,
P: Push<Message<T, C>> + 'static,
{
pub fn give<D>(&self, cap: &Capability<T>, data: D)
where
CapacityContainerBuilder<C>: PushInto<D>,
{
let mut handle = self.handle.borrow_mut();
handle.session_with_builder(cap).give(data);
}
}
impl<T, D, P>
AsyncOutputHandle<T, AccountedStackBuilder<CapacityContainerBuilder<StackWrapper<D>>>, P>
where
D: timely::Data + Columnation,
T: Timestamp,
P: Push<Message<T, StackWrapper<D>>>,
{
pub const MAX_OUTSTANDING_BYTES: usize = 128 * 1024 * 1024;
/// Provides one record at the time specified by the capability. This method will automatically
/// yield back to timely after [Self::MAX_OUTSTANDING_BYTES] have been produced.
pub async fn give_fueled<D2>(&self, cap: &Capability<T>, data: D2)
where
StackWrapper<D>: PushInto<D2>,
{
let should_yield = {
let mut handle = self.handle.borrow_mut();
let mut session = handle.session_with_builder(cap);
session.push_into(data);
let should_yield = session.builder().bytes.get() > Self::MAX_OUTSTANDING_BYTES;
if should_yield {
session.builder().bytes.set(0);
}
should_yield
};
if should_yield {
tokio::task::yield_now().await;
}
}
}
impl<T: Timestamp, CB: ContainerBuilder, P: Push<Message<T, CB::Container>> + 'static> Clone
for AsyncOutputHandle<T, CB, P>
{
fn clone(&self) -> Self {
Self {
handle: Rc::clone(&self.handle),
wrapper: Rc::clone(&self.wrapper),
index: self.index,
}
}
}
/// A trait describing the connection behavior between an input of an operator and zero or more of
/// its outputs.
pub trait InputConnection<T: Timestamp> {
/// The capability type associated with this connection behavior.
type Capability;
/// Generates a summary description of the connection behavior given the number of outputs.
fn describe(&self, outputs: usize) -> Vec<Antichain<T::Summary>>;
/// Accepts an input capability.
fn accept(&self, input_cap: InputCapability<T>) -> Self::Capability;
}
/// A marker type representing a disconnected input.
pub struct Disconnected;
impl<T: Timestamp> InputConnection<T> for Disconnected {
type Capability = T;
fn describe(&self, outputs: usize) -> Vec<Antichain<T::Summary>> {
vec![Antichain::new(); outputs]
}
fn accept(&self, input_cap: InputCapability<T>) -> Self::Capability {
input_cap.time().clone()
}
}
/// A marker type representing an input connected to exactly one output.
pub struct ConnectedToOne(usize);
impl<T: Timestamp> InputConnection<T> for ConnectedToOne {
type Capability = Capability<T>;
fn describe(&self, outputs: usize) -> Vec<Antichain<T::Summary>> {
let mut summary = vec![Antichain::new(); outputs];
summary[self.0] = Antichain::from_elem(T::Summary::default());
summary
}
fn accept(&self, input_cap: InputCapability<T>) -> Self::Capability {
input_cap.retain_for_output(self.0)
}
}
/// A marker type representing an input connected to many outputs.
pub struct ConnectedToMany<const N: usize>([usize; N]);
impl<const N: usize, T: Timestamp> InputConnection<T> for ConnectedToMany<N> {
type Capability = [Capability<T>; N];
fn describe(&self, outputs: usize) -> Vec<Antichain<T::Summary>> {
let mut summary = vec![Antichain::new(); outputs];
for output in self.0 {
summary[output] = Antichain::from_elem(T::Summary::default());
}
summary
}
fn accept(&self, input_cap: InputCapability<T>) -> Self::Capability {
self.0
.map(|output| input_cap.delayed_for_output(input_cap.time(), output))
}
}
/// A helper trait abstracting over an output handle. It facilitates passing type erased
/// output handles during operator construction.
/// It is not meant to be implemented by users.
pub trait OutputIndex {
/// The output index of this handle.
fn index(&self) -> usize;
}
impl<T: Timestamp, CB: ContainerBuilder> OutputIndex
for AsyncOutputHandle<T, CB, Tee<T, CB::Container>>
{
fn index(&self) -> usize {
self.index
}
}
impl<G: Scope> OperatorBuilder<G> {
/// Allocates a new generic async operator builder from its containing scope.
pub fn new(name: String, mut scope: G) -> Self {
let builder = OperatorBuilderRc::new(name, scope.clone());
let info = builder.operator_info();
let activator = scope.activator_for(Rc::clone(&info.address));
let sync_activator = scope.sync_activator_for(info.address.to_vec());
let operator_waker = TimelyWaker {
activator: sync_activator,
active: AtomicBool::new(false),
task_ready: AtomicBool::new(true),
};
let (shutdown_handle, shutdown_button) = button(&mut scope, info.address);
OperatorBuilder {
builder,
activator,
operator_waker: Arc::new(operator_waker),
input_frontiers: Default::default(),
input_queues: Default::default(),
output_flushes: Default::default(),
shutdown_handle,
shutdown_button,
}
}
/// Adds a new input that is connected to the specified output, returning the async input handle to use.
pub fn new_input_for<D, P>(
&mut self,
stream: &StreamCore<G, D>,
pact: P,
output: &dyn OutputIndex,
) -> AsyncInputHandle<G::Timestamp, D, ConnectedToOne>
where
D: Container + 'static,
P: ParallelizationContract<G::Timestamp, D>,
{
let index = output.index();
assert!(index < self.builder.shape().outputs());
self.new_input_connection(stream, pact, ConnectedToOne(index))
}
/// Adds a new input that is connected to the specified outputs, returning the async input handle to use.
pub fn new_input_for_many<const N: usize, D, P>(
&mut self,
stream: &StreamCore<G, D>,
pact: P,
outputs: [&dyn OutputIndex; N],
) -> AsyncInputHandle<G::Timestamp, D, ConnectedToMany<N>>
where
D: Container + 'static,
P: ParallelizationContract<G::Timestamp, D>,
{
let indices = outputs.map(|output| output.index());
for index in indices {
assert!(index < self.builder.shape().outputs());
}
self.new_input_connection(stream, pact, ConnectedToMany(indices))
}
/// Adds a new input that is not connected to any output, returning the async input handle to use.
pub fn new_disconnected_input<D, P>(
&mut self,
stream: &StreamCore<G, D>,
pact: P,
) -> AsyncInputHandle<G::Timestamp, D, Disconnected>
where
D: Container + 'static,
P: ParallelizationContract<G::Timestamp, D>,
{
self.new_input_connection(stream, pact, Disconnected)
}
/// Adds a new input with connection information, returning the async input handle to use.
pub fn new_input_connection<D, P, C>(
&mut self,
stream: &StreamCore<G, D>,
pact: P,
connection: C,
) -> AsyncInputHandle<G::Timestamp, D, C>
where
D: Container + 'static,
P: ParallelizationContract<G::Timestamp, D>,
C: InputConnection<G::Timestamp> + 'static,
{
self.input_frontiers
.push(Antichain::from_elem(G::Timestamp::minimum()));
let outputs = self.builder.shape().outputs();
let handle = self
.builder
.new_input_connection(stream, pact, connection.describe(outputs));
let waker = Default::default();
let queue = Default::default();
let input_queue = InputHandleQueue {
queue: Rc::clone(&queue),
waker: Rc::clone(&waker),
connection,
handle,
};
self.input_queues.push(Box::new(input_queue));
AsyncInputHandle {
queue,
waker,
done: false,
}
}
/// Adds a new output, returning the output handle and stream.
pub fn new_output<CB: ContainerBuilder>(
&mut self,
) -> (
AsyncOutputHandle<G::Timestamp, CB, Tee<G::Timestamp, CB::Container>>,
StreamCore<G, CB::Container>,
) {
let index = self.builder.shape().outputs();
let connection = vec![Antichain::new(); self.builder.shape().inputs()];
let (wrapper, stream) = self.builder.new_output_connection(connection);
let handle = AsyncOutputHandle::new(wrapper, index);
let flush_handle = handle.clone();
self.output_flushes
.push(Box::new(move || flush_handle.cease()));
(handle, stream)
}
/// Creates an operator implementation from supplied logic constructor. It returns a shutdown
/// button that when pressed it will cause the logic future to be dropped and input handles to
/// be drained. The button can be converted into a token by using
/// [`Button::press_on_drop`]
pub fn build<B, L>(self, constructor: B) -> Button
where
B: FnOnce(Vec<Capability<G::Timestamp>>) -> L,
L: Future + 'static,
{
let operator_waker = self.operator_waker;
let mut input_frontiers = self.input_frontiers;
let mut input_queues = self.input_queues;
let mut output_flushes = self.output_flushes;
let mut shutdown_handle = self.shutdown_handle;
self.builder.build_reschedule(move |caps| {
let mut logic_fut = Some(Box::pin(constructor(caps)));
move |new_frontiers| {
operator_waker.active.store(true, Ordering::SeqCst);
for (i, queue) in input_queues.iter_mut().enumerate() {
// First, discover if there are any frontier notifications
let cur = &mut input_frontiers[i];
let new = new_frontiers[i].frontier();
if PartialOrder::less_than(&cur.borrow(), &new) {
queue.notify_progress(new.to_owned());
*cur = new.to_owned();
}
// Then accept all input into local queues. This step registers the received
// messages with progress tracking.
queue.accept_input();
}
operator_waker.active.store(false, Ordering::SeqCst);
// If our worker pressed the button we stop scheduling the logic future and/or
// draining the input handles to stop producing data and frontier updates
// downstream.
if shutdown_handle.local_pressed() {
// When all workers press their buttons we drop the logic future and start
// draining the input handles.
if shutdown_handle.all_pressed() {
logic_fut = None;
for queue in input_queues.iter_mut() {
queue.drain_input();
}
false
} else {
true
}
} else {
// Schedule the logic future if any of the wakers above marked the task as ready
if let Some(fut) = logic_fut.as_mut() {
if operator_waker.task_ready.load(Ordering::SeqCst) {
let waker = futures_util::task::waker_ref(&operator_waker);
let mut cx = Context::from_waker(&waker);
operator_waker.task_ready.store(false, Ordering::SeqCst);
if Pin::new(fut).poll(&mut cx).is_ready() {
// We're done with logic so deallocate the task
logic_fut = None;
}
// Flush all the outputs before exiting
for flush in output_flushes.iter_mut() {
(flush)();
}
}
}
// The timely operator needs to be kept alive if the task is pending
if logic_fut.is_some() {
true
} else {
// Othewise we should keep draining all inputs
for queue in input_queues.iter_mut() {
queue.drain_input();
}
false
}
}
}
});
self.shutdown_button
}
/// Creates a fallible operator implementation from supplied logic constructor. If the `Future`
/// resolves to an error it will be emitted in the returned error stream and then the operator
/// will wait indefinitely until the shutdown button is pressed.
///
/// # Capability handling
///
/// Unlike [`OperatorBuilder::build`], this method does not give owned capabilities to the
/// constructor. All initial capabilities are wrapped in a `CapabilitySet` and a mutable
/// reference to them is given instead. This is done to avoid storing owned capabilities in the
/// state of the logic future which would make using the `?` operator unsafe, since the
/// frontiers would incorrectly advance, potentially causing incorrect actions downstream.
///
/// ```ignore
/// builder.build_fallible(|caps| Box::pin(async move {
/// // Assert that we have the number of capabilities we expect
/// // `cap` will be a `&mut Option<Capability<T>>`:
/// let [cap_set]: &mut [_; 1] = caps.try_into().unwrap();
///
/// // Using cap to send data:
/// output.give(&cap_set[0], 42);
///
/// // Using cap_set to downgrade it:
/// cap_set.downgrade([]);
///
/// // Explicitly dropping the capability:
/// // Simply running `drop(cap_set)` will only drop the reference and not the capability set itself!
/// *cap_set = CapabilitySet::new();
///
/// // !! BIG WARNING !!:
/// // It is tempting to `take` the capability out of the set for convenience. This will
/// // move the capability into the future state, tying its lifetime to it, which will get
/// // dropped when an error is hit, causing incorrect progress statements.
/// let cap = cap_set.delayed(&Timestamp::minimum());
/// *cap_set = CapabilitySet::new(); // DO NOT DO THIS
/// }));
/// ```
pub fn build_fallible<E: 'static, F>(
mut self,
constructor: F,
) -> (Button, StreamCore<G, Vec<Rc<E>>>)
where
F: for<'a> FnOnce(
&'a mut [CapabilitySet<G::Timestamp>],
) -> Pin<Box<dyn Future<Output = Result<(), E>> + 'a>>
+ 'static,
{
// Create a new completely disconnected output
let (error_output, error_stream) = self.new_output();
let button = self.build(|mut caps| async move {
let error_cap = caps.pop().unwrap();
let mut caps = caps
.into_iter()
.map(CapabilitySet::from_elem)
.collect::<Vec<_>>();
if let Err(err) = constructor(&mut *caps).await {
error_output.give(&error_cap, Rc::new(err));
drop(error_cap);
// IMPORTANT: wedge this operator until the button is pressed. Returning would drop
// the capabilities and could produce incorrect progress statements.
std::future::pending().await
}
});
(button, error_stream)
}
/// Creates operator info for the operator.
pub fn operator_info(&self) -> OperatorInfo {
self.builder.operator_info()
}
/// Returns the activator for the operator.
pub fn activator(&self) -> &Activator {
&self.activator
}
}
/// Creates a new coordinated button the worker configuration described by `scope`.
pub fn button<G: Scope>(scope: &mut G, addr: Rc<[usize]>) -> (ButtonHandle, Button) {
let index = scope.new_identifier();
let (pushers, puller) = scope.allocate(index, addr);
let local_pressed = Rc::new(Cell::new(false));
let handle = ButtonHandle {
buttons_remaining: scope.peers(),
local_pressed: Rc::clone(&local_pressed),
puller,
};
let token = Button {
pushers,
local_pressed,
};
(handle, token)
}
/// A button that can be used to coordinate an action after all workers have pressed it.
pub struct ButtonHandle {
/// The number of buttons still unpressed among workers.
buttons_remaining: usize,
/// A flag indicating whether this worker has pressed its button.
local_pressed: Rc<Cell<bool>>,
puller: Box<dyn Pull<Bincode<bool>>>,
}
impl ButtonHandle {
/// Returns whether this worker has pressed its button.
pub fn local_pressed(&self) -> bool {
self.local_pressed.get()
}
/// Returns whether all workers have pressed their buttons.
pub fn all_pressed(&mut self) -> bool {
while self.puller.recv().is_some() {
self.buttons_remaining -= 1;
}
self.buttons_remaining == 0
}
}
pub struct Button {
pushers: Vec<Box<dyn Push<Bincode<bool>>>>,
local_pressed: Rc<Cell<bool>>,
}
impl Button {
/// Presses the button. It is safe to call this function multiple times.
pub fn press(&mut self) {
for mut pusher in self.pushers.drain(..) {
pusher.send(Bincode::from(true));
pusher.done();
}
self.local_pressed.set(true);
}
/// Converts this button into a deadman's switch that will automatically press the button when
/// dropped.
pub fn press_on_drop(self) -> PressOnDropButton {
PressOnDropButton(self)
}
}
pub struct PressOnDropButton(Button);
impl Drop for PressOnDropButton {
fn drop(&mut self) {
self.0.press();
}
}
#[cfg(test)]
mod test {
use futures_util::StreamExt;
use timely::dataflow::channels::pact::Pipeline;
use timely::dataflow::operators::capture::Extract;
use timely::dataflow::operators::{Capture, ToStream};
use timely::WorkerConfig;
use super::*;
#[mz_ore::test]
fn async_operator() {
let capture = timely::example(|scope| {
let input = (0..10).to_stream(scope);
let mut op = OperatorBuilder::new("async_passthru".to_string(), input.scope());
let (output, output_stream) = op.new_output();
let mut input_handle = op.new_input_for(&input, Pipeline, &output);
op.build(move |_capabilities| async move {
tokio::task::yield_now().await;
while let Some(event) = input_handle.next().await {
match event {
Event::Data(cap, data) => {
for item in data.iter().copied() {
tokio::task::yield_now().await;
output.give(&cap, item);
}
}
Event::Progress(_frontier) => {}
}
}
});
output_stream.capture()
});
let extracted = capture.extract();
assert_eq!(extracted, vec![(0, vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9])]);
}
#[mz_ore::test]
fn gh_18837() {
let (builders, other) = timely::CommunicationConfig::Process(2).try_build().unwrap();
timely::execute::execute_from(builders, other, WorkerConfig::default(), |worker| {
let index = worker.index();
let tokens = worker.dataflow::<u64, _, _>(move |scope| {
let mut producer = OperatorBuilder::new("producer".to_string(), scope.clone());
let (_output, output_stream) =
producer.new_output::<CapacityContainerBuilder<Vec<usize>>>();
let producer_button = producer.build(move |mut capabilities| async move {
let mut cap = capabilities.pop().unwrap();
if index != 0 {
return;
}
// Worker 0 downgrades to 1 and keeps the capability around forever
cap.downgrade(&1);
std::future::pending().await
});
let mut consumer = OperatorBuilder::new("consumer".to_string(), scope.clone());
let mut input_handle = consumer.new_disconnected_input(&output_stream, Pipeline);
let consumer_button = consumer.build(move |_| async move {
while let Some(event) = input_handle.next().await {
if let Event::Progress(frontier) = event {
// We should never observe a frontier greater than [1]
assert!(frontier.less_equal(&1));
}
}
});
(
producer_button.press_on_drop(),
consumer_button.press_on_drop(),
)
});
// Run dataflow until only worker 0 holds the frontier to [1]
for _ in 0..100 {
worker.step();
}
// Then drop the tokens of worker 0
if index == 0 {
drop(tokens)
}
// And step the dataflow some more to ensure consumers don't observe frontiers advancing.
for _ in 0..100 {
worker.step();
}
})
.expect("timely panicked");
}
}