Skip to content

Commit

Permalink
Remove Container: Clone + 'static
Browse files Browse the repository at this point in the history
Remove the requirement that all Container implementations are Clone and
'static. This makes implementing types simpler that depend on Container,
and requires us to explicitly mark various places as `Data` such that they
comply with Timely's type requirements.

Signed-off-by: Moritz Hoffmann <[email protected]>
  • Loading branch information
antiguru committed Nov 29, 2023
1 parent 64be92b commit 95792c5
Show file tree
Hide file tree
Showing 22 changed files with 86 additions and 87 deletions.
4 changes: 2 additions & 2 deletions container/src/columnation.rs
Original file line number Diff line number Diff line change
Expand Up @@ -295,7 +295,7 @@ mod container {

use crate::columnation::{Columnation, TimelyStack};

impl<T: Columnation + 'static> Container for TimelyStack<T> {
impl<T: Columnation> Container for TimelyStack<T> {
type Item = T;

fn len(&self) -> usize {
Expand All @@ -315,7 +315,7 @@ mod container {
}
}

impl<T: Columnation + 'static> PushPartitioned for TimelyStack<T> {
impl<T: Columnation> PushPartitioned for TimelyStack<T> {
fn push_partitioned<I, F>(&mut self, buffers: &mut [Self], mut index: I, mut flush: F)
where
I: FnMut(&Self::Item) -> usize,
Expand Down
7 changes: 3 additions & 4 deletions container/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,7 @@ pub mod columnation;
/// We require the container to be cloneable to enable efficient copies when providing references
/// of containers to operators. Care must be taken that the type's `clone_from` implementation
/// is efficient (which is not necessarily the case when deriving `Clone`.)
/// TODO: Don't require `Container: Clone`
pub trait Container: Default + Clone + 'static {
pub trait Container: Default {
/// The type of elements this container holds.
type Item;

Expand All @@ -40,7 +39,7 @@ pub trait Container: Default + Clone + 'static {
fn clear(&mut self);
}

impl<T: Clone + 'static> Container for Vec<T> {
impl<T> Container for Vec<T> {
type Item = T;

fn len(&self) -> usize {
Expand Down Expand Up @@ -132,7 +131,7 @@ pub trait PushPartitioned: Container {
F: FnMut(usize, &mut Self);
}

impl<T: Clone + 'static> PushPartitioned for Vec<T> {
impl<T> PushPartitioned for Vec<T> {
fn push_partitioned<I, F>(&mut self, buffers: &mut [Self], mut index: I, mut flush: F)
where
I: FnMut(&Self::Item) -> usize,
Expand Down
4 changes: 2 additions & 2 deletions timely/examples/rc.rs
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ use abomonation::Abomonation;

#[derive(Debug, Clone)]
pub struct Test {
field: Rc<usize>,
_field: Rc<usize>,
}

impl Abomonation for Test {
Expand All @@ -32,7 +32,7 @@ fn main() {

// introduce data and watch!
for round in 0..10 {
input.send(Test { field: Rc::new(round) } );
input.send(Test { _field: Rc::new(round) } );
input.advance_to(round + 1);
worker.step_while(|| probe.less_than(input.time()));
}
Expand Down
14 changes: 7 additions & 7 deletions timely/src/dataflow/channels/pact.rs
Original file line number Diff line number Diff line change
Expand Up @@ -10,9 +10,9 @@
use std::{fmt::{self, Debug}, marker::PhantomData};
use timely_container::PushPartitioned;

use crate::communication::{Push, Pull, Data};
use crate::communication::{Push, Pull};
use crate::communication::allocator::thread::{ThreadPusher, ThreadPuller};
use crate::Container;
use crate::{Container, ExchangeData};

use crate::worker::AsWorker;
use crate::dataflow::channels::pushers::Exchange as ExchangePusher;
Expand All @@ -33,14 +33,14 @@ pub trait ParallelizationContractCore<T, D> {

/// A `ParallelizationContractCore` specialized for `Vec` containers
/// TODO: Use trait aliases once stable.
pub trait ParallelizationContract<T, D: Clone>: ParallelizationContractCore<T, Vec<D>> { }
impl<T, D: Clone, P: ParallelizationContractCore<T, Vec<D>>> ParallelizationContract<T, D> for P { }
pub trait ParallelizationContract<T, D>: ParallelizationContractCore<T, Vec<D>> { }
impl<T, D, P: ParallelizationContractCore<T, Vec<D>>> ParallelizationContract<T, D> for P { }

/// A direct connection
#[derive(Debug)]
pub struct Pipeline;

impl<T: 'static, D: Container> ParallelizationContractCore<T, D> for Pipeline {
impl<T: 'static, D: Container + 'static> ParallelizationContractCore<T, D> for Pipeline {
type Pusher = LogPusher<T, D, ThreadPusher<BundleCore<T, D>>>;
type Puller = LogPuller<T, D, ThreadPuller<BundleCore<T, D>>>;
fn connect<A: AsWorker>(self, allocator: &mut A, identifier: usize, address: &[usize], logging: Option<Logger>) -> (Self::Pusher, Self::Puller) {
Expand Down Expand Up @@ -69,9 +69,9 @@ impl<C, D, F: FnMut(&D)->u64+'static> ExchangeCore<C, D, F> {
}

// Exchange uses a `Box<Pushable>` because it cannot know what type of pushable will return from the allocator.
impl<T: Timestamp, C, D: Data+Clone, F: FnMut(&D)->u64+'static> ParallelizationContractCore<T, C> for ExchangeCore<C, D, F>
impl<T: Timestamp, C, D: ExchangeData, F: FnMut(&D)->u64+'static> ParallelizationContractCore<T, C> for ExchangeCore<C, D, F>
where
C: Data + Container + PushPartitioned<Item=D>,
C: ExchangeData + Container + PushPartitioned<Item=D>,
{
type Pusher = ExchangePusher<T, C, D, LogPusher<T, C, Box<dyn Push<BundleCore<T, C>>>>, F>;
type Puller = LogPuller<T, C, Box<dyn Pull<BundleCore<T, C>>>>;
Expand Down
2 changes: 1 addition & 1 deletion timely/src/dataflow/channels/pushers/exchange.rs
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ impl<T: Clone, C: Container, D: Data, P: Push<BundleCore<T, C>>, H: FnMut(&D) ->
}
}

impl<T: Eq+Data, C: Container, D: Data, P: Push<BundleCore<T, C>>, H: FnMut(&D) -> u64> Push<BundleCore<T, C>> for Exchange<T, C, D, P, H>
impl<T: Eq+Data, C: Container + Data, D: Data, P: Push<BundleCore<T, C>>, H: FnMut(&D) -> u64> Push<BundleCore<T, C>> for Exchange<T, C, D, P, H>
where
C: PushPartitioned<Item=D>
{
Expand Down
2 changes: 1 addition & 1 deletion timely/src/dataflow/channels/pushers/tee.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ pub struct TeeCore<T, D> {
/// [TeeCore] specialized to `Vec`-based container.
pub type Tee<T, D> = TeeCore<T, Vec<D>>;

impl<T: Data, D: Container> Push<BundleCore<T, D>> for TeeCore<T, D> {
impl<T: Data, D: Container + Data> Push<BundleCore<T, D>> for TeeCore<T, D> {
#[inline]
fn push(&mut self, message: &mut Option<BundleCore<T, D>>) {
let mut pushers = self.shared.borrow_mut();
Expand Down
2 changes: 1 addition & 1 deletion timely/src/dataflow/operators/branch.rs
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ pub trait BranchWhen<T>: Sized {
fn branch_when(&self, condition: impl Fn(&T) -> bool + 'static) -> (Self, Self);
}

impl<S: Scope, C: Container> BranchWhen<S::Timestamp> for StreamCore<S, C> {
impl<S: Scope, C: Container + Data> BranchWhen<S::Timestamp> for StreamCore<S, C> {
fn branch_when(&self, condition: impl Fn(&S::Timestamp) -> bool + 'static) -> (Self, Self) {
let mut builder = OperatorBuilder::new("Branch".to_owned(), self.scope());

Expand Down
8 changes: 4 additions & 4 deletions timely/src/dataflow/operators/capture/capture.rs
Original file line number Diff line number Diff line change
Expand Up @@ -10,14 +10,14 @@ use crate::dataflow::channels::pact::Pipeline;
use crate::dataflow::channels::pullers::Counter as PullCounter;
use crate::dataflow::operators::generic::builder_raw::OperatorBuilder;

use crate::Container;
use crate::{Container, Data};
use crate::progress::ChangeBatch;
use crate::progress::Timestamp;

use super::{EventCore, EventPusherCore};

/// Capture a stream of timestamped data for later replay.
pub trait Capture<T: Timestamp, D: Container> {
pub trait Capture<T: Timestamp, D: Container + Data> {
/// Captures a stream of timestamped data for later replay.
///
/// # Examples
Expand Down Expand Up @@ -113,7 +113,7 @@ pub trait Capture<T: Timestamp, D: Container> {
}
}

impl<S: Scope, D: Container> Capture<S::Timestamp, D> for StreamCore<S, D> {
impl<S: Scope, D: Container + Data> Capture<S::Timestamp, D> for StreamCore<S, D> {
fn capture_into<P: EventPusherCore<S::Timestamp, D>+'static>(&self, mut event_pusher: P) {

let mut builder = OperatorBuilder::new("Capture".to_owned(), self.scope());
Expand Down Expand Up @@ -142,7 +142,7 @@ impl<S: Scope, D: Container> Capture<S::Timestamp, D> for StreamCore<S, D> {
RefOrMut::Ref(reference) => (&reference.time, RefOrMut::Ref(&reference.data)),
RefOrMut::Mut(reference) => (&reference.time, RefOrMut::Mut(&mut reference.data)),
};
let vector = data.replace(Default::default());
let vector = data.take();
event_pusher.push(EventCore::Messages(time.clone(), vector));
}
input.consumed().borrow_mut().drain_into(&mut progress.consumeds[0]);
Expand Down
4 changes: 2 additions & 2 deletions timely/src/dataflow/operators/capture/replay.rs
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ use crate::progress::Timestamp;

use super::EventCore;
use super::event::EventIteratorCore;
use crate::Container;
use crate::{Container, Data};

/// Replay a capture stream into a scope with the same timestamp.
pub trait Replay<T: Timestamp, C> : Sized {
Expand All @@ -62,7 +62,7 @@ pub trait Replay<T: Timestamp, C> : Sized {
fn replay_core<S: Scope<Timestamp=T>>(self, scope: &mut S, period: Option<std::time::Duration>) -> StreamCore<S, C>;
}

impl<T: Timestamp, C: Container, I> Replay<T, C> for I
impl<T: Timestamp, C: Container + Data, I> Replay<T, C> for I
where I : IntoIterator,
<I as IntoIterator>::Item: EventIteratorCore<T, C>+'static {
fn replay_core<S: Scope<Timestamp=T>>(self, scope: &mut S, period: Option<std::time::Duration>) -> StreamCore<S, C>{
Expand Down
8 changes: 4 additions & 4 deletions timely/src/dataflow/operators/concat.rs
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
//! Merges the contents of multiple streams.


use crate::Container;
use crate::{Container, Data};
use crate::dataflow::channels::pact::Pipeline;
use crate::dataflow::{StreamCore, Scope};

Expand All @@ -23,7 +23,7 @@ pub trait Concat<G: Scope, D: Container> {
fn concat(&self, _: &StreamCore<G, D>) -> StreamCore<G, D>;
}

impl<G: Scope, D: Container> Concat<G, D> for StreamCore<G, D> {
impl<G: Scope, D: Container + Data> Concat<G, D> for StreamCore<G, D> {
fn concat(&self, other: &StreamCore<G, D>) -> StreamCore<G, D> {
self.scope().concatenate([self.clone(), other.clone()])
}
Expand Down Expand Up @@ -52,7 +52,7 @@ pub trait Concatenate<G: Scope, D: Container> {
I: IntoIterator<Item=StreamCore<G, D>>;
}

impl<G: Scope, D: Container> Concatenate<G, D> for StreamCore<G, D> {
impl<G: Scope, D: Container + Data> Concatenate<G, D> for StreamCore<G, D> {
fn concatenate<I>(&self, sources: I) -> StreamCore<G, D>
where
I: IntoIterator<Item=StreamCore<G, D>>
Expand All @@ -62,7 +62,7 @@ impl<G: Scope, D: Container> Concatenate<G, D> for StreamCore<G, D> {
}
}

impl<G: Scope, D: Container> Concatenate<G, D> for G {
impl<G: Scope, D: Container + Data> Concatenate<G, D> for G {
fn concatenate<I>(&self, sources: I) -> StreamCore<G, D>
where
I: IntoIterator<Item=StreamCore<G, D>>
Expand Down
6 changes: 3 additions & 3 deletions timely/src/dataflow/operators/enterleave.rs
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,7 @@ pub trait Leave<G: Scope, D: Container> {
fn leave(&self) -> StreamCore<G, D>;
}

impl<'a, G: Scope, D: Clone+Container, T: Timestamp+Refines<G::Timestamp>> Leave<G, D> for StreamCore<Child<'a, G, T>, D> {
impl<'a, G: Scope, D: Container + Data, T: Timestamp+Refines<G::Timestamp>> Leave<G, D> for StreamCore<Child<'a, G, T>, D> {
fn leave(&self) -> StreamCore<G, D> {

let scope = self.scope();
Expand All @@ -160,14 +160,14 @@ impl<'a, G: Scope, D: Clone+Container, T: Timestamp+Refines<G::Timestamp>> Leave
}


struct IngressNub<TOuter: Timestamp, TInner: Timestamp+Refines<TOuter>, TData: Container> {
struct IngressNub<TOuter: Timestamp, TInner: Timestamp+Refines<TOuter>, TData: Container + Data> {
targets: CounterCore<TInner, TData, TeeCore<TInner, TData>>,
phantom: ::std::marker::PhantomData<TOuter>,
activator: crate::scheduling::Activator,
active: bool,
}

impl<TOuter: Timestamp, TInner: Timestamp+Refines<TOuter>, TData: Container> Push<BundleCore<TOuter, TData>> for IngressNub<TOuter, TInner, TData> {
impl<TOuter: Timestamp, TInner: Timestamp+Refines<TOuter>, TData: Container + Data> Push<BundleCore<TOuter, TData>> for IngressNub<TOuter, TInner, TData> {
fn push(&mut self, element: &mut Option<BundleCore<TOuter, TData>>) {
if let Some(message) = element {
let outer_message = message.as_mut();
Expand Down
14 changes: 7 additions & 7 deletions timely/src/dataflow/operators/feedback.rs
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ pub trait Feedback<G: Scope> {
/// .connect_loop(handle);
/// });
/// ```
fn feedback_core<D: Container>(&mut self, summary: <G::Timestamp as Timestamp>::Summary) -> (HandleCore<G, D>, StreamCore<G, D>);
fn feedback_core<D: Container + Data>(&mut self, summary: <G::Timestamp as Timestamp>::Summary) -> (HandleCore<G, D>, StreamCore<G, D>);
}

/// Creates a `Stream` and a `Handle` to later bind the source of that `Stream`.
Expand Down Expand Up @@ -87,15 +87,15 @@ pub trait LoopVariable<'a, G: Scope, T: Timestamp> {
/// });
/// });
/// ```
fn loop_variable<D: Container>(&mut self, summary: T::Summary) -> (HandleCore<Iterative<'a, G, T>, D>, StreamCore<Iterative<'a, G, T>, D>);
fn loop_variable<D: Container + Data>(&mut self, summary: T::Summary) -> (HandleCore<Iterative<'a, G, T>, D>, StreamCore<Iterative<'a, G, T>, D>);
}

impl<G: Scope> Feedback<G> for G {
fn feedback<D: Data>(&mut self, summary: <G::Timestamp as Timestamp>::Summary) -> (Handle<G, D>, Stream<G, D>) {
self.feedback_core(summary)
}

fn feedback_core<D: Container>(&mut self, summary: <G::Timestamp as Timestamp>::Summary) -> (HandleCore<G, D>, StreamCore<G, D>) {
fn feedback_core<D: Container + Data>(&mut self, summary: <G::Timestamp as Timestamp>::Summary) -> (HandleCore<G, D>, StreamCore<G, D>) {

let mut builder = OperatorBuilder::new("Feedback".to_owned(), self.clone());
let (output, stream) = builder.new_output();
Expand All @@ -105,13 +105,13 @@ impl<G: Scope> Feedback<G> for G {
}

impl<'a, G: Scope, T: Timestamp> LoopVariable<'a, G, T> for Iterative<'a, G, T> {
fn loop_variable<D: Container>(&mut self, summary: T::Summary) -> (HandleCore<Iterative<'a, G, T>, D>, StreamCore<Iterative<'a, G, T>, D>) {
fn loop_variable<D: Container + Data>(&mut self, summary: T::Summary) -> (HandleCore<Iterative<'a, G, T>, D>, StreamCore<Iterative<'a, G, T>, D>) {
self.feedback_core(Product::new(Default::default(), summary))
}
}

/// Connect a `Stream` to the input of a loop variable.
pub trait ConnectLoop<G: Scope, D: Container> {
pub trait ConnectLoop<G: Scope, D: Container + Data> {
/// Connect a `Stream` to be the input of a loop variable.
///
/// # Examples
Expand All @@ -132,7 +132,7 @@ pub trait ConnectLoop<G: Scope, D: Container> {
fn connect_loop(&self, _: HandleCore<G, D>);
}

impl<G: Scope, D: Container> ConnectLoop<G, D> for StreamCore<G, D> {
impl<G: Scope, D: Container + Data> ConnectLoop<G, D> for StreamCore<G, D> {
fn connect_loop(&self, helper: HandleCore<G, D>) {

let mut builder = helper.builder;
Expand All @@ -159,7 +159,7 @@ impl<G: Scope, D: Container> ConnectLoop<G, D> for StreamCore<G, D> {

/// A handle used to bind the source of a loop variable.
#[derive(Debug)]
pub struct HandleCore<G: Scope, D: Container> {
pub struct HandleCore<G: Scope, D: Container + Data> {
builder: OperatorBuilder<G>,
summary: <G::Timestamp as Timestamp>::Summary,
output: OutputWrapper<G::Timestamp, D, TeeCore<G::Timestamp, D>>,
Expand Down
6 changes: 3 additions & 3 deletions timely/src/dataflow/operators/generic/builder_rc.rs
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ use crate::progress::{ChangeBatch, Timestamp};
use crate::progress::operate::SharedProgress;
use crate::progress::frontier::{Antichain, MutableAntichain};

use crate::Container;
use crate::{Container, Data};
use crate::dataflow::{Scope, StreamCore};
use crate::dataflow::channels::pushers::TeeCore;
use crate::dataflow::channels::pushers::CounterCore as PushCounter;
Expand Down Expand Up @@ -92,7 +92,7 @@ impl<G: Scope> OperatorBuilder<G> {
}

/// Adds a new output to a generic operator builder, returning the `Push` implementor to use.
pub fn new_output<D: Container>(&mut self) -> (OutputWrapper<G::Timestamp, D, TeeCore<G::Timestamp, D>>, StreamCore<G, D>) {
pub fn new_output<D: Container + Data>(&mut self) -> (OutputWrapper<G::Timestamp, D, TeeCore<G::Timestamp, D>>, StreamCore<G, D>) {
let connection = vec![Antichain::from_elem(Default::default()); self.builder.shape().inputs()];
self.new_output_connection(connection)
}
Expand All @@ -105,7 +105,7 @@ impl<G: Scope> OperatorBuilder<G> {
///
/// Commonly the connections are either the unit summary, indicating the same timestamp might be produced as output, or an empty
/// antichain indicating that there is no connection from the input to the output.
pub fn new_output_connection<D: Container>(&mut self, connection: Vec<Antichain<<G::Timestamp as Timestamp>::Summary>>) -> (OutputWrapper<G::Timestamp, D, TeeCore<G::Timestamp, D>>, StreamCore<G, D>) {
pub fn new_output_connection<D: Container + Data>(&mut self, connection: Vec<Antichain<<G::Timestamp as Timestamp>::Summary>>) -> (OutputWrapper<G::Timestamp, D, TeeCore<G::Timestamp, D>>, StreamCore<G, D>) {

let (tee, stream) = self.builder.new_output_connection(connection.clone());

Expand Down
Loading

0 comments on commit 95792c5

Please sign in to comment.