diff --git a/Cargo.toml b/Cargo.toml index d9d402c79..895c0407e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -23,9 +23,9 @@ rust-version = "1.86" [workspace.dependencies] differential-dataflow = { path = "differential-dataflow", default-features = false, version = "0.22.0" } -timely = { version = "0.28", default-features = false } +#timely = { version = "0.28", default-features = false } columnar = { version = "0.12", default-features = false } -#timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" } +timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" } #timely = { path = "../timely-dataflow/timely/", default-features = false } [workspace.lints.clippy] diff --git a/diagnostics/src/logging.rs b/diagnostics/src/logging.rs index b47721e62..f0fac8d2b 100644 --- a/diagnostics/src/logging.rs +++ b/diagnostics/src/logging.rs @@ -34,7 +34,6 @@ use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::trace::implementations::{KeySpine, ValSpine}; use differential_dataflow::{AsCollection, VecCollection}; -use timely::communication::Allocate; use timely::container::CapacityContainerBuilder; use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::capture::{Event, EventLink, Replay, Capture}; @@ -244,12 +243,11 @@ fn quantize(time: Duration, interval: Duration) -> Duration { } /// Quantize timestamps in a collection's inner stream. -fn quantize_collection( - collection: VecCollection, +fn quantize_collection( + collection: VecCollection, interval: Duration, -) -> VecCollection +) -> VecCollection where - S: Scope, D: differential_dataflow::Data, { collection @@ -276,7 +274,7 @@ where /// /// Returns a [`LoggingState`] with trace handles and a [`SinkHandle`] for /// the WebSocket thread. -pub fn register(worker: &mut Worker, log_logging: bool) -> LoggingState { +pub fn register(worker: &mut Worker, log_logging: bool) -> LoggingState { let start = Instant::now(); // Event links for logging capture (worker-internal, Rc-based). @@ -377,8 +375,8 @@ pub fn register(worker: &mut Worker, log_logging: bool) -> Loggi } } -fn install_loggers( - worker: &mut Worker, +fn install_loggers( + worker: &mut Worker, t_link: Rc>>, d_link: Rc>>, ) { @@ -402,11 +400,11 @@ fn install_loggers( // ============================================================================ /// Internal: collections before arrangement, used for the cross-join. -struct TimelyCollections { - operators: VecCollection), i64>, - channels: VecCollection, (usize, usize), (usize, usize)), i64>, - elapsed: VecCollection, - messages: VecCollection, +struct TimelyCollections { + operators: VecCollection), i64>, + channels: VecCollection, (usize, usize), (usize, usize)), i64>, + elapsed: VecCollection, + messages: VecCollection, } #[derive(Default)] @@ -416,10 +414,10 @@ struct TimelyDemuxState { } /// Build timely logging collections and arrangements. -fn construct_timely>( - scope: &mut S, - stream: Stream>, -) -> (TimelyTraces, TimelyCollections) { +fn construct_timely( + scope: &mut Scope, + stream: Stream>, +) -> (TimelyTraces, TimelyCollections) { type OpUpdate = ((usize, String, Vec), Duration, i64); type ChUpdate = ((usize, Vec, (usize, usize), (usize, usize)), Duration, i64); type ElUpdate = (usize, Duration, i64); @@ -536,21 +534,21 @@ fn construct_timely>( // ============================================================================ /// Internal: collections before arrangement, used for the cross-join. -struct DifferentialCollections { - arrangement_batches: VecCollection, - arrangement_records: VecCollection, - sharing: VecCollection, - batcher_records: VecCollection, - batcher_size: VecCollection, - batcher_capacity: VecCollection, - batcher_allocations: VecCollection, +struct DifferentialCollections { + arrangement_batches: VecCollection, + arrangement_records: VecCollection, + sharing: VecCollection, + batcher_records: VecCollection, + batcher_size: VecCollection, + batcher_capacity: VecCollection, + batcher_allocations: VecCollection, } /// Build differential logging collections and arrangements. -fn construct_differential>( - scope: &mut S, - stream: Stream>, -) -> (DifferentialTraces, DifferentialCollections) { +fn construct_differential( + scope: &mut Scope, + stream: Stream>, +) -> (DifferentialTraces, DifferentialCollections) { type Update = (usize, Duration, i64); let mut demux = OperatorBuilder::new("Differential Demux".to_string(), scope.clone()); diff --git a/differential-dataflow/examples/accumulate.rs b/differential-dataflow/examples/accumulate.rs index 0fa792926..d8da7bdcd 100644 --- a/differential-dataflow/examples/accumulate.rs +++ b/differential-dataflow/examples/accumulate.rs @@ -7,7 +7,7 @@ fn main() { let keys: usize = std::env::args().nth(1).unwrap().parse().unwrap(); let batch: usize = 10_000; - // This computation demonstrates in-place accumulation of arbitrarily large + // This computation demonstrates in-place accumulation of arbitrarily large // volumes of input data, consuming space bounded by the number of distinct keys. timely::execute_from_args(std::env::args().skip(2), move |worker| { @@ -17,11 +17,10 @@ fn main() { let mut input = worker.dataflow::<(), _, _>(|scope| { let (input, data) = scope.new_collection::<_, isize>(); - use timely::dataflow::Scope; scope.iterative::(|inner| { data.enter_at(inner, |_| 0) .consolidate() - .leave() + .leave(&scope) }); input @@ -41,7 +40,7 @@ fn main() { } counter += batch; - worker.step(); + worker.step(); let elapsed = timer.elapsed(); if elapsed.as_secs() as usize > last_sec { @@ -54,4 +53,4 @@ fn main() { } }).unwrap(); -} \ No newline at end of file +} diff --git a/differential-dataflow/examples/bfs.rs b/differential-dataflow/examples/bfs.rs index af875d5d5..db97f3938 100644 --- a/differential-dataflow/examples/bfs.rs +++ b/differential-dataflow/examples/bfs.rs @@ -1,6 +1,5 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; @@ -91,9 +90,9 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord, { // initialize roots as reaching themselves at distance 0 let nodes = roots.map(|x| (x, 0)); diff --git a/differential-dataflow/examples/columnar/columnar_support.rs b/differential-dataflow/examples/columnar/columnar_support.rs index 8b6e10543..03a260218 100644 --- a/differential-dataflow/examples/columnar/columnar_support.rs +++ b/differential-dataflow/examples/columnar/columnar_support.rs @@ -592,7 +592,7 @@ pub mod arrangement { impl Merger for TrieMerger where - U::Time: Ord + PartialOrder + Clone + 'static, + U::Time: 'static, { type Chunk = Updates; type Time = U::Time; @@ -646,7 +646,7 @@ pub mod arrangement { impl TrieMerger where - U::Time: Ord + PartialOrder + Clone + 'static, + U::Time: 'static, { /// Iterator-based merge: flatten, merge, consolidate, form. /// Correct but slow — used as fallback. @@ -1839,13 +1839,12 @@ pub mod updates { /// joins output times with input times, multiplies output diffs with input diffs. /// /// This subsumes map, filter, negate, and enter_at for columnar collections. -pub fn join_function( - input: differential_dataflow::Collection>, +pub fn join_function( + input: differential_dataflow::Collection>, mut logic: L, -) -> differential_dataflow::Collection> +) -> differential_dataflow::Collection> where - G: timely::dataflow::Scope, - G::Timestamp: differential_dataflow::lattice::Lattice, + U::Time: differential_dataflow::lattice::Lattice, U: layout::ColumnarUpdate>, I: IntoIterator, L: FnMut( @@ -1893,12 +1892,11 @@ type DynTime = timely::order::Product( - input: differential_dataflow::Collection>, +pub fn leave_dynamic( + input: differential_dataflow::Collection>, level: usize, -) -> differential_dataflow::Collection> +) -> differential_dataflow::Collection> where - G: timely::dataflow::Scope, K: columnar::Columnar, V: columnar::Columnar, R: columnar::Columnar, @@ -1972,14 +1970,12 @@ where /// /// Cursors through each batch and pushes `(key, val, time, diff)` refs into /// a `ValColBuilder`, which sorts and consolidates on flush. -pub fn as_recorded_updates( +pub fn as_recorded_updates( arranged: differential_dataflow::operators::arrange::Arranged< - G, differential_dataflow::operators::arrange::TraceAgent>, >, -) -> differential_dataflow::Collection> +) -> differential_dataflow::Collection> where - G: timely::dataflow::Scope, U: layout::ColumnarUpdate, { use timely::dataflow::operators::generic::Operator; diff --git a/differential-dataflow/examples/columnar/main.rs b/differential-dataflow/examples/columnar/main.rs index 85b573157..a6a2b9ccd 100644 --- a/differential-dataflow/examples/columnar/main.rs +++ b/differential-dataflow/examples/columnar/main.rs @@ -93,7 +93,6 @@ fn main() { mod reachability { use timely::order::Product; - use timely::dataflow::Scope; use differential_dataflow::Collection; use differential_dataflow::AsCollection; use differential_dataflow::operators::iterate::Variable; @@ -110,14 +109,14 @@ mod reachability { /// Compute the set of nodes reachable from `roots` along directed `edges`. /// /// Returns `(node, ())` for each reachable node. - pub fn reach>( - edges: Collection>, - roots: Collection>, - ) -> Collection> + pub fn reach( + edges: Collection>, + roots: Collection>, + ) -> Collection> { - let mut scope = edges.inner.scope(); + let outer = edges.inner.scope(); - scope.iterative::(|nested| { + outer.iterative::(|nested| { let summary = Product::new(Time::default(), 1); let roots_inner = roots.enter(nested); @@ -128,13 +127,13 @@ mod reachability { let edges_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; let reach_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; - let edges_arr = arrange_core::<_, _, + let edges_arr = arrange_core::<_, ValBatcher, ValBuilder, ValSpine, >(edges_inner.inner, edges_pact, "Edges"); - let reach_arr = arrange_core::<_, _, + let reach_arr = arrange_core::<_, ValBatcher, ValBuilder, ValSpine, @@ -142,7 +141,7 @@ mod reachability { // join_traces with ValColBuilder: produces Stream<_, RecordedUpdates<...>>. let proposed = - join_traces::<_, _, _, _, ValColBuilder<(Node, (), IterTime, Diff)>>( + join_traces::<_, _, _, ValColBuilder<(Node, (), IterTime, Diff)>>( edges_arr, reach_arr, |_src, dst, (), time, d1, d2, session| { @@ -158,7 +157,7 @@ mod reachability { // Arrange for reduce. let combined_pact = ValPact { hashfunc: |k: columnar::Ref<'_, Node>| *k as u64 }; - let combined_arr = arrange_core::<_, _, + let combined_arr = arrange_core::<_, ValBatcher, ValBuilder, ValSpine, @@ -181,12 +180,12 @@ mod reachability { }); // Extract RecordedUpdates from the Arranged's batch stream. - let result_col = as_recorded_updates::<_, (Node, (), IterTime, Diff)>(result); + let result_col = as_recorded_updates::<(Node, (), IterTime, Diff)>(result); variable.set(result_col.clone()); // Leave the iterative scope. - result_col.leave() + result_col.leave(&outer) }) } } diff --git a/differential-dataflow/examples/dynamic.rs b/differential-dataflow/examples/dynamic.rs index a844c28bb..c0206fbde 100644 --- a/differential-dataflow/examples/dynamic.rs +++ b/differential-dataflow/examples/dynamic.rs @@ -1,6 +1,5 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; @@ -91,9 +90,9 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord, { use timely::order::Product; use iterate::Variable; @@ -103,7 +102,8 @@ where let nodes = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - nodes.scope().iterative::, _, _>(|inner| { + let outer = nodes.scope(); + outer.iterative::, _, _>(|inner| { // These enter the statically bound scope, rather than any iterative scopes. // We do not *need* to enter them into the dynamic scope, as they are static @@ -126,7 +126,7 @@ where // Leave the dynamic iteration, stripping off the last timestamp coordinate. next.leave_dynamic(1) .inspect(|x| println!("{:?}", x)) - .leave() + .leave(&outer) }) } diff --git a/differential-dataflow/examples/graspan.rs b/differential-dataflow/examples/graspan.rs index 79c7fd184..4725cf4af 100644 --- a/differential-dataflow/examples/graspan.rs +++ b/differential-dataflow/examples/graspan.rs @@ -5,7 +5,6 @@ use std::fs::File; use timely::progress::Timestamp; use timely::order::Product; use timely::dataflow::Scope; -use timely::dataflow::scopes::ScopeParent; use differential_dataflow::VecCollection; use differential_dataflow::lattice::Lattice; @@ -69,7 +68,7 @@ use differential_dataflow::operators::arrange::{Arranged, TraceAgent}; type TraceKeyHandle = TraceAgent>; type TraceValHandle = TraceAgent>; -type Arrange = Arranged::Timestamp, R>>; +type Arrange = Arranged>; /// An evolving set of edges. /// @@ -79,17 +78,17 @@ type Arrange = Arranged::Tim /// /// An edge variable provides arranged representations of its contents, even before they are /// completely defined, in support of recursively defined productions. -pub struct EdgeVariable> { - variable: VecVariable, - collection: VecCollection, - current: VecCollection, - forward: Option>, - reverse: Option>, +pub struct EdgeVariable { + variable: VecVariable, + collection: VecCollection, + current: VecCollection, + forward: Option>, + reverse: Option>, } -impl> EdgeVariable { +impl EdgeVariable { /// Creates a new variable initialized with `source`. - pub fn from(source: VecCollection, step: ::Summary) -> Self { + pub fn from(source: VecCollection, step: T::Summary) -> Self { let (variable, collection) = VecVariable::new(&mut source.scope(), step); EdgeVariable { variable, @@ -100,7 +99,7 @@ impl> EdgeVariable { } } /// Concatenates `production` into the definition of the variable. - pub fn add_production(&mut self, production: VecCollection) { + pub fn add_production(&mut self, production: VecCollection) { self.current = self.current.clone().concat(production); } /// Finalizes the variable, connecting its recursive definition. @@ -113,14 +112,14 @@ impl> EdgeVariable { self.variable.set(distinct); } /// The collection arranged in the forward direction. - pub fn forward(&mut self) -> &Arrange { + pub fn forward(&mut self) -> &Arrange { if self.forward.is_none() { self.forward = Some(self.collection.clone().arrange_by_key()); } self.forward.as_ref().unwrap() } /// The collection arranged in the reverse direction. - pub fn reverse(&mut self) -> &Arrange { + pub fn reverse(&mut self) -> &Arrange { if self.reverse.is_none() { self.reverse = Some(self.collection.clone().map(|(x,y)| (y,x)).arrange_by_key()); } @@ -151,9 +150,9 @@ impl Query { } /// Creates a dataflow implementing the query, and returns input and trace handles. - pub fn render_in(&self, scope: &mut G) -> BTreeMap> + pub fn render_in(&self, scope: &mut Scope) -> BTreeMap> where - G: Scope, + T: Timestamp + Lattice + ::timely::order::TotalOrder, { // Create new input (handle, stream) pairs let mut input_map = BTreeMap::new(); @@ -171,7 +170,7 @@ impl Query { // create variables and result handles for each named relation. for (name, (input, collection)) in input_map { let edge_variable = EdgeVariable::from(collection.enter(subscope), Product::new(Default::default(), 1)); - let trace = edge_variable.collection.clone().leave().arrange_by_self().trace; + let trace = edge_variable.collection.clone().leave(&scope).arrange_by_self().trace; result_map.insert(name.clone(), RelationHandles { input, trace }); variable_map.insert(name.clone(), edge_variable); } diff --git a/differential-dataflow/examples/interpreted.rs b/differential-dataflow/examples/interpreted.rs index 4c6e3429c..31a622108 100644 --- a/differential-dataflow/examples/interpreted.rs +++ b/differential-dataflow/examples/interpreted.rs @@ -1,5 +1,4 @@ use std::hash::Hash; -use timely::dataflow::*; use timely::dataflow::operators::*; use differential_dataflow::VecCollection; @@ -37,9 +36,9 @@ fn main() { }).unwrap(); } -fn interpret(edges: VecCollection, relations: &[(usize, usize)]) -> VecCollection> +fn interpret(edges: VecCollection, relations: &[(usize, usize)]) -> VecCollection> where - G: Scope, + T: timely::progress::Timestamp + Lattice + Hash + Ord, { // arrange the edge relation three ways. diff --git a/differential-dataflow/examples/iterate_container.rs b/differential-dataflow/examples/iterate_container.rs index f7ef3764e..d2fa85302 100644 --- a/differential-dataflow/examples/iterate_container.rs +++ b/differential-dataflow/examples/iterate_container.rs @@ -3,7 +3,7 @@ use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::Operator; use timely::order::Product; -use timely::dataflow::{Scope, Stream}; +use timely::dataflow::Stream; use timely::dataflow::operators::generic::builder_rc::OperatorBuilder; use differential_dataflow::{AsCollection, Collection}; use differential_dataflow::input::Input; @@ -33,7 +33,7 @@ impl, TS> ResultsIn for ContainerWrapper { #[inline(always)] fn results_in(self, step: &TS) -> Self { ContainerWrapper(self.0.results_in(step)) } } -fn wrap(stream: Stream) -> Stream> { +fn wrap(stream: Stream) -> Stream> { let mut builder = OperatorBuilder::new("Wrap".to_string(), stream.scope()); let (mut output, stream_out) = builder.new_output(); let mut input = builder.new_input(stream, Pipeline); @@ -77,7 +77,7 @@ fn main() { }).as_collection().consolidate(); let result = wrap(result.inner).as_collection(); variable.set(result); - collection.leave() + collection.leave(&scope) }); }) } diff --git a/differential-dataflow/examples/monoid-bfs.rs b/differential-dataflow/examples/monoid-bfs.rs index cce06674e..f658cce0f 100644 --- a/differential-dataflow/examples/monoid-bfs.rs +++ b/differential-dataflow/examples/monoid-bfs.rs @@ -1,7 +1,6 @@ use rand::{Rng, SeedableRng, StdRng}; use serde::{Deserialize, Serialize}; -use timely::dataflow::*; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; @@ -122,12 +121,13 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord, { // repeatedly update minimal distances each node can be reached from each root - roots.scope().iterative::(|scope| { + let outer = roots.scope(); + outer.iterative::(|scope| { use differential_dataflow::operators::iterate::Variable; use differential_dataflow::trace::implementations::{KeySpine, KeyBuilder}; @@ -152,6 +152,6 @@ where .as_collection(|k,()| *k); variable.set(result.clone()); - result.leave() + result.leave(&outer) }) } diff --git a/differential-dataflow/examples/pagerank.rs b/differential-dataflow/examples/pagerank.rs index d779ed36e..b649316ad 100644 --- a/differential-dataflow/examples/pagerank.rs +++ b/differential-dataflow/examples/pagerank.rs @@ -77,9 +77,9 @@ fn main() { // Returns a weighted collection in which the weight of each node is proportional // to its PageRank in the input graph `edges`. -fn pagerank(iters: Iter, edges: VecCollection) -> VecCollection +fn pagerank(iters: Iter, edges: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice, { // initialize many surfers at each node. let nodes = @@ -92,7 +92,8 @@ where .map(|(src,_dst)| src) .count(); - edges.scope().iterative::(|inner| { + let outer = edges.scope(); + outer.iterative::(|inner| { // Bring various collections into the scope. let edges = edges.enter(inner); @@ -130,6 +131,6 @@ where // Bind the recursive variable, return its limit. ranks_bind.set(pushed.clone()); - pushed.leave() + pushed.leave(&outer) }) } diff --git a/differential-dataflow/examples/progress.rs b/differential-dataflow/examples/progress.rs index 634e43d56..0229263d8 100644 --- a/differential-dataflow/examples/progress.rs +++ b/differential-dataflow/examples/progress.rs @@ -1,7 +1,6 @@ //! A demonstration of timely dataflow progress tracking, using differential dataflow operators. use timely::PartialOrder; -use timely::dataflow::*; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; @@ -120,7 +119,7 @@ fn frontier( times: VecCollection, ) -> VecCollection where - G: Scope, + G: Timestamp + Lattice + Ord, T: Timestamp+std::hash::Hash, { // Translate node and edge transitions into a common Location to Location edge with an associated Summary. @@ -154,7 +153,7 @@ fn summarize( edges: VecCollection, ) -> VecCollection where - G: Scope, + G: Timestamp + Lattice + Ord, T: Timestamp, { // Start from trivial reachability from each input to itself. @@ -194,12 +193,12 @@ where /// Identifies cycles along paths that do not increment timestamps. -fn find_cycles( +fn find_cycles( nodes: VecCollection, edges: VecCollection, ) -> VecCollection where - G: Scope, + G: Timestamp + Lattice + Ord, T: Timestamp, { // Retain node connections along "default" timestamp summaries. diff --git a/differential-dataflow/examples/stackoverflow.rs b/differential-dataflow/examples/stackoverflow.rs index 3fe521ee2..5d25b19b1 100644 --- a/differential-dataflow/examples/stackoverflow.rs +++ b/differential-dataflow/examples/stackoverflow.rs @@ -1,7 +1,6 @@ use std::io::{BufRead, BufReader}; use std::fs::File; -use timely::dataflow::*; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::InputSession; @@ -105,9 +104,9 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord, { // initialize roots as reaching themselves at distance 0 let nodes = roots.map(|x| (x, 0)); diff --git a/differential-dataflow/src/algorithms/graphs/bfs.rs b/differential-dataflow/src/algorithms/graphs/bfs.rs index 1513622bd..a9d5ed307 100644 --- a/differential-dataflow/src/algorithms/graphs/bfs.rs +++ b/differential-dataflow/src/algorithms/graphs/bfs.rs @@ -2,16 +2,16 @@ use std::hash::Hash; -use timely::dataflow::*; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::operators::*; use crate::lattice::Lattice; /// Returns pairs (node, dist) indicating distance of each node from a root. -pub fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +pub fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord, N: ExchangeData+Hash, { let edges = edges.arrange_by_key(); @@ -22,9 +22,8 @@ use crate::trace::TraceReader; use crate::operators::arrange::Arranged; /// Returns pairs (node, dist) indicating distance of each node from a root. -pub fn bfs_arranged(edges: Arranged, roots: VecCollection) -> VecCollection +pub fn bfs_arranged(edges: Arranged, roots: VecCollection) -> VecCollection where - G: Scope, N: ExchangeData+Hash, Tr: for<'a> TraceReader=&'a N, Val<'a>=&'a N, Diff=isize>+Clone+'static, { diff --git a/differential-dataflow/src/algorithms/graphs/bijkstra.rs b/differential-dataflow/src/algorithms/graphs/bijkstra.rs index 9c61716ef..82f47ecec 100644 --- a/differential-dataflow/src/algorithms/graphs/bijkstra.rs +++ b/differential-dataflow/src/algorithms/graphs/bijkstra.rs @@ -3,7 +3,7 @@ use std::hash::Hash; use timely::order::Product; -use timely::dataflow::*; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::lattice::Lattice; @@ -19,9 +19,9 @@ use crate::operators::iterate::Variable; /// Goals that cannot reach from the source to the target are relatively expensive, as /// the entire graph must be explored to confirm this. A graph connectivity pre-filter /// could be good insurance here. -pub fn bidijkstra(edges: VecCollection, goals: VecCollection) -> VecCollection +pub fn bidijkstra(edges: VecCollection, goals: VecCollection) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord, N: ExchangeData+Hash, { let forward = edges.clone().arrange_by_key(); @@ -33,19 +33,17 @@ use crate::trace::TraceReader; use crate::operators::arrange::Arranged; /// Bi-directional Dijkstra search using arranged forward and reverse edge collections. -pub fn bidijkstra_arranged( - forward: Arranged, - reverse: Arranged, - goals: VecCollection -) -> VecCollection +pub fn bidijkstra_arranged( + forward: Arranged, + reverse: Arranged, + goals: VecCollection +) -> VecCollection where - G: Scope, N: ExchangeData+Hash, Tr: for<'a> TraceReader=&'a N, Val<'a>=&'a N, Diff=isize>+Clone+'static, { - forward - .stream - .scope().iterative::(|inner| { + let outer = forward.stream.scope(); + outer.iterative::(|inner| { let forward_edges = forward.enter(inner); let reverse_edges = reverse.enter(inner); @@ -120,6 +118,6 @@ where reverse_bind.set(reverse_next); - reached.leave() + reached.leave(&outer) }) } diff --git a/differential-dataflow/src/algorithms/graphs/propagate.rs b/differential-dataflow/src/algorithms/graphs/propagate.rs index 2728de6e2..54efb079d 100644 --- a/differential-dataflow/src/algorithms/graphs/propagate.rs +++ b/differential-dataflow/src/algorithms/graphs/propagate.rs @@ -2,7 +2,7 @@ use std::hash::Hash; -use timely::dataflow::*; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::lattice::Lattice; @@ -13,9 +13,9 @@ use crate::difference::{Abelian, Multiply}; /// This algorithm naively propagates all labels at once, much like standard label propagation. /// To more carefully control the label propagation, consider `propagate_core` which supports a /// method to limit the introduction of labels. -pub fn propagate(edges: VecCollection, nodes: VecCollection) -> VecCollection +pub fn propagate(edges: VecCollection, nodes: VecCollection) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord + Hash, N: ExchangeData+Hash, R: ExchangeData+Abelian, R: Multiply, @@ -30,9 +30,9 @@ where /// This algorithm naively propagates all labels at once, much like standard label propagation. /// To more carefully control the label propagation, consider `propagate_core` which supports a /// method to limit the introduction of labels. -pub fn propagate_at(edges: VecCollection, nodes: VecCollection, logic: F) -> VecCollection +pub fn propagate_at(edges: VecCollection, nodes: VecCollection, logic: F) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord + Hash, N: ExchangeData+Hash, R: ExchangeData+Abelian, R: Multiply, @@ -51,15 +51,14 @@ use crate::operators::arrange::arrangement::Arranged; /// This variant takes a pre-arranged edge collection, to facilitate re-use, and allows /// a method `logic` to specify the rounds in which we introduce various labels. The output /// of `logic should be a number in the interval \[0,64\], -pub fn propagate_core(edges: Arranged, nodes: VecCollection, logic: F) -> VecCollection +pub fn propagate_core(edges: Arranged, nodes: VecCollection, logic: F) -> VecCollection where - G: Scope, N: ExchangeData+Hash, R: ExchangeData+Abelian, R: Multiply, R: From, L: ExchangeData, - Tr: for<'a> TraceReader=&'a N, Val<'a>=&'a N, Time:Hash, Diff=R>+Clone+'static, + Tr: for<'a> TraceReader=&'a N, Val<'a>=&'a N, Diff=R, Time: Hash>+Clone+'static, F: Fn(&L)->u64+Clone+'static, { // Morally the code performs the following iterative computation. However, in the interest of a simplified @@ -78,7 +77,8 @@ where // }) use timely::order::Product; - nodes.scope().scoped::,_,_>("Propagate", |scope| { + let outer = nodes.scope(); + outer.scoped::,_,_>("Propagate", |scope| { use crate::operators::iterate::Variable; use crate::trace::implementations::{ValBuilder, ValSpine}; @@ -104,6 +104,6 @@ where labels .as_collection(|k,v| (k.clone(), v.clone())) - .leave() + .leave(&outer) }) } diff --git a/differential-dataflow/src/algorithms/graphs/scc.rs b/differential-dataflow/src/algorithms/graphs/scc.rs index b730302e7..f158cdc08 100644 --- a/differential-dataflow/src/algorithms/graphs/scc.rs +++ b/differential-dataflow/src/algorithms/graphs/scc.rs @@ -3,7 +3,7 @@ use std::mem; use std::hash::Hash; -use timely::dataflow::*; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::lattice::Lattice; @@ -12,16 +12,17 @@ use crate::difference::{Abelian, Multiply}; use super::propagate::propagate; /// Returns the subset of edges in the same strongly connected component. -pub fn strongly_connected(graph: VecCollection) -> VecCollection +pub fn strongly_connected(graph: VecCollection) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord + Hash, N: ExchangeData + Hash, R: ExchangeData + Abelian, R: Multiply, R: From { use timely::order::Product; - graph.scope().scoped::,_,_>("StronglyConnected", |scope| { + let outer = graph.scope(); + outer.scoped::,_,_>("StronglyConnected", |scope| { // Bring in edges and transposed edges. let edges = graph.enter(&scope); let trans = edges.clone().map_in_place(|x| mem::swap(&mut x.0, &mut x.1)); @@ -31,20 +32,21 @@ where let result = trim_edges(trim_edges(inner, edges), trans); variable.set(result.clone()); - result.leave() + result.leave(&outer) }) } -fn trim_edges(cycle: VecCollection, edges: VecCollection) - -> VecCollection +fn trim_edges(cycle: VecCollection, edges: VecCollection) + -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord + Hash, N: ExchangeData + Hash, R: ExchangeData + Abelian, R: Multiply, R: From { - edges.inner.scope().region_named("TrimEdges", |region| { + let outer = edges.inner.scope(); + outer.region_named("TrimEdges", |region| { let cycle = cycle.enter_region(region); let edges = edges.enter_region(region); @@ -62,6 +64,6 @@ where .join_core(labels, |e2,(e1,l1),l2| [((e1.clone(),e2.clone()),(l1.clone(),l2.clone()))]) .filter(|(_,(l1,l2))| l1 == l2) .map(|((x1,x2),_)| (x2,x1)) - .leave_region() + .leave_region(&outer) }) } diff --git a/differential-dataflow/src/algorithms/graphs/sequential.rs b/differential-dataflow/src/algorithms/graphs/sequential.rs index 354ad32c2..8ba203fa4 100644 --- a/differential-dataflow/src/algorithms/graphs/sequential.rs +++ b/differential-dataflow/src/algorithms/graphs/sequential.rs @@ -2,16 +2,16 @@ use std::hash::Hash; -use timely::dataflow::*; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::lattice::Lattice; use crate::operators::*; use crate::hashable::Hashable; -fn _color(edges: VecCollection) -> VecCollection)> +fn _color(edges: VecCollection) -> VecCollection)> where - G: Scope, + T: Timestamp + Lattice + Ord + Hash, N: ExchangeData+Hash, { // need some bogus initial values. @@ -40,12 +40,12 @@ where /// a node "fires" once all of its neighbors with lower identifier have /// fired, and we apply `logic` to the new state of lower neighbors and /// the old state (input) of higher neighbors. -pub fn sequence( - state: VecCollection, - edges: VecCollection, - logic: F) -> VecCollection)> +pub fn sequence( + state: VecCollection, + edges: VecCollection, + logic: F) -> VecCollection)> where - G: Scope, + T: Timestamp + Lattice + Hash + Ord, N: ExchangeData+Hashable, V: ExchangeData, F: Fn(&N, &[(&V, isize)])->V+'static diff --git a/differential-dataflow/src/algorithms/identifiers.rs b/differential-dataflow/src/algorithms/identifiers.rs index 267e4c453..8c0678a53 100644 --- a/differential-dataflow/src/algorithms/identifiers.rs +++ b/differential-dataflow/src/algorithms/identifiers.rs @@ -1,6 +1,6 @@ //! Assign unique identifiers to records. -use timely::dataflow::Scope; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData, Hashable}; use crate::lattice::Lattice; @@ -8,7 +8,7 @@ use crate::operators::*; use crate::difference::Abelian; /// Assign unique identifiers to elements of a collection. -pub trait Identifiers { +pub trait Identifiers { /// Assign unique identifiers to elements of a collection. /// /// # Example @@ -27,16 +27,16 @@ pub trait Identifiers { /// .assert_empty(); /// }); /// ``` - fn identifiers(self) -> VecCollection; + fn identifiers(self) -> VecCollection; } -impl Identifiers for VecCollection +impl Identifiers for VecCollection where - G: Scope, + T: Timestamp + Lattice, D: ExchangeData + ::std::hash::Hash, R: ExchangeData + Abelian, { - fn identifiers(self) -> VecCollection { + fn identifiers(self) -> VecCollection { // The design here is that we iteratively develop a collection // of pairs (round, record), where each pair is a proposal that diff --git a/differential-dataflow/src/algorithms/prefix_sum.rs b/differential-dataflow/src/algorithms/prefix_sum.rs index f76f59d9c..f7401d892 100644 --- a/differential-dataflow/src/algorithms/prefix_sum.rs +++ b/differential-dataflow/src/algorithms/prefix_sum.rs @@ -1,13 +1,13 @@ //! Implementation of Parallel Prefix Sum -use timely::dataflow::Scope; +use timely::progress::Timestamp; use crate::{VecCollection, ExchangeData}; use crate::lattice::Lattice; use crate::operators::*; /// Extension trait for the prefix_sum method. -pub trait PrefixSum { +pub trait PrefixSum { /// Computes the prefix sum for each element in the collection. /// /// The prefix sum is data-parallel, in the sense that the sums are computed independently for @@ -16,12 +16,12 @@ pub trait PrefixSum { fn prefix_sum(self, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static; /// Determine the prefix sum at each element of `location`. - fn prefix_sum_at(self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static; + fn prefix_sum_at(self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static; } -impl PrefixSum for VecCollection +impl PrefixSum for VecCollection where - G: Scope, + T: Timestamp + Lattice, K: ExchangeData + ::std::hash::Hash, D: ExchangeData + ::std::hash::Hash, { @@ -29,7 +29,7 @@ where self.clone().prefix_sum_at(self.map(|(x,_)| x), zero, combine) } - fn prefix_sum_at(self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static { + fn prefix_sum_at(self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static { let combine1 = ::std::rc::Rc::new(combine); let combine2 = combine1.clone(); @@ -40,9 +40,9 @@ where } /// Accumulate data in `collection` into all powers-of-two intervals containing them. -pub fn aggregate(collection: VecCollection, combine: F) -> VecCollection +pub fn aggregate(collection: VecCollection, combine: F) -> VecCollection where - G: Scope, + T: Timestamp + Lattice, K: ExchangeData + ::std::hash::Hash, D: ExchangeData + ::std::hash::Hash, F: Fn(&K,&D,&D)->D + 'static, @@ -73,13 +73,13 @@ where } /// Produces the accumulated values at each of the `usize` locations in `queries`. -pub fn broadcast( - ranges: VecCollection, - queries: VecCollection, +pub fn broadcast( + ranges: VecCollection, + queries: VecCollection, zero: D, - combine: F) -> VecCollection + combine: F) -> VecCollection where - G: Scope, + T: Timestamp + Lattice + Ord + ::std::fmt::Debug, K: ExchangeData + ::std::hash::Hash, D: ExchangeData + ::std::hash::Hash, F: Fn(&K,&D,&D)->D + 'static, diff --git a/differential-dataflow/src/capture.rs b/differential-dataflow/src/capture.rs index 74cced0ca..65069bd74 100644 --- a/differential-dataflow/src/capture.rs +++ b/differential-dataflow/src/capture.rs @@ -230,7 +230,7 @@ pub mod source { use timely::dataflow::{Scope, Stream, operators::{Capability, CapabilitySet}}; use timely::dataflow::operators::generic::OutputBuilder; use timely::progress::Timestamp; - use timely::scheduling::SyncActivator; + use timely::scheduling::{Scheduler, SyncActivator}; // TODO(guswynn): implement this generally in timely struct DropActivator { @@ -250,12 +250,11 @@ pub mod source { /// The stream is built in the supplied `scope` and continues to run until /// the returned `Box` token is dropped. The `source_builder` argument /// is invoked with a `SyncActivator` that will re-activate the source. - pub fn build( - scope: G, + pub fn build( + scope: Scope, source_builder: B, - ) -> (Box, Stream>) + ) -> (Box, Stream>) where - G: Scope, B: FnOnce(SyncActivator) -> I, I: Iterator> + 'static, D: ExchangeData + Hash, @@ -560,9 +559,10 @@ pub mod sink { use timely::order::PartialOrder; use timely::progress::{Antichain, ChangeBatch, Timestamp}; - use timely::dataflow::{Scope, Stream}; + use timely::dataflow::Stream; use timely::dataflow::channels::pact::{Exchange, Pipeline}; use timely::dataflow::operators::generic::{builder_rc::OperatorBuilder, OutputBuilder}; + use timely::scheduling::Scheduler; use crate::{lattice::Lattice, ExchangeData}; use super::{Writer, Message, Progress}; @@ -573,13 +573,12 @@ pub mod sink { /// will *not* perform the consolidation on the stream's behalf. If this is not /// performed before calling the method, the recorded output may not be correctly /// reconstructed by readers. - pub fn build( - stream: Stream>, + pub fn build( + stream: Stream>, sink_hash: u64, updates_sink: Weak>, progress_sink: Weak>, ) where - G: Scope, BS: Writer> + 'static, D: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a>, T: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a> + Timestamp + Lattice, @@ -742,9 +741,9 @@ pub mod sink { // use crate::lattice::Lattice; // /// Creates a Kafka source from supplied configuration information. -// pub fn create_source(scope: G, addr: &str, topic: &str, group: &str) -> (Box, Stream>) +// pub fn create_source(scope: T, addr: &str, topic: &str, group: &str) -> (Box, Stream>) // where -// G: Scope, +// T: Scope, // D: ExchangeData + Hash + for<'a> serde::Deserialize<'a>, // T: ExchangeData + Hash + for<'a> serde::Deserialize<'a> + Timestamp + Lattice, // R: ExchangeData + Hash + for<'a> serde::Deserialize<'a>, @@ -757,9 +756,9 @@ pub mod sink { // }) // } -// pub fn create_sink(stream: &Stream>, addr: &str, topic: &str) -> Box +// pub fn create_sink(stream: &Stream>, addr: &str, topic: &str) -> Box // where -// G: Scope, +// T: Scope, // D: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a>, // T: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a> + Timestamp + Lattice, // R: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a>, diff --git a/differential-dataflow/src/collection.rs b/differential-dataflow/src/collection.rs index 6981cc3da..d33463863 100644 --- a/differential-dataflow/src/collection.rs +++ b/differential-dataflow/src/collection.rs @@ -10,7 +10,6 @@ use timely::Container; use timely::progress::Timestamp; -use timely::dataflow::scopes::Child; use timely::dataflow::{Scope, Stream}; use timely::dataflow::operators::*; @@ -22,7 +21,7 @@ use crate::difference::Abelian; /// in order to expose some of this functionality (e.g. negation, timestamp manipulation). Other actions /// on the containers, and streams of containers, are left to the container implementor to describe. #[derive(Clone)] -pub struct Collection { +pub struct Collection { /// The underlying timely dataflow stream. /// /// This field is exposed to support direct timely dataflow manipulation when required, but it is @@ -31,10 +30,10 @@ pub struct Collection { /// The timestamp in the data is required to always be at least the timestamp _of_ the data, in /// the timely-dataflow sense. If this invariant is not upheld, differential operators may behave /// unexpectedly. - pub inner: Stream, + pub inner: Stream, } -impl Collection { +impl Collection { /// Creates a new Collection from a timely dataflow stream. /// /// This method seems to be rarely used, with the `as_collection` method on streams being a more @@ -44,9 +43,9 @@ impl Collection { /// /// This stream should satisfy the timestamp invariant as documented on [Collection]; this /// method does not check it. - pub fn new(stream: Stream) -> Self { Self { inner: stream } } + pub fn new(stream: Stream) -> Self { Self { inner: stream } } } -impl Collection { +impl Collection { /// Creates a new collection accumulating the contents of the two collections. /// /// Despite the name, differential dataflow collections are unordered. This method is so named because the @@ -109,7 +108,7 @@ impl Collection { /// /// This method is a specialization of `enter` to the case where the nested scope is a region. /// It removes the need for an operator that adjusts the timestamp. - pub fn enter_region<'a>(self, child: &Child<'a, G, ::Timestamp>) -> Collection::Timestamp>, C> { + pub fn enter_region(self, child: &Scope) -> Self { self.inner .enter(child) .as_collection() @@ -134,7 +133,7 @@ impl Collection { /// ``` pub fn inspect_container(self, func: F) -> Self where - F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static, + F: FnMut(Result<(&T, &C), &[T]>)+'static, { self.inner .inspect_container(func) @@ -144,7 +143,7 @@ impl Collection { /// /// This probe is used to determine when the state of the Collection has stabilized and can /// be read out. - pub fn probe(self) -> (probe::Handle, Self) { + pub fn probe(self) -> (probe::Handle, Self) { let (handle, stream) = self.inner.probe(); (handle, stream.as_collection()) } @@ -154,11 +153,11 @@ impl Collection { /// In addition, a probe is also often use to limit the number of rounds of input in flight at any moment; a /// computation can wait until the probe has caught up to the input before introducing more rounds of data, to /// avoid swamping the system. - pub fn probe_with(self, handle: &probe::Handle) -> Self { + pub fn probe_with(self, handle: &probe::Handle) -> Self { Self::new(self.inner.probe_with(handle)) } /// The scope containing the underlying timely dataflow stream. - pub fn scope(&self) -> G { + pub fn scope(&self) -> Scope { self.inner.scope() } @@ -209,16 +208,16 @@ impl Collection { /// let result = scope.region(|child| { /// data.clone() /// .enter(child) - /// .leave() + /// .leave(&scope) /// }); /// /// data.assert_eq(result); /// }); /// ``` - pub fn enter<'a, T>(self, child: &Child<'a, G, T>) -> Collection, ::Timestamp, T>>::InnerContainer> + pub fn enter(self, child: &Scope) -> Collection>::InnerContainer> where - C: containers::Enter<::Timestamp, T, InnerContainer: Container>, - T: Refines<::Timestamp>, + C: containers::Enter, + TInner: Refines, { use timely::dataflow::channels::pact::Pipeline; self.inner @@ -249,9 +248,9 @@ impl Collection { /// data.results_in(summary1); /// }); /// ``` - pub fn results_in(self, step: ::Summary) -> Self + pub fn results_in(self, step: T::Summary) -> Self where - C: containers::ResultsIn<::Summary>, + C: containers::ResultsIn, { use timely::dataflow::channels::pact::Pipeline; self.inner @@ -262,14 +261,10 @@ impl Collection { } } -use timely::dataflow::scopes::ScopeParent; use timely::progress::timestamp::Refines; /// Methods requiring a nested scope. -impl<'a, G: Scope, T: Timestamp, C: Container> Collection, C> -where - C: containers::Leave, - T: Refines<::Timestamp>, +impl Collection { /// Returns the final value of a Collection from a nested scope to its containing scope. /// @@ -286,33 +281,37 @@ where /// let result = scope.region(|child| { /// data.clone() /// .enter(child) - /// .leave() + /// .leave(scope) /// }); /// /// data.assert_eq(result); /// }); /// ``` - pub fn leave(self) -> Collection>::OuterContainer> { + pub fn leave(self, outer: &Scope) -> Collection>::OuterContainer> + where + TOuter: Timestamp, + T: Refines, + C: containers::Leave, + { use timely::dataflow::channels::pact::Pipeline; self.inner - .leave() + .leave(outer) .unary(Pipeline, "Leave", move |_,_| move |input, output| { input.for_each(|time, data| output.session(&time).give_container(&mut std::mem::take(data).leave())); }) .as_collection() } -} -/// Methods requiring a region as the scope. -impl Collection, C> -{ /// Returns the value of a Collection from a nested region to its containing scope. /// /// This method is a specialization of `leave` to the case that of a nested region. /// It removes the need for an operator that adjusts the timestamp. - pub fn leave_region(self) -> Collection { + pub fn leave_region(self, outer: &Scope) -> Self + where + C: Clone + 'static, + { self.inner - .leave() + .leave(outer) .as_collection() } } @@ -325,8 +324,7 @@ pub mod vec { use timely::progress::Timestamp; use timely::order::Product; - use timely::dataflow::scopes::child::Iterative; - use timely::dataflow::{Scope, ScopeParent}; + use timely::dataflow::scope::Iterative; use timely::dataflow::operators::*; use timely::dataflow::operators::vec::*; @@ -344,18 +342,19 @@ pub mod vec { /// propagate changes through your functional computation and report the corresponding changes to the /// output collections. /// - /// Each vec collection has three generic parameters. The parameter `G` is for the scope in which the - /// collection exists; as you write more complicated programs you may wish to introduce nested scopes - /// (e.g. for iteration) and this parameter tracks the scope (for timely dataflow's benefit). The `D` - /// parameter is the type of data in your collection, for example `String`, or `(u32, Vec>)`. + /// Each vec collection has three generic parameters. The parameter `T` is the timestamp type of the + /// scope in which the collection exists; as you write more complicated programs you may wish to + /// introduce nested scopes (e.g. for iteration), and this parameter tracks the scope's timestamp + /// (for timely dataflow's benefit). The `D` parameter is the type of data in your collection, for + /// example `String`, or `(u32, Vec>)`. /// The `R` parameter represents the types of changes that the data undergo, and is most commonly (and /// defaults to) `isize`, representing changes to the occurrence count of each record. /// - /// This type definition instantiates the [`Collection`] type with a `Vec<(D, G::Timestamp, R)>`. - pub type Collection = super::Collection::Timestamp, R)>>; + /// This type definition instantiates the [`Collection`] type with a `Vec<(D, T, R)>`. + pub type Collection = super::Collection>; - impl Collection { + impl Collection { /// Creates a new collection by applying the supplied function to each input element. /// /// # Examples @@ -370,7 +369,7 @@ pub mod vec { /// .assert_empty(); /// }); /// ``` - pub fn map(self, mut logic: L) -> Collection + pub fn map(self, mut logic: L) -> Collection where D2: Clone+'static, L: FnMut(D) -> D2 + 'static, @@ -397,7 +396,7 @@ pub mod vec { /// .assert_empty(); /// }); /// ``` - pub fn map_in_place(self, mut logic: L) -> Collection + pub fn map_in_place(self, mut logic: L) -> Collection where L: FnMut(&mut D) + 'static, { @@ -421,9 +420,9 @@ pub mod vec { /// .flat_map(|x| 0 .. x); /// }); /// ``` - pub fn flat_map(self, mut logic: L) -> Collection + pub fn flat_map(self, mut logic: L) -> Collection where - G::Timestamp: Clone, + T: Clone, I: IntoIterator, L: FnMut(D) -> I + 'static, { @@ -445,7 +444,7 @@ pub mod vec { /// .assert_empty(); /// }); /// ``` - pub fn filter(self, mut logic: L) -> Collection + pub fn filter(self, mut logic: L) -> Collection where L: FnMut(&D) -> bool + 'static, { @@ -473,7 +472,7 @@ pub mod vec { /// x1.assert_eq(x2); /// }); /// ``` - pub fn explode(self, mut logic: L) -> Collection>::Output> + pub fn explode(self, mut logic: L) -> Collection>::Output> where D2: Clone+'static, R2: Semigroup+Multiply, @@ -507,12 +506,12 @@ pub mod vec { /// ); /// }); /// ``` - pub fn join_function(self, mut logic: L) -> Collection>::Output> + pub fn join_function(self, mut logic: L) -> Collection>::Output> where - G::Timestamp: Lattice, + T: Lattice, D2: Clone+'static, R2: Semigroup+Multiply, - I: IntoIterator, + I: IntoIterator, L: FnMut(D)->I+'static, { self.inner @@ -537,17 +536,16 @@ pub mod vec { /// let result = scope.iterative::(|child| { /// data.clone() /// .enter_at(child, |x| *x) - /// .leave() + /// .leave(&scope) /// }); /// /// data.assert_eq(result); /// }); /// ``` - pub fn enter_at<'a, T, F>(self, child: &Iterative<'a, G, T>, mut initial: F) -> Collection, D, R> + pub fn enter_at(self, child: &Iterative, mut initial: F) -> Collection, D, R> where - T: Timestamp+Hash, - F: FnMut(&D) -> T + Clone + 'static, - G::Timestamp: Hash, + TInner: Timestamp+Hash, + F: FnMut(&D) -> TInner + Clone + 'static, { self.inner .enter(child) @@ -565,10 +563,10 @@ pub mod vec { /// ordered, they should have the same order or compare equal once `func` is applied to them (this /// is because we advance the timely capability with the same logic, and it must remain `less_equal` /// to all of the data timestamps). - pub fn delay(self, func: F) -> Collection + pub fn delay(self, func: F) -> Collection where - G::Timestamp: Hash, - F: FnMut(&G::Timestamp) -> G::Timestamp + Clone + 'static, + T: Hash, + F: FnMut(&T) -> T + Clone + 'static, { let mut func1 = func.clone(); let mut func2 = func.clone(); @@ -602,9 +600,9 @@ pub mod vec { /// .inspect(|x| println!("error: {:?}", x)); /// }); /// ``` - pub fn inspect(self, func: F) -> Collection + pub fn inspect(self, func: F) -> Collection where - F: FnMut(&(D, G::Timestamp, R))+'static, + F: FnMut(&(D, T, R))+'static, { self.inner .inspect(func) @@ -628,9 +626,9 @@ pub mod vec { /// .inspect_batch(|t,xs| println!("errors @ {:?}: {:?}", t, xs)); /// }); /// ``` - pub fn inspect_batch(self, mut func: F) -> Collection + pub fn inspect_batch(self, mut func: F) -> Collection where - F: FnMut(&G::Timestamp, &[(D, G::Timestamp, R)])+'static, + F: FnMut(&T, &[(D, T, R)])+'static, { self.inner .inspect_batch(move |time, data| func(time, data)) @@ -660,7 +658,7 @@ pub mod vec { where D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Hashable + Semigroup, - G::Timestamp: Lattice+Ord, + T: Lattice+Ord, { self.consolidate() .inspect(|x| panic!("Assertion failed: non-empty collection: {:?}", x)); @@ -668,7 +666,7 @@ pub mod vec { } /// Methods requiring an Abelian difference, to support negation. - impl, D: Clone+'static, R: Abelian+'static> Collection { + impl Collection { /// Assert if the collections are ever different. /// /// Because this is a dataflow fragment, the test is only applied as the computation is run. If the computation @@ -696,7 +694,7 @@ pub mod vec { where D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Hashable, - G::Timestamp: Lattice+Ord, + T: Lattice+Ord, { self.negate() .concat(other) @@ -707,9 +705,9 @@ pub mod vec { use crate::trace::{Trace, Builder}; use crate::operators::arrange::{Arranged, TraceAgent}; - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, K: crate::ExchangeData+Hashable, V: crate::ExchangeData, R: crate::ExchangeData+Semigroup, @@ -741,13 +739,13 @@ pub mod vec { /// }); /// }); /// ``` - pub fn reduce(self, logic: L) -> Collection + pub fn reduce(self, logic: L) -> Collection where L: FnMut(&K, &[(&V, R)], &mut Vec<(V2, R2)>)+'static { self.reduce_named("Reduce", logic) } /// As `reduce` with the ability to name the operator. - pub fn reduce_named(self, name: &str, logic: L) -> Collection + pub fn reduce_named(self, name: &str, logic: L) -> Collection where L: FnMut(&K, &[(&V, R)], &mut Vec<(V2, R2)>)+'static { use crate::trace::implementations::{ValBuilder, ValSpine}; @@ -784,9 +782,9 @@ pub mod vec { /// .trace; /// }); /// ``` - pub fn reduce_abelian(self, name: &str, mut logic: L) -> Arranged> + pub fn reduce_abelian(self, name: &str, mut logic: L) -> Arranged> where - T2: for<'a> Trace= &'a K, ValOwn = V, Time=G::Timestamp, Diff: Abelian>+'static, + T2: for<'a> Trace= &'a K, ValOwn = V, Time=T, Diff: Abelian>+'static, Bu: Builder, Output = T2::Batch>, L: FnMut(&K, &[(&V, R)], &mut Vec<(V, T2::Diff)>)+'static, { @@ -802,10 +800,10 @@ pub mod vec { /// Unlike `reduce_arranged`, this method may be called with an empty `input`, /// and it may not be safe to index into the first element. /// At least one of the two collections will be non-empty. - pub fn reduce_core(self, name: &str, logic: L) -> Arranged> + pub fn reduce_core(self, name: &str, logic: L) -> Arranged> where V: Clone+'static, - T2: for<'a> Trace=&'a K, ValOwn = V, Time=G::Timestamp>+'static, + T2: for<'a> Trace=&'a K, ValOwn = V, Time=T>+'static, Bu: Builder, Output = T2::Batch>, L: FnMut(&K, &[(&V, R)], &mut Vec<(V,T2::Diff)>, &mut Vec<(V, T2::Diff)>)+'static, { @@ -818,9 +816,9 @@ pub mod vec { } } - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, K: crate::ExchangeData+Hashable, R1: crate::ExchangeData+Semigroup { @@ -839,7 +837,7 @@ pub mod vec { /// .distinct(); /// }); /// ``` - pub fn distinct(self) -> Collection { + pub fn distinct(self) -> Collection { self.distinct_core() } @@ -848,7 +846,7 @@ pub mod vec { /// This method allows `distinct` to produce collections whose difference /// type is something other than an `isize` integer, for example perhaps an /// `i32`. - pub fn distinct_core>(self) -> Collection { + pub fn distinct_core>(self) -> Collection { self.threshold_named("Distinct", |_,_| R2::from(1i8)) } @@ -870,16 +868,16 @@ pub mod vec { /// .threshold(|_,c| c % 2); /// }); /// ``` - pub fn thresholdR2+'static>(self, thresh: F) -> Collection { + pub fn thresholdR2+'static>(self, thresh: F) -> Collection { self.threshold_named("Threshold", thresh) } /// A `threshold` with the ability to name the operator. - pub fn threshold_namedR2+'static>(self, name: &str, mut thresh: F) -> Collection { + pub fn threshold_namedR2+'static>(self, name: &str, mut thresh: F) -> Collection { use crate::trace::implementations::{KeyBuilder, KeySpine}; self.arrange_by_self_named(&format!("Arrange: {}", name)) - .reduce_abelian::<_,KeyBuilder,KeySpine,_>( + .reduce_abelian::<_,KeyBuilder,KeySpine,_>( name, move |k,s,t| t.push(((), thresh(k, &s[0].1))), |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, @@ -889,9 +887,9 @@ pub mod vec { } - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, K: crate::ExchangeData+Hashable, R: crate::ExchangeData+Semigroup { @@ -910,17 +908,17 @@ pub mod vec { /// .count(); /// }); /// ``` - pub fn count(self) -> Collection { self.count_core() } + pub fn count(self) -> Collection { self.count_core() } /// Count for general integer differences. /// /// This method allows `count` to produce collections whose difference /// type is something other than an `isize` integer, for example perhaps an /// `i32`. - pub fn count_core + 'static>(self) -> Collection { + pub fn count_core + 'static>(self) -> Collection { use crate::trace::implementations::{ValBuilder, ValSpine}; self.arrange_by_self_named("Arrange: Count") - .reduce_abelian::<_,ValBuilder,ValSpine,_>( + .reduce_abelian::<_,ValBuilder,ValSpine,_>( "Count", |_k,s,t| t.push((s[0].1.clone(), R2::from(1i8))), |vec, key, upds| { vec.clear(); vec.extend(upds.drain(..).map(|(v,t,r)| ((key.clone(), v),t,r))); }, @@ -930,9 +928,9 @@ pub mod vec { } /// Methods which require data be arrangeable. - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Clone + 'static + Lattice, D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Semigroup, { @@ -966,8 +964,8 @@ pub mod vec { /// and provide the function `reify` to produce owned keys and values.. pub fn consolidate_named(self, name: &str, reify: F) -> Self where - Ba: crate::trace::Batcher, Time=G::Timestamp> + 'static, - Tr: for<'a> crate::trace::Trace+'static, + Ba: crate::trace::Batcher, Time=T> + 'static, + Tr: for<'a> crate::trace::Trace+'static, Bu: crate::trace::Builder, F: Fn(Tr::Key<'_>, Tr::Val<'_>) -> D + 'static, { @@ -1025,82 +1023,82 @@ pub mod vec { use crate::trace::implementations::{KeySpine, KeyBatcher, KeyBuilder}; use crate::operators::arrange::Arrange; - impl Arrange> for Collection + impl Arrange> for Collection where - G: Scope, + T: Timestamp + Lattice, K: crate::ExchangeData + Hashable, V: crate::ExchangeData, R: crate::ExchangeData + Semigroup, { - fn arrange_named(self, name: &str) -> Arranged> + fn arrange_named(self, name: &str) -> Arranged> where - Ba: crate::trace::Batcher, Time=G::Timestamp> + 'static, - Bu: crate::trace::Builder, - Tr: crate::trace::Trace + 'static, + Ba: crate::trace::Batcher, Time=T> + 'static, + Bu: crate::trace::Builder, + Tr: crate::trace::Trace + 'static, { - let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,V),G::Timestamp,R)| (update.0).0.hashed().into()); - crate::operators::arrange::arrangement::arrange_core::<_, _, Ba, Bu, _>(self.inner, exchange, name) + let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,V),T,R)| (update.0).0.hashed().into()); + crate::operators::arrange::arrangement::arrange_core::<_, Ba, Bu, _>(self.inner, exchange, name) } } - impl Arrange> for Collection + impl Arrange> for Collection where - G: Scope, + T: Timestamp + Lattice + Ord, { - fn arrange_named(self, name: &str) -> Arranged> + fn arrange_named(self, name: &str) -> Arranged> where - Ba: crate::trace::Batcher, Time=G::Timestamp> + 'static, - Bu: crate::trace::Builder, - Tr: crate::trace::Trace + 'static, + Ba: crate::trace::Batcher, Time=T> + 'static, + Bu: crate::trace::Builder, + Tr: crate::trace::Trace + 'static, { - let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,()),G::Timestamp,R)| (update.0).0.hashed().into()); - crate::operators::arrange::arrangement::arrange_core::<_,_,Ba,Bu,_>(self.map(|k| (k, ())).inner, exchange, name) + let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,()),T,R)| (update.0).0.hashed().into()); + crate::operators::arrange::arrangement::arrange_core::<_,Ba,Bu,_>(self.map(|k| (k, ())).inner, exchange, name) } } - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, { /// Arranges a collection of `(Key, Val)` records by `Key`. /// /// This operator arranges a stream of values into a shared trace, whose contents it maintains. /// This trace is current for all times completed by the output stream, which can be used to /// safely identify the stable times and values in the trace. - pub fn arrange_by_key(self) -> Arranged>> { + pub fn arrange_by_key(self) -> Arranged>> { self.arrange_by_key_named("ArrangeByKey") } /// As `arrange_by_key` but with the ability to name the arrangement. - pub fn arrange_by_key_named(self, name: &str) -> Arranged>> { + pub fn arrange_by_key_named(self, name: &str) -> Arranged>> { self.arrange_named::,ValBuilder<_,_,_,_>,_>(name) } } - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, { /// Arranges a collection of `Key` records by `Key`. /// /// This operator arranges a collection of records into a shared trace, whose contents it maintains. /// This trace is current for all times complete in the output stream, which can be used to safely /// identify the stable times and values in the trace. - pub fn arrange_by_self(self) -> Arranged>> { + pub fn arrange_by_self(self) -> Arranged>> { self.arrange_by_self_named("ArrangeBySelf") } /// As `arrange_by_self` but with the ability to name the arrangement. - pub fn arrange_by_self_named(self, name: &str) -> Arranged>> { + pub fn arrange_by_self_named(self, name: &str) -> Arranged>> { self.map(|k| (k, ())) .arrange_named::,KeyBuilder<_,_,_>,_>(name) } } - impl Collection + impl Collection where - G: Scope, + T: Timestamp + Lattice + Ord, K: crate::ExchangeData+Hashable, V: crate::ExchangeData, R: crate::ExchangeData+Semigroup, @@ -1124,7 +1122,7 @@ pub mod vec { /// .assert_eq(z); /// }); /// ``` - pub fn join(self, other: Collection) -> Collection>::Output> + pub fn join(self, other: Collection) -> Collection>::Output> where K: crate::ExchangeData, V2: crate::ExchangeData, @@ -1151,7 +1149,7 @@ pub mod vec { /// .assert_eq(z); /// }); /// ``` - pub fn join_map(self, other: Collection, mut logic: L) -> Collection>::Output> + pub fn join_map(self, other: Collection, mut logic: L) -> Collection>::Output> where R: Multiply, L: FnMut(&K, &V, &V2)->D+'static { let arranged1 = self.arrange_by_key(); let arranged2 = other.arrange_by_key(); @@ -1179,7 +1177,7 @@ pub mod vec { /// .assert_eq(z); /// }); /// ``` - pub fn semijoin(self, other: Collection) -> Collection>::Output> + pub fn semijoin(self, other: Collection) -> Collection>::Output> where R: Multiply { let arranged1 = self.arrange_by_key(); let arranged2 = other.arrange_by_self(); @@ -1211,7 +1209,7 @@ pub mod vec { /// .assert_eq(z); /// }); /// ``` - pub fn antijoin(self, other: Collection) -> Collection + pub fn antijoin(self, other: Collection) -> Collection where R: Multiply, R: Abelian+'static { self.clone().concat(self.semijoin(other).negate()) } @@ -1222,7 +1220,7 @@ pub mod vec { /// which produces something implementing `IntoIterator`, where the output collection will have an entry for /// every value returned by the iterator. /// - /// This trait is implemented for arrangements (`Arranged`) rather than collections. The `Join` trait + /// This trait is implemented for arrangements (`Arranged`) rather than collections. The `Join` trait /// contains the implementations for collections. /// /// # Examples @@ -1244,9 +1242,9 @@ pub mod vec { /// .assert_eq(z); /// }); /// ``` - pub fn join_core (self, stream2: Arranged, result: L) -> Collection>::Output> + pub fn join_core (self, stream2: Arranged, result: L) -> Collection>::Output> where - Tr2: for<'a> crate::trace::TraceReader=&'a K, Time=G::Timestamp>+Clone+'static, + Tr2: for<'a> crate::trace::TraceReader=&'a K, Time=T>+Clone+'static, R: Multiply, I: IntoIterator, L: FnMut(&K,&V,Tr2::Val<'_>)->I+'static, @@ -1258,18 +1256,18 @@ pub mod vec { } /// Conversion to a differential dataflow Collection. -pub trait AsCollection { +pub trait AsCollection { /// Converts the type to a differential dataflow collection. - fn as_collection(self) -> Collection; + fn as_collection(self) -> Collection; } -impl AsCollection for Stream { +impl AsCollection for Stream { /// Converts the type to a differential dataflow collection. /// /// By calling this method, you guarantee that the timestamp invariant (as documented on /// [Collection]) is upheld. This method will not check it. - fn as_collection(self) -> Collection { - Collection::::new(self) + fn as_collection(self) -> Collection { + Collection::::new(self) } } @@ -1294,11 +1292,11 @@ impl AsCollection for Stream { /// .assert_eq(data); /// }); /// ``` -pub fn concatenate(scope: &mut G, iterator: I) -> Collection +pub fn concatenate(scope: &mut Scope, iterator: I) -> Collection where - G: Scope, + T: Timestamp, C: Container, - I: IntoIterator>, + I: IntoIterator>, { scope .concatenate(iterator.into_iter().map(|x| x.inner)) diff --git a/differential-dataflow/src/dynamic/mod.rs b/differential-dataflow/src/dynamic/mod.rs index 19c5472a6..9efe58f6c 100644 --- a/differential-dataflow/src/dynamic/mod.rs +++ b/differential-dataflow/src/dynamic/mod.rs @@ -13,7 +13,6 @@ pub mod pointstamp; -use timely::dataflow::Scope; use timely::order::Product; use timely::progress::Timestamp; use timely::dataflow::operators::generic::{OutputBuilder, builder_rc::OperatorBuilder}; @@ -26,9 +25,8 @@ use crate::collection::AsCollection; use crate::dynamic::pointstamp::PointStamp; use crate::dynamic::pointstamp::PointStampSummary; -impl VecCollection +impl VecCollection>, D, R> where - G: Scope>>, D: Data, R: Semigroup+'static, T: Timestamp+Default, diff --git a/differential-dataflow/src/input.rs b/differential-dataflow/src/input.rs index 43a4987e8..0ef606ac7 100644 --- a/differential-dataflow/src/input.rs +++ b/differential-dataflow/src/input.rs @@ -9,7 +9,7 @@ use timely::progress::Timestamp; use timely::dataflow::operators::vec::Input as TimelyInput; use timely::dataflow::operators::vec::input::Handle; -use timely::dataflow::scopes::ScopeParent; +use timely::dataflow::Scope; use crate::Data; use crate::difference::Semigroup; @@ -41,7 +41,7 @@ pub trait Input : TimelyInput { /// /// }).unwrap(); /// ``` - fn new_collection(&mut self) -> (InputSession<::Timestamp, D, R>, VecCollection) + fn new_collection(&mut self) -> (InputSession, VecCollection) where D: Data, R: Semigroup+'static; /// Create a new collection and input handle from initial data. /// @@ -67,7 +67,7 @@ pub trait Input : TimelyInput { /// /// }).unwrap(); /// ``` - fn new_collection_from(&mut self, data: I) -> (InputSession<::Timestamp, I::Item, isize>, VecCollection) + fn new_collection_from(&mut self, data: I) -> (InputSession, VecCollection) where I: IntoIterator + 'static; /// Create a new collection and input handle from initial data. /// @@ -93,28 +93,28 @@ pub trait Input : TimelyInput { /// /// }).unwrap(); /// ``` - fn new_collection_from_raw(&mut self, data: I) -> (InputSession<::Timestamp, D, R>, VecCollection) - where I: IntoIterator::Timestamp,R)>+'static, D: Data, R: Semigroup+'static; + fn new_collection_from_raw(&mut self, data: I) -> (InputSession, VecCollection) + where I: IntoIterator+'static, D: Data, R: Semigroup+'static; } use crate::lattice::Lattice; -impl Input for G where ::Timestamp: Lattice { - fn new_collection(&mut self) -> (InputSession<::Timestamp, D, R>, VecCollection) +impl Input for Scope { + fn new_collection(&mut self) -> (InputSession, VecCollection) where D: Data, R: Semigroup+'static, { let (handle, stream) = self.new_input(); (InputSession::from(handle), stream.as_collection()) } - fn new_collection_from(&mut self, data: I) -> (InputSession<::Timestamp, I::Item, isize>, VecCollection) + fn new_collection_from(&mut self, data: I) -> (InputSession, VecCollection) where I: IntoIterator+'static, I::Item: Data { - self.new_collection_from_raw(data.into_iter().map(|d| (d, ::minimum(), 1))) + self.new_collection_from_raw(data.into_iter().map(|d| (d, ::minimum(), 1))) } - fn new_collection_from_raw(&mut self, data: I) -> (InputSession<::Timestamp, D, R>, VecCollection) + fn new_collection_from_raw(&mut self, data: I) -> (InputSession, VecCollection) where D: Data, R: Semigroup+'static, - I: IntoIterator::Timestamp,R)>+'static, + I: IntoIterator+'static, { use timely::dataflow::operators::ToStream; @@ -122,7 +122,8 @@ impl Input for G where ::Timestamp: Lattice { let source = data.to_stream(self).as_collection(); (InputSession::from(handle), stream.as_collection().concat(source)) - }} + } +} /// An input session wrapping a single timely dataflow capability. /// @@ -198,9 +199,9 @@ impl InputSession { impl InputSession { /// Introduces a handle as collection. - pub fn to_collection(&mut self, scope: &mut G) -> VecCollection + pub fn to_collection(&mut self, scope: &mut Scope) -> VecCollection where - G: ScopeParent, + T: timely::order::TotalOrder, { scope .input_from(&mut self.handle) diff --git a/differential-dataflow/src/logging.rs b/differential-dataflow/src/logging.rs index 4fb6ef430..1ba9833f8 100644 --- a/differential-dataflow/src/logging.rs +++ b/differential-dataflow/src/logging.rs @@ -10,9 +10,8 @@ pub type DifferentialEventBuilder = timely::container::CapacityContainerBuilder< pub type Logger = ::timely::logging_core::TypedLogger; /// Enables logging of differential dataflow events. -pub fn enable(worker: &mut timely::worker::Worker, writer: W) -> Option> +pub fn enable(worker: &mut timely::worker::Worker, writer: W) -> Option> where - A: timely::communication::Allocate, W: std::io::Write + 'static, { worker.log_register().and_then(|mut log_register| { diff --git a/differential-dataflow/src/operators/arrange/agent.rs b/differential-dataflow/src/operators/arrange/agent.rs index 715542f55..7ae253d2c 100644 --- a/differential-dataflow/src/operators/arrange/agent.rs +++ b/differential-dataflow/src/operators/arrange/agent.rs @@ -9,6 +9,7 @@ use timely::dataflow::operators::generic::{OperatorInfo, source}; use timely::progress::Timestamp; use timely::progress::{Antichain, frontier::AntichainRef}; use timely::dataflow::operators::CapabilitySet; +use timely::scheduling::Scheduler; use crate::trace::{Trace, TraceReader, BatchReader}; use crate::trace::wrappers::rc::TraceBox; @@ -101,7 +102,7 @@ impl TraceAgent { }; let writer = TraceWriter::new( - vec![::minimum()], + vec![Tr::Time::minimum()], Rc::downgrade(&trace), queues, ); @@ -125,7 +126,7 @@ impl TraceAgent { .borrow_mut() .trace .map_batches(|batch| { - new_queue.push_back(TraceReplayInstruction::Batch(batch.clone(), Some(::minimum()))); + new_queue.push_back(TraceReplayInstruction::Batch(batch.clone(), Some(Tr::Time::minimum()))); upper = Some(batch.upper().clone()); }); @@ -215,17 +216,13 @@ impl TraceAgent { /// /// }).unwrap(); /// ``` - pub fn import(&mut self, scope: &G) -> Arranged> - where - G: Scope, + pub fn import(&mut self, scope: &Scope) -> Arranged> { self.import_named(scope, "ArrangedSource") } /// Same as `import`, but allows to name the source. - pub fn import_named(&mut self, scope: &G, name: &str) -> Arranged> - where - G: Scope, + pub fn import_named(&mut self, scope: &Scope, name: &str) -> Arranged> { // Drop ShutdownButton and return only the arrangement. self.import_core(scope, name).0 @@ -278,9 +275,7 @@ impl TraceAgent { /// /// }).unwrap(); /// ``` - pub fn import_core(&mut self, scope: &G, name: &str) -> (Arranged>, ShutdownButton>) - where - G: Scope, + pub fn import_core(&mut self, scope: &Scope, name: &str) -> (Arranged>, ShutdownButton>) { let trace = self.clone(); @@ -393,9 +388,8 @@ impl TraceAgent { /// /// }).unwrap(); /// ``` - pub fn import_frontier(&mut self, scope: &G, name: &str) -> (Arranged>>, ShutdownButton>) + pub fn import_frontier(&mut self, scope: &Scope, name: &str) -> (Arranged>>, ShutdownButton>) where - G: Scope, Tr: TraceReader, { // This frontier describes our only guarantee on the compaction frontier. @@ -411,9 +405,8 @@ impl TraceAgent { /// /// Invoking this method with an `until` of `Antichain::new()` will perform no filtering, as the empty /// frontier indicates the end of times. - pub fn import_frontier_core(&mut self, scope: &G, name: &str, since: Antichain, until: Antichain) -> (Arranged>>, ShutdownButton>) + pub fn import_frontier_core(&mut self, scope: &Scope, name: &str, since: Antichain, until: Antichain) -> (Arranged>>, ShutdownButton>) where - G: Scope, Tr: TraceReader, { let trace = self.clone(); diff --git a/differential-dataflow/src/operators/arrange/arrangement.rs b/differential-dataflow/src/operators/arrange/arrangement.rs index cd1bd0894..4034bb9a1 100644 --- a/differential-dataflow/src/operators/arrange/arrangement.rs +++ b/differential-dataflow/src/operators/arrange/arrangement.rs @@ -25,6 +25,8 @@ use timely::dataflow::channels::pact::{ParallelizationContract, Pipeline}; use timely::progress::Timestamp; use timely::progress::Antichain; use timely::dataflow::operators::Capability; +use timely::scheduling::Scheduler; +use timely::worker::AsWorker; use crate::{Data, VecCollection, AsCollection}; use crate::difference::Semigroup; @@ -41,9 +43,8 @@ use super::TraceAgent; /// /// An `Arranged` allows multiple differential operators to share the resources (communication, /// computation, memory) required to produce and maintain an indexed representation of a collection. -pub struct Arranged +pub struct Arranged where - G: Scope, Tr: TraceReader+Clone, { /// A stream containing arranged updates. @@ -51,16 +52,15 @@ where /// This stream contains the same batches of updates the trace itself accepts, so there should /// be no additional overhead to receiving these records. The batches can be navigated just as /// the batches in the trace, by key and by value. - pub stream: Stream>, + pub stream: Stream>, /// A shared trace, updated by the `Arrange` operator and readable by others. pub trace: Tr, // TODO : We might have an `Option>` here, which `as_collection` sets and // returns when invoked, so as to not duplicate work with multiple calls to `as_collection`. } -impl Clone for Arranged +impl Clone for Arranged where - G: Scope, Tr: TraceReader + Clone, { fn clone(&self) -> Self { @@ -71,13 +71,11 @@ where } } -use ::timely::dataflow::scopes::Child; use ::timely::progress::timestamp::Refines; use timely::Container; -impl Arranged +impl Arranged where - G: Scope, Tr: TraceReader + Clone, { /// Brings an arranged collection into a nested scope. @@ -85,10 +83,10 @@ where /// This method produces a proxy trace handle that uses the same backing data, but acts as if the timestamps /// have all been extended with an additional coordinate with the default value. The resulting collection does /// not vary with the new timestamp coordinate. - pub fn enter<'a, TInner>(self, child: &Child<'a, G, TInner>) - -> Arranged, TraceEnter> + pub fn enter(self, child: &Scope) + -> Arranged> where - TInner: Refines+Lattice+Timestamp+Clone, + TInner: Refines+Lattice, { Arranged { stream: self.stream.enter(child).map(|bw| BatchEnter::make_from(bw)), @@ -100,8 +98,7 @@ where /// /// This method only applies to *regions*, which are subscopes with the same timestamp /// as their containing scope. In this case, the trace type does not need to change. - pub fn enter_region<'a>(self, child: &Child<'a, G, G::Timestamp>) - -> Arranged, Tr> { + pub fn enter_region(self, child: &Scope) -> Self { Arranged { stream: self.stream.enter(child), trace: self.trace, @@ -113,10 +110,10 @@ where /// This method produces a proxy trace handle that uses the same backing data, but acts as if the timestamps /// have all been extended with an additional coordinate with the default value. The resulting collection does /// not vary with the new timestamp coordinate. - pub fn enter_at<'a, TInner, F, P>(self, child: &Child<'a, G, TInner>, logic: F, prior: P) - -> Arranged, TraceEnterAt> + pub fn enter_at(self, child: &Scope, logic: F, prior: P) + -> Arranged> where - TInner: Refines+Lattice+Timestamp+Clone+'static, + TInner: Refines+Lattice+'static, F: FnMut(Tr::Key<'_>, Tr::Val<'_>, Tr::TimeGat<'_>)->TInner+Clone+'static, P: FnMut(&TInner)->Tr::Time+Clone+'static, { @@ -132,7 +129,7 @@ where /// /// This method is like `self.stream.flat_map`, except that it produces containers /// directly, rather than form a container of containers as `flat_map` would. - pub fn as_container(self, mut logic: L) -> crate::Collection + pub fn as_container(self, mut logic: L) -> crate::Collection where I: IntoIterator, L: FnMut(Tr::Batch) -> I+'static, @@ -152,10 +149,10 @@ where /// Flattens the stream into a `VecCollection`. /// - /// The underlying `Stream>>` is a much more efficient way to access the data, + /// The underlying `Stream>>` is a much more efficient way to access the data, /// and this method should only be used when the data need to be transformed or exchanged, rather than /// supplied as arguments to an operator using the same key-value structure. - pub fn as_collection(self, mut logic: L) -> VecCollection + pub fn as_collection(self, mut logic: L) -> VecCollection where L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> D+'static, { @@ -164,14 +161,14 @@ where /// Flattens the stream into a `VecCollection`. /// - /// The underlying `Stream>>` is a much more efficient way to access the data, + /// The underlying `Stream>>` is a much more efficient way to access the data, /// and this method should only be used when the data need to be transformed or exchanged, rather than /// supplied as arguments to an operator using the same key-value structure. /// /// The method takes `K` and `V` as generic arguments, in order to constrain the reference types to support /// cloning into owned types. If this bound does not work, the `as_collection` method allows arbitrary logic /// on the reference types. - pub fn as_vecs(self) -> VecCollection + pub fn as_vecs(self) -> VecCollection where K: crate::ExchangeData, V: crate::ExchangeData, @@ -184,7 +181,7 @@ where /// /// The supplied logic may produce an iterator over output values, allowing either /// filtering or flat mapping as part of the extraction. - pub fn flat_map_ref(self, logic: L) -> VecCollection + pub fn flat_map_ref(self, logic: L) -> VecCollection where I: IntoIterator, L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> I+'static, @@ -199,7 +196,7 @@ where /// /// This method exists for streams of batches without the corresponding arrangement. /// If you have the arrangement, its `flat_map_ref` method is equivalent to this. - pub fn flat_map_batches(stream: Stream>, mut logic: L) -> VecCollection + pub fn flat_map_batches(stream: Stream>, mut logic: L) -> VecCollection where I: IntoIterator, L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> I+'static, @@ -231,29 +228,28 @@ where use crate::difference::Multiply; // Direct join implementations. -impl Arranged +impl Arranged where - G: Scope, - T1: TraceReader + Clone + 'static, + Tr1: TraceReader + Clone + 'static, { /// A convenience method to join and produce `VecCollection` output. /// /// Avoid this method, as it is likely to evolve into one without the `VecCollection` opinion. - pub fn join_core(self, other: Arranged, mut result: L) -> VecCollection>::Output> + pub fn join_core(self, other: Arranged, mut result: L) -> VecCollection>::Output> where - T2: for<'a> TraceReader=T1::Key<'a>,Time=T1::Time>+Clone+'static, - T1::Diff: Multiply, + Tr2: for<'a> TraceReader=Tr1::Key<'a>,Time=Tr1::Time>+Clone+'static, + Tr1::Diff: Multiply, I: IntoIterator, - L: FnMut(T1::Key<'_>,T1::Val<'_>,T2::Val<'_>)->I+'static + L: FnMut(Tr1::Key<'_>,Tr1::Val<'_>,Tr2::Val<'_>)->I+'static { - let mut result = move |k: T1::Key<'_>, v1: T1::Val<'_>, v2: T2::Val<'_>, t: &G::Timestamp, r1: &T1::Diff, r2: &T2::Diff| { + let mut result = move |k: Tr1::Key<'_>, v1: Tr1::Val<'_>, v2: Tr2::Val<'_>, t: &Tr1::Time, r1: &Tr1::Diff, r2: &Tr2::Diff| { let t = t.clone(); let r = (r1.clone()).multiply(r2); result(k, v1, v2).into_iter().map(move |d| (d, t.clone(), r.clone())) }; use crate::operators::join::join_traces; - join_traces::<_, _, _, _, crate::consolidation::ConsolidatingContainerBuilder<_>>( + join_traces::<_, _, _, crate::consolidation::ConsolidatingContainerBuilder<_>>( self, other, move |k, v1, v2, t, d1, d2, c| { @@ -268,26 +264,24 @@ where // Direct reduce implementations. use crate::difference::Abelian; -impl Arranged +impl Arranged where - G: Scope, - T1: TraceReader + Clone + 'static, + Tr1: TraceReader + Clone + 'static, { /// A direct implementation of `ReduceCore::reduce_abelian`. - pub fn reduce_abelian(self, name: &str, mut logic: L, push: P) -> Arranged> + pub fn reduce_abelian(self, name: &str, mut logic: L, push: P) -> Arranged> where - T1: TraceReader, - T2: for<'a> Trace< - Key<'a>= T1::Key<'a>, + Tr2: for<'a> Trace< + Key<'a>= Tr1::Key<'a>, ValOwn: Data, - Time=T1::Time, + Time=Tr1::Time, Diff: Abelian, >+'static, - Bu: Builder, - L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, - P: FnMut(&mut Bu::Input, T1::Key<'_>, &mut Vec<(T2::ValOwn, T2::Time, T2::Diff)>) + 'static, + Bu: Builder, + L: FnMut(Tr1::Key<'_>, &[(Tr1::Val<'_>, Tr1::Diff)], &mut Vec<(Tr2::ValOwn, Tr2::Diff)>)+'static, + P: FnMut(&mut Bu::Input, Tr1::Key<'_>, &mut Vec<(Tr2::ValOwn, Tr2::Time, Tr2::Diff)>) + 'static, { - self.reduce_core::<_,Bu,T2,_>(name, move |key, input, output, change| { + self.reduce_core::<_,Bu,Tr2,_>(name, move |key, input, output, change| { if !input.is_empty() { logic(key, input, change); } @@ -297,63 +291,62 @@ where } /// A direct implementation of `ReduceCore::reduce_core`. - pub fn reduce_core(self, name: &str, logic: L, push: P) -> Arranged> + pub fn reduce_core(self, name: &str, logic: L, push: P) -> Arranged> where - T1: TraceReader, - T2: for<'a> Trace< - Key<'a>=T1::Key<'a>, + Tr2: for<'a> Trace< + Key<'a>=Tr1::Key<'a>, ValOwn: Data, - Time=T1::Time, + Time=Tr1::Time, >+'static, - Bu: Builder, - L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn, T2::Diff)>, &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, - P: FnMut(&mut Bu::Input, T1::Key<'_>, &mut Vec<(T2::ValOwn, T2::Time, T2::Diff)>) + 'static, + Bu: Builder, + L: FnMut(Tr1::Key<'_>, &[(Tr1::Val<'_>, Tr1::Diff)], &mut Vec<(Tr2::ValOwn, Tr2::Diff)>, &mut Vec<(Tr2::ValOwn, Tr2::Diff)>)+'static, + P: FnMut(&mut Bu::Input, Tr1::Key<'_>, &mut Vec<(Tr2::ValOwn, Tr2::Time, Tr2::Diff)>) + 'static, { use crate::operators::reduce::reduce_trace; - reduce_trace::<_,_,Bu,_,_,_>(self, name, logic, push) + reduce_trace::<_,Bu,_,_,_>(self, name, logic, push) } } -impl<'a, G, Tr> Arranged, Tr> +impl Arranged where - G: Scope, Tr: TraceReader + Clone, { /// Brings an arranged collection out of a nested region. /// /// This method only applies to *regions*, which are subscopes with the same timestamp /// as their containing scope. In this case, the trace type does not need to change. - pub fn leave_region(self) -> Arranged { + pub fn leave_region(self, outer: &Scope) -> Self + { use timely::dataflow::operators::Leave; Arranged { - stream: self.stream.leave(), + stream: self.stream.leave(outer), trace: self.trace, } } } /// A type that can be arranged as if a collection of updates. -pub trait Arrange : Sized +pub trait Arrange : Sized where - G: Scope, + T: Timestamp + Lattice, { /// Arranges updates into a shared trace. - fn arrange(self) -> Arranged> + fn arrange(self) -> Arranged> where - Ba: Batcher + 'static, - Bu: Builder, - Tr: Trace + 'static, + Ba: Batcher + 'static, + Bu: Builder, + Tr: Trace + 'static, { self.arrange_named::("Arrange") } /// Arranges updates into a shared trace, with a supplied name. - fn arrange_named(self, name: &str) -> Arranged> + fn arrange_named(self, name: &str) -> Arranged> where - Ba: Batcher + 'static, - Bu: Builder, - Tr: Trace + 'static, + Ba: Batcher + 'static, + Bu: Builder, + Tr: Trace + 'static, ; } @@ -362,13 +355,12 @@ where /// This operator arranges a stream of values into a shared trace, whose contents it maintains. /// It uses the supplied parallelization contract to distribute the data, which does not need to /// be consistently by key (though this is the most common). -pub fn arrange_core(stream: Stream, pact: P, name: &str) -> Arranged> +pub fn arrange_core(stream: Stream, pact: P, name: &str) -> Arranged> where - G: Scope, - P: ParallelizationContract, - Ba: Batcher + 'static, - Bu: Builder, - Tr: Trace+'static, + P: ParallelizationContract, + Ba: Batcher + 'static, + Bu: Builder, + Tr: Trace+'static, { // The `Arrange` operator is tasked with reacting to an advancing input // frontier by producing the sequence of batches whose lower and upper @@ -400,7 +392,7 @@ where let mut batcher = Ba::new(logger.clone(), info.global_id); // Capabilities for the lower envelope of updates in `batcher`. - let mut capabilities = Antichain::>::new(); + let mut capabilities = Antichain::>::new(); let activator = Some(scope.activator_for(info.address.clone())); let mut empty_trace = Tr::new(info.clone(), logger.clone(), activator); @@ -414,7 +406,7 @@ where *reader_ref = Some(reader_local); // Initialize to the minimal input frontier. - let mut prev_frontier = Antichain::from_elem(::minimum()); + let mut prev_frontier = Antichain::from_elem(Tr::Time::minimum()); move |(input, frontier), output| { diff --git a/differential-dataflow/src/operators/arrange/upsert.rs b/differential-dataflow/src/operators/arrange/upsert.rs index a8322eaa3..33b62a0e8 100644 --- a/differential-dataflow/src/operators/arrange/upsert.rs +++ b/differential-dataflow/src/operators/arrange/upsert.rs @@ -59,7 +59,7 @@ //! use differential_dataflow::operators::arrange::upsert; //! //! let stream = scope.input_from(&mut input); -//! let arranged = upsert::arrange_from_upsert::<_, ValBuilder, ValSpine,String,String>(stream, &"test"); +//! let arranged = upsert::arrange_from_upsert::, ValSpine,String,String>(stream, &"test"); //! //! arranged //! .as_collection(|k,v| (k.clone(), v.clone())) @@ -101,12 +101,13 @@ use std::collections::{BinaryHeap, BTreeMap}; use timely::order::{PartialOrder, TotalOrder}; -use timely::dataflow::{Scope, Stream}; +use timely::dataflow::Stream; use timely::dataflow::operators::generic::Operator; use timely::dataflow::channels::pact::Exchange; -use timely::progress::Timestamp; -use timely::progress::Antichain; +use timely::progress::{Antichain, Timestamp}; use timely::dataflow::operators::Capability; +use timely::scheduling::Scheduler; +use timely::worker::AsWorker; use crate::operators::arrange::arrangement::Arranged; use crate::trace::{Builder, Description}; @@ -127,12 +128,11 @@ use super::TraceAgent; /// This method is only implemented for totally ordered times, as we do not yet /// understand what a "sequence" of upserts would mean for partially ordered /// timestamps. -pub fn arrange_from_upsert( - stream: Stream, G::Timestamp)>>, +pub fn arrange_from_upsert( + stream: Stream, Tr::Time)>>, name: &str, -) -> Arranged> +) -> Arranged> where - G: Scope, K: ExchangeData+Hashable+std::hash::Hash, V: ExchangeData, Tr: for<'a> Trace< @@ -141,7 +141,7 @@ where Time: TotalOrder+ExchangeData, Diff=isize, >+'static, - Bu: Builder, Output = Tr::Batch>, + Bu: Builder, Output = Tr::Batch>, { let mut reader: Option> = None; @@ -150,7 +150,7 @@ where let reader = &mut reader; - let exchange = Exchange::new(move |update: &(K,Option,G::Timestamp)| (update.0).hashed().into()); + let exchange = Exchange::new(move |update: &(K,Option,Tr::Time)| (update.0).hashed().into()); let scope = stream.scope(); stream.unary_frontier(exchange, name, move |_capability, info| { @@ -159,7 +159,7 @@ where let logger = scope.logger_for::("differential/arrange").map(Into::into); // Tracks the lower envelope of times in `priority_queue`. - let mut capabilities = Antichain::>::new(); + let mut capabilities = Antichain::>::new(); // Form the trace we will both use internally and publish. let activator = Some(scope.activator_for(info.address.clone())); let mut empty_trace = Tr::new(info.clone(), logger.clone(), activator); @@ -173,10 +173,10 @@ where *reader = Some(reader_local.clone()); // Tracks the input frontier, used to populate the lower bound of new batches. - let mut prev_frontier = Antichain::from_elem(::minimum()); + let mut prev_frontier = Antichain::from_elem(Tr::Time::minimum()); // For stashing input upserts, ordered increasing by time (`BinaryHeap` is a max-heap). - let mut priority_queue = BinaryHeap::)>>::new(); + let mut priority_queue = BinaryHeap::)>>::new(); let mut updates = Vec::new(); move |(input, frontier), output| { @@ -279,7 +279,7 @@ where updates.sort(); builder.push(&mut updates); } - let description = Description::new(prev_frontier.clone(), upper.clone(), Antichain::from_elem(G::Timestamp::minimum())); + let description = Description::new(prev_frontier.clone(), upper.clone(), Antichain::from_elem(Tr::Time::minimum())); let batch = builder.done(description); prev_frontier.clone_from(&upper); diff --git a/differential-dataflow/src/operators/count.rs b/differential-dataflow/src/operators/count.rs index 713c73c86..8f4e7ad7a 100644 --- a/differential-dataflow/src/operators/count.rs +++ b/differential-dataflow/src/operators/count.rs @@ -1,7 +1,7 @@ //! Count the number of occurrences of each element. use timely::order::TotalOrder; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::Operator; use timely::dataflow::channels::pact::Pipeline; @@ -14,7 +14,7 @@ use crate::operators::arrange::Arranged; use crate::trace::{BatchReader, Cursor, TraceReader}; /// Extension trait for the `count` differential dataflow method. -pub trait CountTotal, K: ExchangeData, R: Semigroup> : Sized { +pub trait CountTotal : Sized { /// Counts the number of occurrences of each element. /// /// # Examples @@ -30,7 +30,7 @@ pub trait CountTotal, K: ExchangeDat /// .count_total(); /// }); /// ``` - fn count_total(self) -> VecCollection { + fn count_total(self) -> VecCollection { self.count_total_core() } @@ -39,39 +39,38 @@ pub trait CountTotal, K: ExchangeDat /// This method allows `count_total` to produce collections whose difference /// type is something other than an `isize` integer, for example perhaps an /// `i32`. - fn count_total_core + 'static>(self) -> VecCollection; + fn count_total_core + 'static>(self) -> VecCollection; } -impl CountTotal for VecCollection +impl CountTotal for VecCollection where - G: Scope, + T: Timestamp + TotalOrder + Lattice + Ord, { - fn count_total_core + 'static>(self) -> VecCollection { + fn count_total_core + 'static>(self) -> VecCollection { self.arrange_by_self_named("Arrange: CountTotal") .count_total_core() } } -impl CountTotal for Arranged +impl CountTotal for Arranged where - G: Scope, - T1: for<'a> TraceReader< + Tr: for<'a> TraceReader< Key<'a> = &'a K, Val<'a>=&'a (), Time: TotalOrder, - Diff: ExchangeData+Semigroup> + Diff: ExchangeData+Semigroup> >+Clone+'static, K: ExchangeData, { - fn count_total_core + 'static>(self) -> VecCollection { + fn count_total_core + 'static>(self) -> VecCollection { let mut trace = self.trace.clone(); self.stream.unary_frontier(Pipeline, "CountTotal", move |_,_| { // tracks the lower and upper limit of received batches. - let mut lower_limit = timely::progress::frontier::Antichain::from_elem(::minimum()); - let mut upper_limit = timely::progress::frontier::Antichain::from_elem(::minimum()); + let mut lower_limit = timely::progress::frontier::Antichain::from_elem(Tr::Time::minimum()); + let mut upper_limit = timely::progress::frontier::Antichain::from_elem(Tr::Time::minimum()); move |(input, _frontier), output| { @@ -103,13 +102,13 @@ where let (mut trace_cursor, trace_storage) = trace.cursor_through(lower_limit.borrow()).unwrap(); while let Some(key) = batch_cursor.get_key(&batch_storage) { - let mut count: Option = None; + let mut count: Option = None; trace_cursor.seek_key(&trace_storage, key); if trace_cursor.get_key(&trace_storage) == Some(key) { trace_cursor.map_times(&trace_storage, |_, diff| { count.as_mut().map(|c| c.plus_equals(&diff)); - if count.is_none() { count = Some(T1::owned_diff(diff)); } + if count.is_none() { count = Some(Tr::owned_diff(diff)); } }); } @@ -117,14 +116,14 @@ where if let Some(count) = count.as_ref() { if !count.is_zero() { - session.give(((key.clone(), count.clone()), T1::owned_time(time), R2::from(-1i8))); + session.give(((key.clone(), count.clone()), Tr::owned_time(time), R2::from(-1i8))); } } count.as_mut().map(|c| c.plus_equals(&diff)); - if count.is_none() { count = Some(T1::owned_diff(diff)); } + if count.is_none() { count = Some(Tr::owned_diff(diff)); } if let Some(count) = count.as_ref() { if !count.is_zero() { - session.give(((key.clone(), count.clone()), T1::owned_time(time), R2::from(1i8))); + session.give(((key.clone(), count.clone()), Tr::owned_time(time), R2::from(1i8))); } } }); diff --git a/differential-dataflow/src/operators/iterate.rs b/differential-dataflow/src/operators/iterate.rs index e5486bb6e..140e8888a 100644 --- a/differential-dataflow/src/operators/iterate.rs +++ b/differential-dataflow/src/operators/iterate.rs @@ -37,7 +37,7 @@ use timely::progress::Timestamp; use timely::order::Product; use timely::dataflow::*; -use timely::dataflow::scopes::child::Iterative; +use timely::dataflow::scope::Iterative; use timely::dataflow::operators::{Feedback, ConnectLoop}; use timely::dataflow::operators::feedback::Handle; @@ -46,7 +46,7 @@ use crate::difference::{Semigroup, Abelian}; use crate::lattice::Lattice; /// An extension trait for the `iterate` method. -pub trait Iterate, D: Data, R: Semigroup> { +pub trait Iterate { /// Iteratively apply `logic` to the source collection until convergence. /// /// Importantly, this method does not automatically consolidate results. @@ -73,17 +73,18 @@ pub trait Iterate, D: Data, R: Semigroup> { /// }); /// }); /// ``` - fn iterate(self, logic: F) -> VecCollection + fn iterate(self, logic: F) -> VecCollection where - for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>; + for<'a> F: FnOnce(Iterative, VecCollection, D, R>)->VecCollection, D, R>; } -impl, D: Ord+Data+Debug, R: Abelian+'static> Iterate for VecCollection { - fn iterate(self, logic: F) -> VecCollection +impl Iterate for VecCollection { + fn iterate(self, logic: F) -> VecCollection where - for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>, + for<'a> F: FnOnce(Iterative, VecCollection, D, R>)->VecCollection, D, R>, { - self.inner.scope().scoped("Iterate", |subgraph| { + let outer = self.inner.scope(); + outer.scoped("Iterate", |subgraph| { // create a new variable, apply logic, bind variable, return. // // this could be much more succinct if we returned the collection @@ -93,16 +94,17 @@ impl, D: Ord+Data+Debug, R: Abelian+'static> Iterat let (variable, collection) = Variable::new_from(self.enter(subgraph), Product::new(Default::default(), 1)); let result = logic(subgraph.clone(), collection); variable.set(result.clone()); - result.leave() + result.leave(&outer) }) } } -impl, D: Ord+Data+Debug, R: Semigroup+'static> Iterate for G { - fn iterate(mut self, logic: F) -> VecCollection +impl Iterate for Scope { + fn iterate(self, logic: F) -> VecCollection where - for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>, + for<'a> F: FnOnce(Iterative, VecCollection, D, R>)->VecCollection, D, R>, { + let outer = self.clone(); self.scoped("Iterate", |subgraph| { // create a new variable, apply logic, bind variable, return. // @@ -113,7 +115,7 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter let (variable, collection) = Variable::new(subgraph, Product::new(Default::default(), 1)); let result = logic(subgraph.clone(), collection); variable.set(result.clone()); - result.leave() + result.leave(&outer) } ) } @@ -146,7 +148,7 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter /// let result = collection.map(|x| if x % 2 == 0 { x/2 } else { x }) /// .consolidate(); /// variable.set(result.clone()); -/// result.leave() +/// result.leave(&scope) /// }); /// }) /// ``` @@ -187,23 +189,23 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter /// By iteratively developing a variable of the *edits* to the input, we can produce and circulate /// a smaller volume of updates. This can be especially impactful when the initial collection is /// large, and the edits to perform are relatively smaller. -pub struct Variable +pub struct Variable where - G: Scope, + T: Timestamp + Lattice, C: Container, { - feedback: Handle, - source: Option>, - step: ::Summary, + feedback: Handle, + source: Option>, + step: T::Summary, } /// A `Variable` specialized to a vector container of update triples (data, time, diff). -pub type VecVariable = Variable::Timestamp, R)>>; +pub type VecVariable = Variable>; -impl Variable +impl Variable where - G: Scope, - C: crate::collection::containers::ResultsIn<::Summary>, + T: Timestamp + Lattice, + C: crate::collection::containers::ResultsIn, { /// Creates a new initially empty `Variable` and its associated `Collection`. /// @@ -216,9 +218,9 @@ where /// will produce its fixed point in the outer scope. /// /// In a non-iterative scope the mechanics are the same, but the interpretation varies. - pub fn new(scope: &mut G, step: ::Summary) -> (Self, Collection) { + pub fn new(scope: &mut Scope, step: T::Summary) -> (Self, Collection) { let (feedback, updates) = scope.feedback(step.clone()); - let collection = Collection::::new(updates); + let collection = Collection::::new(updates); (Self { feedback, source: None, step }, collection) } @@ -247,9 +249,9 @@ where /// adding the source, doing the logic, then subtracting the source, it is appropriate to do. /// For example, if the logic modifies a few records it is possible to produce this update /// directly without using the backstop implementation this method provides. - pub fn new_from(source: Collection, step: ::Summary) -> (Self, Collection) where C: Clone + crate::collection::containers::Negate { + pub fn new_from(source: Collection, step: T::Summary) -> (Self, Collection) where C: Clone + crate::collection::containers::Negate { let (feedback, updates) = source.inner.scope().feedback(step.clone()); - let collection = Collection::::new(updates).concat(source.clone()); + let collection = Collection::::new(updates).concat(source.clone()); (Variable { feedback, source: Some(source.negate()), step }, collection) } @@ -257,7 +259,7 @@ where /// /// This method binds the `Variable` to be equal to the supplied collection, /// which may be recursively defined in terms of the variable itself. - pub fn set(mut self, mut result: Collection) { + pub fn set(mut self, mut result: Collection) { if let Some(source) = self.source.take() { result = result.concat(source); } diff --git a/differential-dataflow/src/operators/join.rs b/differential-dataflow/src/operators/join.rs index dae4d31b3..7d514cfec 100644 --- a/differential-dataflow/src/operators/join.rs +++ b/differential-dataflow/src/operators/join.rs @@ -9,10 +9,11 @@ use timely::{Accountable, ContainerBuilder}; use timely::container::PushInto; use timely::order::PartialOrder; use timely::progress::Timestamp; -use timely::dataflow::{Scope, Stream}; +use timely::dataflow::Stream; use timely::dataflow::operators::generic::{Operator, OutputBuilderSession, Session}; use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::Capability; +use timely::scheduling::Scheduler; use crate::lattice::Lattice; use crate::operators::arrange::Arranged; @@ -66,12 +67,11 @@ impl, D> PushInto for EffortBuilder { /// The "correctness" of this method depends heavily on the behavior of the supplied `result` function. /// /// [`AsCollection`]: crate::collection::AsCollection -pub fn join_traces(arranged1: Arranged, arranged2: Arranged, mut result: L) -> Stream +pub fn join_traces(arranged1: Arranged, arranged2: Arranged, mut result: L) -> Stream where - G: Scope, - T1: TraceReader+Clone+'static, - T2: for<'a> TraceReader=T1::Key<'a>, Time=T1::Time>+Clone+'static, - L: FnMut(T1::Key<'_>,T1::Val<'_>,T2::Val<'_>,&G::Timestamp,&T1::Diff,&T2::Diff,&mut JoinSession>)+'static, + Tr1: TraceReader+Clone+'static, + Tr2: for<'a> TraceReader=Tr1::Key<'a>, Time = Tr1::Time>+Clone+'static, + L: FnMut(Tr1::Key<'_>,Tr1::Val<'_>,Tr2::Val<'_>,&Tr1::Time,&Tr1::Diff,&Tr2::Diff,&mut JoinSession>)+'static, CB: ContainerBuilder, { // Rename traces for symmetry from here on out. @@ -97,8 +97,8 @@ where // the physical compaction frontier of their corresponding trace. // Should we ever *drop* a trace, these are 1. much harder to maintain correctly, but 2. no longer used. use timely::progress::frontier::Antichain; - let mut acknowledged1 = Antichain::from_elem(::minimum()); - let mut acknowledged2 = Antichain::from_elem(::minimum()); + let mut acknowledged1 = Antichain::from_elem(Tr1::Time::minimum()); + let mut acknowledged2 = Antichain::from_elem(Tr1::Time::minimum()); // deferred work of batches from each input. let mut todo1 = std::collections::VecDeque::new(); diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 909199295..4fcaefd08 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -9,9 +9,10 @@ use crate::Data; use timely::progress::frontier::Antichain; use timely::progress::Timestamp; -use timely::dataflow::*; use timely::dataflow::operators::Operator; use timely::dataflow::channels::pact::Pipeline; +use timely::scheduling::Scheduler; +use timely::worker::AsWorker; use crate::operators::arrange::{Arranged, TraceAgent}; use crate::trace::{BatchReader, Cursor, Trace, Builder, ExertionLogic, Description}; @@ -32,14 +33,13 @@ use crate::trace::TraceReader; /// the value updates, as appropriate for the container. It is critical that it clear the container as /// the operator has no ability to do this otherwise, and failing to do so represents a leak from one /// key's computation to another, and will likely introduce non-determinism. -pub fn reduce_trace(trace: Arranged, name: &str, mut logic: L, mut push: P) -> Arranged> +pub fn reduce_trace(trace: Arranged, name: &str, mut logic: L, mut push: P) -> Arranged> where - G: Scope, - T1: TraceReader + Clone + 'static, - T2: for<'a> Trace=T1::Key<'a>, ValOwn: Data, Time=T1::Time> + 'static, - Bu: Builder, - L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn,T2::Diff)>, &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, - P: FnMut(&mut Bu::Input, T1::Key<'_>, &mut Vec<(T2::ValOwn, T2::Time, T2::Diff)>) + 'static, + Tr1: TraceReader + Clone + 'static, + Tr2: for<'a> Trace=Tr1::Key<'a>, ValOwn: Data, Time = Tr1::Time> + 'static, + Bu: Builder, + L: FnMut(Tr1::Key<'_>, &[(Tr1::Val<'_>, Tr1::Diff)], &mut Vec<(Tr2::ValOwn,Tr2::Diff)>, &mut Vec<(Tr2::ValOwn, Tr2::Diff)>)+'static, + P: FnMut(&mut Bu::Input, Tr1::Key<'_>, &mut Vec<(Tr2::ValOwn, Tr2::Time, Tr2::Diff)>) + 'static, { let mut result_trace = None; @@ -54,7 +54,7 @@ where let logger = scope.logger_for::("differential/arrange").map(Into::into); let activator = Some(scope.activator_for(operator_info.address.clone())); - let mut empty = T2::new(operator_info.clone(), logger.clone(), activator); + let mut empty = Tr2::new(operator_info.clone(), logger.clone(), activator); // If there is default exert logic set, install it. if let Some(exert_logic) = scope.config().get::("differential/default_exert_logic").cloned() { empty.set_exert_logic(exert_logic); @@ -66,26 +66,26 @@ where *result_trace = Some(output_reader.clone()); - let mut new_interesting_times = Vec::::new(); + let mut new_interesting_times = Vec::::new(); // Our implementation maintains a list of outstanding `(key, time)` synthetic interesting times, // sorted by (key, time), as well as capabilities for the lower envelope of the times. - let mut pending_keys = T1::KeyContainer::with_capacity(0); - let mut pending_time = T1::TimeContainer::with_capacity(0); - let mut next_pending_keys = T1::KeyContainer::with_capacity(0); - let mut next_pending_time = T1::TimeContainer::with_capacity(0); - let mut capabilities = timely::dataflow::operators::CapabilitySet::::default(); + let mut pending_keys = Tr1::KeyContainer::with_capacity(0); + let mut pending_time = Tr1::TimeContainer::with_capacity(0); + let mut next_pending_keys = Tr1::KeyContainer::with_capacity(0); + let mut next_pending_time = Tr1::TimeContainer::with_capacity(0); + let mut capabilities = timely::dataflow::operators::CapabilitySet::::default(); // buffers and logic for computing per-key interesting times "efficiently". - let mut interesting_times = Vec::::new(); + let mut interesting_times = Vec::::new(); // Upper and lower frontiers for the pending input and output batches to process. - let mut upper_limit = Antichain::from_elem(::minimum()); - let mut lower_limit = Antichain::from_elem(::minimum()); + let mut upper_limit = Antichain::from_elem(::minimum()); + let mut lower_limit = Antichain::from_elem(::minimum()); // Output batches may need to be built piecemeal, and these temp storage help there. - let mut output_upper = Antichain::from_elem(::minimum()); - let mut output_lower = Antichain::from_elem(::minimum()); + let mut output_upper = Antichain::from_elem(::minimum()); + let mut output_lower = Antichain::from_elem(::minimum()); move |(input, frontier), output| { @@ -130,14 +130,14 @@ where if capabilities.iter().any(|c| !upper_limit.less_equal(c.time())) { // cursors for navigating input and output traces. - let (mut source_cursor, ref source_storage): (T1::Cursor, _) = source_trace.cursor_through(lower_limit.borrow()).expect("failed to acquire source cursor"); - let (mut output_cursor, ref output_storage): (T2::Cursor, _) = output_reader.cursor_through(lower_limit.borrow()).expect("failed to acquire output cursor"); + let (mut source_cursor, ref source_storage): (Tr1::Cursor, _) = source_trace.cursor_through(lower_limit.borrow()).expect("failed to acquire source cursor"); + let (mut output_cursor, ref output_storage): (Tr2::Cursor, _) = output_reader.cursor_through(lower_limit.borrow()).expect("failed to acquire output cursor"); let (mut batch_cursor, ref batch_storage) = (CursorList::new(batch_cursors, &batch_storage), batch_storage); // Prepare an output buffer and builder for each capability. // TODO: It would be better if all updates went into one batch, but timely dataflow prevents // this as long as it requires that there is only one capability for each message. - let mut buffers = Vec::<(G::Timestamp, Vec<(T2::ValOwn, G::Timestamp, T2::Diff)>)>::new(); + let mut buffers = Vec::<(Tr1::Time, Vec<(Tr2::ValOwn, Tr1::Time, Tr2::Diff)>)>::new(); let mut builders = Vec::new(); for cap in capabilities.iter() { buffers.push((cap.time().clone(), Vec::new())); @@ -171,7 +171,7 @@ where let prior_pos = pending_pos; interesting_times.clear(); while pending_keys.get(pending_pos) == Some(key) { - let owned_time = T1::owned_time(pending_time.index(pending_pos)); + let owned_time = Tr1::owned_time(pending_time.index(pending_pos)); if !upper_limit.less_equal(&owned_time) { interesting_times.push(owned_time); } pending_pos += 1; } @@ -201,7 +201,7 @@ where // Merge novel pending times with any prior pending times we did not process. // TODO: This could be a merge, not a sort_dedup, because both lists should be sorted. for pos in prior_pos .. pending_pos { - let owned_time = T1::owned_time(pending_time.index(pos)); + let owned_time = Tr1::owned_time(pending_time.index(pos)); if upper_limit.less_equal(&owned_time) { new_interesting_times.push(owned_time); } } sort_dedup(&mut new_interesting_times); @@ -252,7 +252,7 @@ where if output_upper.borrow() != output_lower.borrow() { - let description = Description::new(output_lower.clone(), output_upper.clone(), Antichain::from_elem(G::Timestamp::minimum())); + let description = Description::new(output_lower.clone(), output_upper.clone(), Antichain::from_elem(Tr1::Time::minimum())); let batch = builder.done(description); // ship batch to the output, and commit to the output trace. @@ -272,10 +272,10 @@ where pending_time.clear(); std::mem::swap(&mut next_pending_time, &mut pending_time); // Update `capabilities` to reflect pending times. - let mut frontier = Antichain::::new(); - let mut owned_time = T1::Time::minimum(); + let mut frontier = Antichain::::new(); + let mut owned_time = Tr1::Time::minimum(); for pos in 0 .. pending_time.len() { - T1::clone_time_onto(pending_time.index(pos), &mut owned_time); + Tr1::clone_time_onto(pending_time.index(pos), &mut owned_time); frontier.insert_ref(&owned_time); } capabilities.downgrade(frontier); diff --git a/differential-dataflow/src/operators/threshold.rs b/differential-dataflow/src/operators/threshold.rs index 91bbd16fe..22dfec32b 100644 --- a/differential-dataflow/src/operators/threshold.rs +++ b/differential-dataflow/src/operators/threshold.rs @@ -4,7 +4,7 @@ //! `distinct` and `distinct_u` operators for the case in which time is totally ordered. use timely::order::TotalOrder; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::Operator; use timely::dataflow::channels::pact::Pipeline; @@ -17,9 +17,9 @@ use crate::operators::arrange::Arranged; use crate::trace::{BatchReader, Cursor, TraceReader}; /// Extension trait for the `distinct` differential dataflow method. -pub trait ThresholdTotal, K: ExchangeData, R: ExchangeData+Semigroup> : Sized { +pub trait ThresholdTotal : Sized { /// Reduces the collection to one occurrence of each distinct element. - fn threshold_semigroup(self, thresh: F) -> VecCollection + fn threshold_semigroup(self, thresh: F) -> VecCollection where R2: Semigroup+'static, F: FnMut(&K,&R,Option<&R>)->Option+'static, @@ -39,7 +39,7 @@ pub trait ThresholdTotal, K: Exchang /// .threshold_total(|_,c| c % 2); /// }); /// ``` - fn threshold_totalR2+'static>(self, mut thresh: F) -> VecCollection { + fn threshold_totalR2+'static>(self, mut thresh: F) -> VecCollection { self.threshold_semigroup(move |key, new, old| { let mut new = thresh(key, new); if let Some(old) = old { @@ -69,7 +69,7 @@ pub trait ThresholdTotal, K: Exchang /// .distinct_total(); /// }); /// ``` - fn distinct_total(self) -> VecCollection { + fn distinct_total(self) -> VecCollection { self.distinct_total_core() } @@ -78,17 +78,17 @@ pub trait ThresholdTotal, K: Exchang /// This method allows `distinct` to produce collections whose difference /// type is something other than an `isize` integer, for example perhaps an /// `i32`. - fn distinct_total_core+'static>(self) -> VecCollection { + fn distinct_total_core+'static>(self) -> VecCollection { self.threshold_total(|_,_| R2::from(1i8)) } } -impl ThresholdTotal for VecCollection +impl ThresholdTotal for VecCollection where - G: Scope, + T: Timestamp + TotalOrder + Lattice + Ord, { - fn threshold_semigroup(self, thresh: F) -> VecCollection + fn threshold_semigroup(self, thresh: F) -> VecCollection where R2: Semigroup+'static, F: FnMut(&K,&R,Option<&R>)->Option+'static, @@ -98,21 +98,20 @@ where } } -impl ThresholdTotal for Arranged +impl ThresholdTotal for Arranged where - G: Scope, - T1: for<'a> TraceReader< + Tr: for<'a> TraceReader< Key<'a>=&'a K, Val<'a>=&'a (), Time: TotalOrder, - Diff : ExchangeData + Semigroup>, + Diff : ExchangeData + Semigroup>, >+Clone+'static, K: ExchangeData, { - fn threshold_semigroup(self, mut thresh: F) -> VecCollection + fn threshold_semigroup(self, mut thresh: F) -> VecCollection where R2: Semigroup+'static, - F: for<'a> FnMut(T1::Key<'a>,&T1::Diff,Option<&T1::Diff>)->Option+'static, + F: for<'a> FnMut(Tr::Key<'a>,&Tr::Diff,Option<&Tr::Diff>)->Option+'static, { let mut trace = self.trace.clone(); @@ -120,8 +119,8 @@ where self.stream.unary_frontier(Pipeline, "ThresholdTotal", move |_,_| { // tracks the lower and upper limit of received batches. - let mut lower_limit = timely::progress::frontier::Antichain::from_elem(::minimum()); - let mut upper_limit = timely::progress::frontier::Antichain::from_elem(::minimum()); + let mut lower_limit = timely::progress::frontier::Antichain::from_elem(Tr::Time::minimum()); + let mut upper_limit = timely::progress::frontier::Antichain::from_elem(Tr::Time::minimum()); move |(input, _frontier), output| { @@ -153,14 +152,14 @@ where let (mut trace_cursor, trace_storage) = trace.cursor_through(lower_limit.borrow()).unwrap(); while let Some(key) = batch_cursor.get_key(&batch_storage) { - let mut count: Option = None; + let mut count: Option = None; // Compute the multiplicity of this key before the current batch. trace_cursor.seek_key(&trace_storage, key); if trace_cursor.get_key(&trace_storage) == Some(key) { trace_cursor.map_times(&trace_storage, |_, diff| { count.as_mut().map(|c| c.plus_equals(&diff)); - if count.is_none() { count = Some(T1::owned_diff(diff)); } + if count.is_none() { count = Some(Tr::owned_diff(diff)); } }); } @@ -175,7 +174,7 @@ where temp.plus_equals(&diff); thresh(key, &temp, Some(old)) }, - None => { thresh(key, &T1::owned_diff(diff), None) }, + None => { thresh(key, &Tr::owned_diff(diff), None) }, }; // Either add or assign `diff` to `count`. @@ -183,12 +182,12 @@ where count.plus_equals(&diff); } else { - count = Some(T1::owned_diff(diff)); + count = Some(Tr::owned_diff(diff)); } if let Some(difference) = difference { if !difference.is_zero() { - session.give((key.clone(), T1::owned_time(time), difference)); + session.give((key.clone(), Tr::owned_time(time), difference)); } } }); diff --git a/differential-dataflow/tests/bfs.rs b/differential-dataflow/tests/bfs.rs index fd6aea26c..d3a3bccb8 100644 --- a/differential-dataflow/tests/bfs.rs +++ b/differential-dataflow/tests/bfs.rs @@ -4,7 +4,6 @@ use std::sync::{Arc, Mutex}; use timely::Config; -use timely::dataflow::*; use timely::dataflow::operators::Capture; use timely::dataflow::operators::capture::Extract; @@ -202,9 +201,9 @@ fn bfs_differential( } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord, { // initialize roots as reaching themselves at distance 0 let nodes = roots.map(|x| (x, 0)); diff --git a/differential-dataflow/tests/scc.rs b/differential-dataflow/tests/scc.rs index 52eecdf1d..2e48b41f3 100644 --- a/differential-dataflow/tests/scc.rs +++ b/differential-dataflow/tests/scc.rs @@ -7,7 +7,6 @@ use std::mem; use timely::Config; -use timely::dataflow::*; use timely::dataflow::operators::Capture; use timely::dataflow::operators::capture::Extract; @@ -215,9 +214,9 @@ fn scc_differential( .collect() } -fn _strongly_connected(graph: VecCollection) -> VecCollection +fn _strongly_connected(graph: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord + Hash, { graph.clone().iterate(|scope, inner| { let edges = graph.enter(&scope); @@ -226,9 +225,9 @@ where }) } -fn _trim_edges(cycle: VecCollection, edges: VecCollection) -> VecCollection +fn _trim_edges(cycle: VecCollection, edges: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord + Hash, { let nodes = edges.clone() .map_in_place(|x| x.0 = x.1) @@ -244,9 +243,9 @@ where .map(|((x1,x2),_)| (x2,x1)) } -fn _reachability(edges: VecCollection, nodes: VecCollection) -> VecCollection +fn _reachability(edges: VecCollection, nodes: VecCollection) -> VecCollection where - G: Scope, + T: timely::progress::Timestamp + Lattice + Ord + Hash, { edges.clone() // <-- wth is this. .filter(|_| false) diff --git a/dogsdogsdogs/examples/delta_query.rs b/dogsdogsdogs/examples/delta_query.rs index af9eb5bfd..74d24b246 100644 --- a/dogsdogsdogs/examples/delta_query.rs +++ b/dogsdogsdogs/examples/delta_query.rs @@ -1,4 +1,3 @@ -use timely::dataflow::Scope; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; use graph_map::GraphMMap; @@ -90,7 +89,7 @@ fn main() { let changes3 = validate(changes3, reverse_self_alt.clone(), key2.clone()); let changes3 = changes3.map(|((a,c),b)| (a,b,c)); - let prev_changes = changes1.concat(changes2).concat(changes3).leave(); + let prev_changes = changes1.concat(changes2).concat(changes3).leave(&scope); // New ideas let d_edges = edges.differentiate(inner); @@ -116,7 +115,7 @@ fn main() { .join_core(forward_key_alt, |a,c,b| Some(((*c, *b), *a))) .join_core(reverse_self_alt, |(c,b), a, &()| Some((*a,*b,*c))); - let next_changes = changes1.concat(changes2).concat(changes3).integrate(); + let next_changes = changes1.concat(changes2).concat(changes3).integrate(&scope); (prev_changes, next_changes) }); diff --git a/dogsdogsdogs/examples/delta_query2.rs b/dogsdogsdogs/examples/delta_query2.rs index 6b088895e..f29a95a1a 100644 --- a/dogsdogsdogs/examples/delta_query2.rs +++ b/dogsdogsdogs/examples/delta_query2.rs @@ -1,4 +1,3 @@ -use timely::dataflow::Scope; use timely::order::Product; use timely::dataflow::operators::probe::Handle; use timely::dataflow::operators::vec::UnorderedInput; diff --git a/dogsdogsdogs/examples/delta_query_wcoj.rs b/dogsdogsdogs/examples/delta_query_wcoj.rs index afceef1ee..760c0afc1 100644 --- a/dogsdogsdogs/examples/delta_query_wcoj.rs +++ b/dogsdogsdogs/examples/delta_query_wcoj.rs @@ -1,4 +1,3 @@ -use timely::dataflow::Scope; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; use graph_map::GraphMMap; @@ -80,7 +79,7 @@ fn main() { ]) .map(|((a,c),b)| (a,b,c)); - changes1.concat(changes2).concat(changes3).leave() + changes1.concat(changes2).concat(changes3).leave(&scope) }); triangles diff --git a/dogsdogsdogs/examples/ngo.rs b/dogsdogsdogs/examples/ngo.rs index 340b6c988..9c1177a58 100644 --- a/dogsdogsdogs/examples/ngo.rs +++ b/dogsdogsdogs/examples/ngo.rs @@ -1,5 +1,4 @@ use std::hash::Hash; -use timely::dataflow::*; use timely::dataflow::operators::*; use timely::dataflow::operators::vec::count::Accumulate; @@ -38,9 +37,9 @@ fn main() { }).unwrap(); } -fn triangles(edges: VecCollection) -> VecCollection +fn triangles(edges: VecCollection) -> VecCollection where - G: Scope, + G: timely::progress::Timestamp + Lattice + Hash + Ord, { // only use forward-pointing edges. let edges = edges.filter(|&(src, dst)| src < dst); diff --git a/dogsdogsdogs/src/altneu.rs b/dogsdogsdogs/src/altneu.rs index 95d69cbc0..0ddb9572a 100644 --- a/dogsdogsdogs/src/altneu.rs +++ b/dogsdogsdogs/src/altneu.rs @@ -66,7 +66,7 @@ impl Refines for AltNeu { fn to_outer(self: AltNeu) -> T { self.time } - fn summarize(_path: ()) -> ::Summary { + fn summarize(_path: ()) -> T::Summary { Default::default() } } diff --git a/dogsdogsdogs/src/calculus.rs b/dogsdogsdogs/src/calculus.rs index 94b443231..8109d1262 100644 --- a/dogsdogsdogs/src/calculus.rs +++ b/dogsdogsdogs/src/calculus.rs @@ -13,7 +13,7 @@ //! almost everywhere empty (and so has a low memory footprint, if the system works as planned). use timely::dataflow::Scope; -use timely::dataflow::scopes::Child; +use timely::progress::Timestamp; use timely::dataflow::operators::vec::{Filter, Map}; use differential_dataflow::{AsCollection, VecCollection, Data}; use differential_dataflow::difference::Abelian; @@ -21,23 +21,23 @@ use differential_dataflow::difference::Abelian; use crate::altneu::AltNeu; /// Produce a collection containing the changes at the moments they happen. -pub trait Differentiate { - fn differentiate<'a>(self, child: &Child<'a, G, AltNeu>) -> VecCollection>, D, R>; +pub trait Differentiate { + fn differentiate(self, child: &Scope>) -> VecCollection, D, R>; } /// Collect instantaneous changes back in to a collection. -pub trait Integrate { - fn integrate(self) -> VecCollection; +pub trait Integrate { + fn integrate(self, outer: &Scope) -> VecCollection; } -impl Differentiate for VecCollection +impl Differentiate for VecCollection where - G: Scope, + T: Timestamp, D: Data, R: Abelian + 'static, { // For each (data, Alt(time), diff) we add a (data, Neu(time), -diff). - fn differentiate<'a>(self, child: &Child<'a, G, AltNeu>) -> VecCollection>, D, R> { + fn differentiate(self, child: &Scope>) -> VecCollection, D, R> { self.enter(child) .inner .flat_map(|(data, time, diff)| { @@ -51,17 +51,17 @@ where } } -impl<'a, G, D, R> Integrate for VecCollection>, D, R> +impl Integrate for VecCollection, D, R> where - G: Scope, + T: Timestamp, D: Data, R: Abelian + 'static, { // We discard each `neu` variant and strip off the `alt` wrapper. - fn integrate(self) -> VecCollection { + fn integrate(self, outer: &Scope) -> VecCollection { self.inner .filter(|(_d,t,_r)| !t.neu) .as_collection() - .leave() + .leave(outer) } } diff --git a/dogsdogsdogs/src/lib.rs b/dogsdogsdogs/src/lib.rs index f704d5988..e2699ee0d 100644 --- a/dogsdogsdogs/src/lib.rs +++ b/dogsdogsdogs/src/lib.rs @@ -1,6 +1,5 @@ use std::hash::Hash; -use timely::dataflow::Scope; use timely::progress::Timestamp; use timely::dataflow::operators::vec::Partition; use timely::dataflow::operators::Concatenate; @@ -20,37 +19,37 @@ pub mod operators; Implementors of `PrefixExtension` provide types and methods for extending a differential dataflow collection, via the three methods `count`, `propose`, and `validate`. **/ -pub trait PrefixExtender> { +pub trait PrefixExtender> { /// The required type of prefix to extend. type Prefix; /// The type to be produced as extension. type Extension; /// Annotates prefixes with the number of extensions the relation would propose. - fn count(&mut self, prefixes: VecCollection, index: usize) -> VecCollection; + fn count(&mut self, prefixes: VecCollection, index: usize) -> VecCollection; /// Extends each prefix with corresponding extensions. - fn propose(&mut self, prefixes: VecCollection) -> VecCollection; + fn propose(&mut self, prefixes: VecCollection) -> VecCollection; /// Restricts proposed extensions by those the extender would have proposed. - fn validate(&mut self, extensions: VecCollection) -> VecCollection; + fn validate(&mut self, extensions: VecCollection) -> VecCollection; } -pub trait ProposeExtensionMethod> { - fn propose_using>(self, extender: &mut PE) -> VecCollection; - fn extend(self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection; +pub trait ProposeExtensionMethod> { + fn propose_using>(self, extender: &mut PE) -> VecCollection; + fn extend(self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection; } -impl ProposeExtensionMethod for VecCollection +impl ProposeExtensionMethod for VecCollection where - G: Scope, + T: Timestamp, P: ExchangeData+Ord, R: Monoid+Multiply+'static, { - fn propose_using(self, extender: &mut PE) -> VecCollection + fn propose_using(self, extender: &mut PE) -> VecCollection where - PE: PrefixExtender + PE: PrefixExtender { extender.propose(self) } - fn extend(self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection + fn extend(self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection where E: ExchangeData+Ord { @@ -81,12 +80,12 @@ where } } -pub trait ValidateExtensionMethod, P, E> { - fn validate_using>(self, extender: &mut PE) -> VecCollection; +pub trait ValidateExtensionMethod, P, E> { + fn validate_using>(self, extender: &mut PE) -> VecCollection; } -impl, P, E> ValidateExtensionMethod for VecCollection { - fn validate_using>(self, extender: &mut PE) -> VecCollection { +impl, P, E> ValidateExtensionMethod for VecCollection { + fn validate_using>(self, extender: &mut PE) -> VecCollection { extender.validate(self) } } @@ -137,7 +136,7 @@ where R: Monoid+Multiply+ExchangeData, { - pub fn index>(collection: VecCollection) -> Self { + pub fn index(collection: VecCollection) -> Self { // We need to count the number of (k, v) pairs and not rely on the given Monoid R and its binary addition operation. // counts and validate can share the base arrangement let arranged = collection.clone().arrange_by_self(); @@ -180,9 +179,9 @@ where key_selector: F, } -impl PrefixExtender for CollectionExtender +impl PrefixExtender for CollectionExtender where - G: Scope, + T: Timestamp + Lattice + ExchangeData + Hash, K: ExchangeData+Hash+Default, V: ExchangeData+Hash+Default, P: ExchangeData, @@ -192,17 +191,17 @@ where type Prefix = P; type Extension = V; - fn count(&mut self, prefixes: VecCollection, index: usize) -> VecCollection { + fn count(&mut self, prefixes: VecCollection, index: usize) -> VecCollection { let counts = self.indices.count_trace.import(&prefixes.scope()); operators::count::count(prefixes, counts, self.key_selector.clone(), index) } - fn propose(&mut self, prefixes: VecCollection) -> VecCollection { + fn propose(&mut self, prefixes: VecCollection) -> VecCollection { let propose = self.indices.propose_trace.import(&prefixes.scope()); operators::propose::propose(prefixes, propose, self.key_selector.clone()) } - fn validate(&mut self, extensions: VecCollection) -> VecCollection { + fn validate(&mut self, extensions: VecCollection) -> VecCollection { let validate = self.indices.validate_trace.import(&extensions.scope()); operators::validate::validate(extensions, validate, self.key_selector.clone()) } diff --git a/dogsdogsdogs/src/operators/count.rs b/dogsdogsdogs/src/operators/count.rs index 42367a2e1..a92066f1e 100644 --- a/dogsdogsdogs/src/operators/count.rs +++ b/dogsdogsdogs/src/operators/count.rs @@ -1,5 +1,3 @@ -use timely::dataflow::Scope; - use differential_dataflow::{ExchangeData, VecCollection, Hashable}; use differential_dataflow::difference::{Semigroup, Monoid, Multiply}; use differential_dataflow::operators::arrange::Arranged; @@ -11,14 +9,13 @@ use differential_dataflow::trace::TraceReader; /// For each triple, it extracts a key using `key_selector`, and finds the /// associated count in `arrangement`. If the found count is less than `count`, /// the `count` and `index` fields are overwritten with their new values. -pub fn count( - prefixes: VecCollection, - arrangement: Arranged, +pub fn count( + prefixes: VecCollection, + arrangement: Arranged, key_selector: F, index: usize, -) -> VecCollection +) -> VecCollection where - G: Scope, Tr: TraceReader+Clone+'static, Tr::KeyContainer: differential_dataflow::trace::implementations::BatchContainer, for<'a> Tr::Diff : Semigroup>, diff --git a/dogsdogsdogs/src/operators/half_join.rs b/dogsdogsdogs/src/operators/half_join.rs index 2a18a9aa1..8bedb4cbf 100644 --- a/dogsdogsdogs/src/operators/half_join.rs +++ b/dogsdogsdogs/src/operators/half_join.rs @@ -37,9 +37,11 @@ use std::time::Instant; use timely::ContainerBuilder; use timely::container::CapacityContainerBuilder; -use timely::dataflow::{Scope, ScopeParent, Stream}; +use timely::dataflow::Stream; +use timely::scheduling::Scheduler; use timely::dataflow::channels::pact::{Pipeline, Exchange}; use timely::dataflow::operators::{Capability, Operator, generic::Session}; +use timely::PartialOrder; use timely::progress::Antichain; use timely::progress::frontier::AntichainRef; @@ -73,45 +75,44 @@ use differential_dataflow::trace::implementations::BatchContainer; /// Notice that the time is hoisted up into data. The expectation is that /// once out of the "delta flow region", the updates will be `delay`d to the /// times specified in the payloads. -pub fn half_join( - stream: VecCollection, - arrangement: Arranged, +pub fn half_join( + stream: VecCollection, + arrangement: Arranged, frontier_func: FF, comparison: CF, mut output_func: S, -) -> VecCollection>::Output> +) -> VecCollection>::Output> where - G: Scope, K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, Tr: TraceReader+Clone+'static, Tr::KeyContainer: BatchContainer, R: Mul, - FF: Fn(&G::Timestamp, &mut Antichain) + 'static, - CF: Fn(Tr::TimeGat<'_>, &G::Timestamp) -> bool + 'static, + FF: Fn(&Tr::Time, &mut Antichain) + 'static, + CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, DOut: Clone+'static, S: FnMut(&K, &V, Tr::Val<'_>)->DOut+'static, { - let output_func = move |session: &mut SessionFor, k: &K, v1: &V, v2: Tr::Val<'_>, initial: &G::Timestamp, diff1: &R, output: &mut Vec<(G::Timestamp, Tr::Diff)>| { + let output_func = move |session: &mut SessionFor, k: &K, v1: &V, v2: Tr::Val<'_>, initial: &Tr::Time, diff1: &R, output: &mut Vec<(Tr::Time, Tr::Diff)>| { for (time, diff2) in output.drain(..) { let diff = diff1.clone() * diff2.clone(); let dout = (output_func(k, v1, v2), time.clone()); session.give((dout, initial.clone(), diff)); } }; - half_join_internal_unsafe::<_, _, _, _, _, _,_,_,_, CapacityContainerBuilder>>(stream, arrangement, frontier_func, comparison, |_timer, _count| false, output_func) + half_join_internal_unsafe::<_, _, _, _, _, _,_,_, CapacityContainerBuilder>>(stream, arrangement, frontier_func, comparison, |_timer, _count| false, output_func) .as_collection() } -/// A session with lifetime `'a` in a scope `G` with a container builder `CB`. +/// A session with lifetime `'a` over timestamp `T` with a container builder `CB`. /// /// This is a shorthand primarily for the reson of readability. -type SessionFor<'a, 'b, G, CB> = +type SessionFor<'a, 'b, T, CB> = Session<'a, 'b, - ::Timestamp, + T, CB, - Capability<::Timestamp>, + Capability, >; /// An unsafe variant of `half_join` where the `output_func` closure takes @@ -139,25 +140,24 @@ type SessionFor<'a, 'b, G, CB> = /// yield control, as a function of the elapsed time and the number of matched /// records. Note this is not the number of *output* records, owing mainly to /// the number of matched records being easiest to record with low overhead. -pub fn half_join_internal_unsafe( - stream: VecCollection, - mut arrangement: Arranged, +pub fn half_join_internal_unsafe( + stream: VecCollection, + mut arrangement: Arranged, frontier_func: FF, comparison: CF, yield_function: Y, mut output_func: S, -) -> Stream +) -> Stream where - G: Scope, K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, Tr: TraceReader+Clone+'static, Tr::KeyContainer: BatchContainer, - FF: Fn(&G::Timestamp, &mut Antichain) + 'static, + FF: Fn(&Tr::Time, &mut Antichain) + 'static, CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, Y: Fn(std::time::Instant, usize) -> bool + 'static, - S: FnMut(&mut SessionFor, &K, &V, Tr::Val<'_>, &G::Timestamp, &R, &mut Vec<(G::Timestamp, Tr::Diff)>) + 'static, + S: FnMut(&mut SessionFor, &K, &V, Tr::Val<'_>, &Tr::Time, &R, &mut Vec<(Tr::Time, Tr::Diff)>) + 'static, CB: ContainerBuilder, { // No need to block physical merging for this operator. @@ -167,7 +167,7 @@ where let mut stash = HashMap::new(); - let exchange = Exchange::new(move |update: &((K, V, G::Timestamp),G::Timestamp,R)| (update.0).0.hashed().into()); + let exchange = Exchange::new(move |update: &((K, V, Tr::Time),Tr::Time,R)| (update.0).0.hashed().into()); // Stash for (time, diff) accumulation. let mut output_buffer = Vec::new(); @@ -215,7 +215,7 @@ where // Update yielded: We can only go from false to {false, true} as // we're checking that `!yielded` holds before entering this block. - yielded = process_proposals::( + yielded = process_proposals::<_, _, _, _, _, _, _, _>( &comparison, &yield_function, &mut output_func, @@ -247,7 +247,6 @@ where // Any remaining times should peel off elements from `proposals`. let mut additions = vec![Vec::new(); antichain.len()]; for (data, initial, diff) in proposals.drain(..) { - use timely::PartialOrder; let position = antichain.iter().position(|t| t.less_equal(&initial)).unwrap(); additions[position].push((data, initial, diff)); } @@ -300,7 +299,7 @@ where /// Leaves a zero diff in place for all proposals that were processed. /// /// Returns `true` if the operator should yield. -fn process_proposals( +fn process_proposals( comparison: &CF, yield_function: &Y, output_func: &mut S, @@ -309,16 +308,15 @@ fn process_proposals( work: &mut usize, trace: &mut Tr, proposals: &mut Vec<((K, V, Tr::Time), Tr::Time, R)>, - mut session: SessionFor, + mut session: SessionFor, frontier: AntichainRef ) -> bool where - G: Scope, Tr: TraceReader, Tr::KeyContainer: BatchContainer, CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, Y: Fn(Instant, usize) -> bool + 'static, - S: FnMut(&mut SessionFor, &K, &V, Tr::Val<'_>, &G::Timestamp, &R, &mut Vec<(G::Timestamp, Tr::Diff)>) + 'static, + S: FnMut(&mut SessionFor, &K, &V, Tr::Val<'_>, &Tr::Time, &R, &mut Vec<(Tr::Time, Tr::Diff)>) + 'static, CB: ContainerBuilder, K: Ord, V: Ord, diff --git a/dogsdogsdogs/src/operators/half_join2.rs b/dogsdogsdogs/src/operators/half_join2.rs index 08029f96e..6b19c0bbd 100644 --- a/dogsdogsdogs/src/operators/half_join2.rs +++ b/dogsdogsdogs/src/operators/half_join2.rs @@ -22,7 +22,8 @@ use std::ops::Mul; use timely::ContainerBuilder; use timely::container::CapacityContainerBuilder; -use timely::dataflow::{Scope, Stream}; +use timely::dataflow::Stream; +use timely::scheduling::Scheduler; use timely::dataflow::channels::pact::{Pipeline, Exchange}; use timely::dataflow::operators::Operator; use timely::PartialOrder; @@ -61,27 +62,26 @@ use timely::dataflow::operators::CapabilitySet; /// Notice that the time is hoisted up into data. The expectation is that /// once out of the "delta flow region", the updates will be `delay`d to the /// times specified in the payloads. -pub fn half_join( - stream: VecCollection, - arrangement: Arranged, +pub fn half_join( + stream: VecCollection, + arrangement: Arranged, frontier_func: FF, comparison: CF, mut output_func: S, -) -> VecCollection>::Output> +) -> VecCollection>::Output> where - G: Scope, K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, Tr: TraceReader+Clone+'static, Tr::KeyContainer: BatchContainer, R: Mul, - FF: Fn(&G::Timestamp, &mut Antichain) + 'static, - CF: Fn(Tr::TimeGat<'_>, &G::Timestamp) -> bool + 'static, + FF: Fn(&Tr::Time, &mut Antichain) + 'static, + CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, DOut: Clone+'static, S: FnMut(&K, &V, Tr::Val<'_>)->DOut+'static, { - let output_func = move |builder: &mut CapacityContainerBuilder>, k: &K, v1: &V, v2: Tr::Val<'_>, initial: &G::Timestamp, diff1: &R, output: &mut Vec<(G::Timestamp, Tr::Diff)>| { + let output_func = move |builder: &mut CapacityContainerBuilder>, k: &K, v1: &V, v2: Tr::Val<'_>, initial: &Tr::Time, diff1: &R, output: &mut Vec<(Tr::Time, Tr::Diff)>| { for (time, diff2) in output.drain(..) { let diff = diff1.clone() * diff2.clone(); let dout = (output_func(k, v1, v2), time.clone()); @@ -89,7 +89,7 @@ where builder.push_into((dout, initial.clone(), diff)); } }; - half_join_internal_unsafe::<_, _, _, _, _, _,_,_,_, CapacityContainerBuilder>>(stream, arrangement, frontier_func, comparison, |_timer, _count| false, output_func) + half_join_internal_unsafe::<_, _, _, _, _, _,_,_, CapacityContainerBuilder>>(stream, arrangement, frontier_func, comparison, |_timer, _count| false, output_func) .as_collection() } @@ -118,25 +118,24 @@ where /// yield control, as a function of the elapsed time and the number of matched /// records. Note this is not the number of *output* records, owing mainly to /// the number of matched records being easiest to record with low overhead. -pub fn half_join_internal_unsafe( - stream: VecCollection, - mut arrangement: Arranged, +pub fn half_join_internal_unsafe( + stream: VecCollection, + mut arrangement: Arranged, frontier_func: FF, comparison: CF, yield_function: Y, mut output_func: S, -) -> Stream +) -> Stream where - G: Scope, K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, Tr: TraceReader+Clone+'static, Tr::KeyContainer: BatchContainer, - FF: Fn(&G::Timestamp, &mut Antichain) + 'static, + FF: Fn(&Tr::Time, &mut Antichain) + 'static, CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, Y: Fn(std::time::Instant, usize) -> bool + 'static, - S: FnMut(&mut CB, &K, &V, Tr::Val<'_>, &G::Timestamp, &R, &mut Vec<(G::Timestamp, Tr::Diff)>) + 'static, + S: FnMut(&mut CB, &K, &V, Tr::Val<'_>, &Tr::Time, &R, &mut Vec<(Tr::Time, Tr::Diff)>) + 'static, CB: ContainerBuilder, { // No need to block physical merging for this operator. @@ -144,7 +143,7 @@ where let mut arrangement_trace = Some(arrangement.trace); let arrangement_stream = arrangement.stream; - let exchange = Exchange::new(move |update: &((K, V, G::Timestamp),G::Timestamp,R)| (update.0).0.hashed().into()); + let exchange = Exchange::new(move |update: &((K, V, Tr::Time),Tr::Time,R)| (update.0).0.hashed().into()); // Stash for (time, diff) accumulation. let mut output_buffer = Vec::new(); @@ -152,7 +151,7 @@ where // Unified blobs: each blob holds data in (T, D, R) order, with a stuck_count // tracking how many elements at the back are not yet eligible for processing. // The ready prefix is sorted by (D, T, R) for cursor traversal. - let mut blobs: Vec> = Vec::new(); + let mut blobs: Vec> = Vec::new(); let scope = stream.scope(); stream.inner.binary_frontier(arrangement_stream, exchange, Pipeline, "HalfJoin", move |_,info| { @@ -163,7 +162,7 @@ where move |(input1, frontier1), (input2, frontier2), output| { // Drain all input into a single buffer. - let mut arriving: Vec<(G::Timestamp, (K, V, G::Timestamp), R)> = Vec::new(); + let mut arriving: Vec<(Tr::Time, (K, V, Tr::Time), R)> = Vec::new(); let mut caps = CapabilitySet::new(); input1.for_each(|capability, data| { caps.insert(capability.retain(0)); @@ -189,7 +188,7 @@ where if let Some(min_time) = frontier.iter().min() { time_con.push_own(min_time); } - let eligible = |initial: &G::Timestamp| -> bool { + let eligible = |initial: &Tr::Time| -> bool { !(0..time_con.len()).any(|i| comparison(time_con.index(i), initial)) }; @@ -265,7 +264,7 @@ where let (mut cursor, storage) = trace.cursor(); let mut key_con = Tr::KeyContainer::with_capacity(1); - let mut removals: ChangeBatch = ChangeBatch::new(); + let mut removals: ChangeBatch = ChangeBatch::new(); // Process ready elements from the front. while blob.data.len() > blob.stuck_count { diff --git a/dogsdogsdogs/src/operators/lookup_map.rs b/dogsdogsdogs/src/operators/lookup_map.rs index 65108ed59..b15aba314 100644 --- a/dogsdogsdogs/src/operators/lookup_map.rs +++ b/dogsdogsdogs/src/operators/lookup_map.rs @@ -1,9 +1,8 @@ use std::collections::HashMap; -use timely::PartialOrder; -use timely::dataflow::Scope; use timely::dataflow::channels::pact::{Pipeline, Exchange}; use timely::dataflow::operators::Operator; +use timely::PartialOrder; use timely::progress::Antichain; use differential_dataflow::{ExchangeData, VecCollection, AsCollection, Hashable}; @@ -17,17 +16,16 @@ use differential_dataflow::trace::implementations::BatchContainer; /// This method takes a stream of prefixes and for each determines a /// key with `key_selector` and then proposes all pair af the prefix /// and values associated with the key in `arrangement`. -pub fn lookup_map( - prefixes: VecCollection, - mut arrangement: Arranged, +pub fn lookup_map( + prefixes: VecCollection, + mut arrangement: Arranged, key_selector: F, mut output_func: S, supplied_key0: K, supplied_key1: K, supplied_key2: K, -) -> VecCollection +) -> VecCollection where - G: Scope, Tr: for<'a> TraceReader< Time: std::hash::Hash, Diff : Semigroup>+Monoid+ExchangeData, @@ -51,7 +49,7 @@ where let mut logic2 = key_selector.clone(); let mut key: K = supplied_key0; - let exchange = Exchange::new(move |update: &(D,G::Timestamp,R)| { + let exchange = Exchange::new(move |update: &(D,Tr::Time,R)| { logic1(&update.0, &mut key); key.hashed().into() }); diff --git a/dogsdogsdogs/src/operators/propose.rs b/dogsdogsdogs/src/operators/propose.rs index 6cdcb130d..869e67aab 100644 --- a/dogsdogsdogs/src/operators/propose.rs +++ b/dogsdogsdogs/src/operators/propose.rs @@ -1,5 +1,3 @@ -use timely::dataflow::Scope; - use differential_dataflow::{ExchangeData, VecCollection, Hashable}; use differential_dataflow::difference::{Semigroup, Monoid, Multiply}; use differential_dataflow::operators::arrange::Arranged; @@ -13,13 +11,12 @@ use differential_dataflow::trace::TraceReader; /// create a join if the `prefixes` collection is also arranged and responds to changes that /// `arrangement` undergoes. More complicated patterns are also appropriate, as in the case /// of delta queries. -pub fn propose( - prefixes: VecCollection, - arrangement: Arranged, +pub fn propose( + prefixes: VecCollection, + arrangement: Arranged, key_selector: F, -) -> VecCollection +) -> VecCollection where - G: Scope, Tr: for<'a> TraceReader< ValOwn = V, Time: std::hash::Hash, @@ -47,13 +44,12 @@ where /// Unlike `propose`, this method does not scale the multiplicity of matched /// prefixes by the number of matches in `arrangement`. This can be useful to /// avoid the need to prepare an arrangement of distinct extensions. -pub fn propose_distinct( - prefixes: VecCollection, - arrangement: Arranged, +pub fn propose_distinct( + prefixes: VecCollection, + arrangement: Arranged, key_selector: F, -) -> VecCollection +) -> VecCollection where - G: Scope, Tr: for<'a> TraceReader< ValOwn = V, Time: std::hash::Hash, diff --git a/dogsdogsdogs/src/operators/validate.rs b/dogsdogsdogs/src/operators/validate.rs index 0ab7297bb..0a28c2483 100644 --- a/dogsdogsdogs/src/operators/validate.rs +++ b/dogsdogsdogs/src/operators/validate.rs @@ -1,7 +1,5 @@ use std::hash::Hash; -use timely::dataflow::Scope; - use differential_dataflow::{ExchangeData, VecCollection}; use differential_dataflow::difference::{Semigroup, Monoid, Multiply}; use differential_dataflow::operators::arrange::Arranged; @@ -12,13 +10,12 @@ use differential_dataflow::trace::TraceReader; /// This method takes a stream of prefixes and for each determines a /// key with `key_selector` and then proposes all pair af the prefix /// and values associated with the key in `arrangement`. -pub fn validate( - extensions: VecCollection, - arrangement: Arranged, +pub fn validate( + extensions: VecCollection, + arrangement: Arranged, key_selector: F, -) -> VecCollection +) -> VecCollection where - G: Scope, Tr: for<'a> TraceReader< Time: std::hash::Hash, Diff : Semigroup>+Monoid+Multiply+ExchangeData, diff --git a/experiments/Cargo.toml b/experiments/Cargo.toml index edcf55fb3..ff46dd329 100644 --- a/experiments/Cargo.toml +++ b/experiments/Cargo.toml @@ -12,7 +12,7 @@ workspace = true [dependencies] core_affinity = "0.5.9" rand="0.3.13" -timely = { workspace = true } +timely = { workspace = true, features = ["getopts"] } differential-dataflow = { workspace = true } graph_map = { git = "https://github.com/frankmcsherry/graph-map" } serde = { version = "1.0", features = ["derive"] } diff --git a/experiments/src/bin/deals-interactive.rs b/experiments/src/bin/deals-interactive.rs index 3ea53cd14..a3c29b5e3 100644 --- a/experiments/src/bin/deals-interactive.rs +++ b/experiments/src/bin/deals-interactive.rs @@ -1,7 +1,6 @@ use std::time::Instant; use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; use timely::WorkerConfig; use differential_dataflow::input::Input; @@ -13,7 +12,7 @@ use differential_dataflow::trace::implementations::ValSpine; use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; -type Arrange = Arranged::Timestamp, R>>>; +type Arrange = Arranged>>; type Node = u32; @@ -204,13 +203,12 @@ fn main() { }).unwrap(); } -fn interactive( +fn interactive( edges: Arrange, tc_1: VecCollection, tc_2: VecCollection, sg_x: VecCollection -) -> VecCollection -where G::Timestamp: Lattice{ +) -> VecCollection { // descendants of tc_1: let tc_1_enter = tc_1.clone(); diff --git a/experiments/src/bin/deals.rs b/experiments/src/bin/deals.rs index 782941e15..89c9699a0 100644 --- a/experiments/src/bin/deals.rs +++ b/experiments/src/bin/deals.rs @@ -1,7 +1,5 @@ use std::time::Instant; -use timely::dataflow::*; - use differential_dataflow::input::Input; use differential_dataflow::VecCollection; use differential_dataflow::operators::*; @@ -11,9 +9,10 @@ use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; use differential_dataflow::operators::arrange::Arrange; use differential_dataflow::operators::iterate::Variable; +use differential_dataflow::lattice::Lattice; use differential_dataflow::difference::Present; -type EdgeArranged = Arranged::Timestamp, R>>>; +type EdgeArranged = Arranged>>; type Node = u32; type Edge = (Node, Node); @@ -83,10 +82,11 @@ fn main() { use timely::order::Product; // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn tc>(edges: EdgeArranged) -> VecCollection { +fn tc(edges: EdgeArranged) -> VecCollection { // repeatedly update minimal distances each node can be reached from each root - edges.stream.scope().iterative::(|scope| { + let outer = edges.stream.scope(); + outer.iterative::(|scope| { let (inner, inner_collection) = Variable::new(scope, Product::new(Default::default(), 1)); let edges = edges.enter(scope); @@ -102,18 +102,19 @@ fn tc>(edges: EdgeArranged) -> Ve ; inner.set(result.clone()); - result.leave() + result.leave(&outer) } ) } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn sg>(edges: EdgeArranged) -> VecCollection { +fn sg(edges: EdgeArranged) -> VecCollection { let peers = edges.clone().join_core(edges.clone(), |_,&x,&y| Some((x,y))).filter(|&(x,y)| x != y); // repeatedly update minimal distances each node can be reached from each root - peers.scope().iterative::(|scope| { + let outer = peers.scope(); + outer.iterative::(|scope| { let (inner, inner_collection) = Variable::new(scope, Product::new(Default::default(), 1)); let edges = edges.enter(scope); @@ -131,7 +132,7 @@ fn sg>(edges: EdgeArranged) -> Ve ; inner.set(result.clone()); - result.leave() + result.leave(&outer) } ) } diff --git a/experiments/src/bin/graphs-interactive-alt.rs b/experiments/src/bin/graphs-interactive-alt.rs index 6e5b0ffaa..0ba7ddd4b 100644 --- a/experiments/src/bin/graphs-interactive-alt.rs +++ b/experiments/src/bin/graphs-interactive-alt.rs @@ -2,7 +2,7 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::probe::Handle; use timely::order::Product; @@ -258,14 +258,14 @@ use differential_dataflow::trace::implementations::ValSpine; use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; -type Arrange = Arranged::Timestamp, R>>>; +type Arrange = Arranged>>; // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn three_hop( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn three_hop( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); @@ -288,13 +288,14 @@ where G::Timestamp: Lattice+Ord { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn _bidijkstra( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn _bidijkstra( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ - goals.scope().iterative::(|inner| { + let outer = goals.scope(); + outer.iterative::(|inner| { // Our plan is to start evolving distances from both sources and destinations. // The evolution from a source or destination should continue as long as there @@ -356,13 +357,14 @@ where G::Timestamp: Lattice+Ord { reverse.set(reverse_next); - reached.leave() + reached.leave(&outer) }) } -fn connected_components(graph: Arrange) -> VecCollection -where G::Timestamp: Lattice + std::hash::Hash { +fn connected_components(graph: Arrange) -> VecCollection +where T: Lattice + std::hash::Hash +{ // each edge (x,y) means that we need at least a label for the min of x and y. let nodes = diff --git a/experiments/src/bin/graphs-interactive-neu-zwei.rs b/experiments/src/bin/graphs-interactive-neu-zwei.rs index 35a118cd9..0a5ee0d25 100644 --- a/experiments/src/bin/graphs-interactive-neu-zwei.rs +++ b/experiments/src/bin/graphs-interactive-neu-zwei.rs @@ -1,6 +1,6 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::probe::Handle; use differential_dataflow::input::Input; @@ -226,14 +226,14 @@ use differential_dataflow::trace::implementations::ValSpine; use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; -type Arrange = Arranged::Timestamp, R>>>; +type Arrange = Arranged>>; // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn three_hop( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn three_hop( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); @@ -256,12 +256,12 @@ where G::Timestamp: Lattice+Ord { } // // returns pairs (n, s) indicating node n can be reached from a root in s steps. -// fn bidijkstra( -// forward_graph: &Arrange, -// reverse_graph: &Arrange, -// goals: VecCollection, -// bound: u64) -> VecCollection -// where G::Timestamp: Lattice+Ord { +// fn bidijkstra( +// forward_graph: &Arrange, +// reverse_graph: &Arrange, +// goals: VecCollection, +// bound: u64) -> VecCollection +// where T: Lattice+Ord { // goals.scope().scoped(|inner| { diff --git a/experiments/src/bin/graphs-interactive-neu.rs b/experiments/src/bin/graphs-interactive-neu.rs index 7e27e7ed8..ceda7913c 100644 --- a/experiments/src/bin/graphs-interactive-neu.rs +++ b/experiments/src/bin/graphs-interactive-neu.rs @@ -2,7 +2,7 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::probe::Handle; use timely::order::Product; @@ -291,14 +291,14 @@ use differential_dataflow::trace::implementations::ValSpine; use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; -type Arrange = Arranged::Timestamp, R>>>; +type Arrange = Arranged>>; // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn three_hop( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn three_hop( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); @@ -321,13 +321,14 @@ where G::Timestamp: Lattice+Ord { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn _bidijkstra( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn _bidijkstra( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ - goals.scope().iterative::(|inner| { + let outer = goals.scope(); + outer.iterative::(|inner| { // Our plan is to start evolving distances from both sources and destinations. // The evolution from a source or destination should continue as long as there @@ -389,6 +390,6 @@ where G::Timestamp: Lattice+Ord { reverse.set(reverse_next); - reached.leave() + reached.leave(&outer) }) } diff --git a/experiments/src/bin/graphs-interactive.rs b/experiments/src/bin/graphs-interactive.rs index 2ec74e93f..9ee11bf7b 100644 --- a/experiments/src/bin/graphs-interactive.rs +++ b/experiments/src/bin/graphs-interactive.rs @@ -1,6 +1,6 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; +use timely::progress::Timestamp; use timely::dataflow::operators::probe::Handle; use timely::order::Product; @@ -194,15 +194,15 @@ use differential_dataflow::trace::implementations::ValSpine; use differential_dataflow::operators::arrange::TraceAgent; use differential_dataflow::operators::arrange::Arranged; -type Arrange = Arranged::Timestamp, R>>>; +type Arrange = Arranged>>; // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn three_hop( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn three_hop( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); @@ -225,13 +225,14 @@ where G::Timestamp: Lattice+Ord { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn _bidijkstra( - forward_graph: Arrange, - reverse_graph: Arrange, - goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +fn _bidijkstra( + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection +{ - goals.scope().iterative::(|inner| { + let outer = goals.scope(); + outer.iterative::(|inner| { // Our plan is to start evolving distances from both sources and destinations. // The evolution from a source or destination should continue as long as there @@ -293,6 +294,6 @@ where G::Timestamp: Lattice+Ord { reverse.set(reverse_next); - reached.leave() + reached.leave(&outer) }) } diff --git a/experiments/src/bin/graphs-static.rs b/experiments/src/bin/graphs-static.rs index 16cbbd6dd..46a89b4a2 100644 --- a/experiments/src/bin/graphs-static.rs +++ b/experiments/src/bin/graphs-static.rs @@ -1,5 +1,3 @@ -use timely::dataflow::*; - use timely::order::Product; use timely::dataflow::operators::ToStream; @@ -106,14 +104,15 @@ use differential_dataflow::operators::arrange::TraceAgent; type TraceHandle = TraceAgent; -fn reach> ( +fn reach( graph: &mut TraceHandle, - roots: VecCollection -) -> VecCollection { + roots: VecCollection<(), Node, Diff> +) -> VecCollection<(), Node, Diff> { let graph = graph.import(&roots.scope()); - roots.scope().iterative::(|scope| { + let outer = roots.scope(); + outer.iterative::(|scope| { let graph = graph.enter(scope); let roots = roots.enter(scope); @@ -126,20 +125,21 @@ fn reach> ( .threshold_total(|_,_| 1); inner.set(result.clone()); - result.leave() + result.leave(&outer) }) } -fn bfs> ( +fn bfs( graph: &mut TraceHandle, - roots: VecCollection -) -> VecCollection { + roots: VecCollection<(), Node, Diff> +) -> VecCollection<(), (Node, u32), Diff> { let graph = graph.import(&roots.scope()); let roots = roots.map(|r| (r,0)); - roots.scope().iterative::(|scope| { + let outer = roots.scope(); + outer.iterative::(|scope| { let graph = graph.enter(scope); let roots = roots.enter(scope); @@ -151,15 +151,15 @@ fn bfs> ( .reduce(|_key, input, output| output.push((*input[0].0,1))); inner.set(result.clone()); - result.leave() + result.leave(&outer) }) } -fn connected_components>( - scope: &mut G, +fn connected_components( + scope: &mut timely::dataflow::Scope<()>, forward: &mut TraceHandle, reverse: &mut TraceHandle, -) -> VecCollection { +) -> VecCollection<(), (Node, Node), Diff> { let forward = forward.import(scope); let reverse = reverse.import(scope); @@ -169,14 +169,14 @@ fn connected_components>( let nodes_r = reverse.clone().flat_map_ref(|k,v| if k < v { Some(*k) } else { None }); let nodes = nodes_f.concat(nodes_r).consolidate().map(|x| (x,x)); - scope.iterative(|scope| { + scope.iterative(|inner_scope| { // import arrangements, nodes. - let forward = forward.enter(scope); - let reverse = reverse.enter(scope); - let nodes = nodes.enter(scope); + let forward = forward.enter(inner_scope); + let reverse = reverse.enter(inner_scope); + let nodes = nodes.enter(inner_scope); - let (inner, inner_collection) = Variable::new(scope, Product::new(Default::default(), 1)); + let (inner, inner_collection) = Variable::new(inner_scope, Product::new(Default::default(), 1)); let labels = inner_collection.clone().arrange_by_key(); let f_prop = labels.clone().join_core(forward, |_k,l,d| Some((*d,*l))); @@ -197,6 +197,6 @@ fn connected_components>( .reduce(|_, s, t| { t.push((*s[0].0, 1)); }); inner.set(result.clone()); - result.leave() + result.leave(scope) }) } diff --git a/experiments/src/bin/graphs.rs b/experiments/src/bin/graphs.rs index ea7420120..7461d006a 100644 --- a/experiments/src/bin/graphs.rs +++ b/experiments/src/bin/graphs.rs @@ -1,7 +1,5 @@ use rand::{Rng, SeedableRng, StdRng}; -use timely::dataflow::*; - use differential_dataflow::input::Input; use differential_dataflow::VecCollection; use differential_dataflow::operators::*; @@ -88,10 +86,10 @@ use differential_dataflow::operators::arrange::TraceAgent; type TraceHandle = TraceAgent; -fn reach> ( +fn reach( graph: &mut TraceHandle, - roots: VecCollection -) -> VecCollection { + roots: VecCollection<(), Node> +) -> VecCollection<(), Node> { let graph = graph.import(&roots.scope()); @@ -110,10 +108,10 @@ fn reach> ( } -fn bfs> ( +fn bfs( graph: &mut TraceHandle, - roots: VecCollection -) -> VecCollection { + roots: VecCollection<(), Node> +) -> VecCollection<(), (Node, u32)> { let graph = graph.import(&roots.scope()); let roots = roots.map(|r| (r,0)); @@ -129,9 +127,9 @@ fn bfs> ( }) } -// fn connected_components>( +// fn connected_components( // graph: &mut TraceHandle -// ) -> VecCollection { +// ) -> VecCollection<(), (Node, Node)> { // // each edge (x,y) means that we need at least a label for the min of x and y. // let nodes = diff --git a/experiments/src/bin/graspan1.rs b/experiments/src/bin/graspan1.rs index 507f5c46b..571b46f1b 100644 --- a/experiments/src/bin/graspan1.rs +++ b/experiments/src/bin/graspan1.rs @@ -1,7 +1,6 @@ use std::io::{BufRead, BufReader}; use std::fs::File; -use timely::dataflow::Scope; use timely::order::Product; use differential_dataflow::difference::Present; @@ -35,8 +34,9 @@ fn main() { // a N c <- a N b && b E c // N(a,c) <- N(a,b), E(b, c) + let outer = nodes.scope(); let reached = - nodes.scope().iterative::(|inner| { + outer.iterative::(|inner| { let nodes = nodes.enter(inner).map(|(a,b)| (b,a)); let edges = edges.enter(inner); @@ -51,7 +51,7 @@ fn main() { .threshold_semigroup(|_,_,x: Option<&Present>| if x.is_none() { Some(Present) } else { None }); labels.set(next.clone()); - next.leave() + next.leave(&outer) }); reached diff --git a/experiments/src/bin/graspan2.rs b/experiments/src/bin/graspan2.rs index 4ce8574a9..f101ae246 100644 --- a/experiments/src/bin/graspan2.rs +++ b/experiments/src/bin/graspan2.rs @@ -1,7 +1,6 @@ use std::io::{BufRead, BufReader}; use std::fs::File; -use timely::dataflow::Scope; use timely::order::Product; use differential_dataflow::operators::iterate::Variable; @@ -51,14 +50,14 @@ fn unoptimized() { let (value_flow, memory_alias, value_alias) = scope - .iterative::(|scope| { + .iterative::(|inner_scope| { - let nodes = nodes.enter(scope); - let assignment = assignment.enter(scope); - let dereference = dereference.enter(scope); + let nodes = nodes.enter(inner_scope); + let assignment = assignment.enter(inner_scope); + let dereference = dereference.enter(inner_scope); - let (value_flow, value_flow_collection) = Variable::new(scope, Product::new(Default::default(), 1)); - let (memory_alias, memory_alias_collection) = Variable::new(scope, Product::new(Default::default(), 1)); + let (value_flow, value_flow_collection) = Variable::new(inner_scope, Product::new(Default::default(), 1)); + let (memory_alias, memory_alias_collection) = Variable::new(inner_scope, Product::new(Default::default(), 1)); let value_flow_arranged = value_flow_collection.arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>(); let memory_alias_arranged = memory_alias_collection.arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>(); @@ -108,7 +107,7 @@ fn unoptimized() { value_flow.set(value_flow_next.clone()); memory_alias.set(memory_alias_next.clone()); - (value_flow_next.leave(), memory_alias_next.leave(), value_alias_next.leave()) + (value_flow_next.leave(&scope), memory_alias_next.leave(&scope), value_alias_next.leave(&scope)) }); value_flow.map(|_| ()).consolidate().inspect(|x| println!("VF: {:?}", x)); @@ -176,14 +175,14 @@ fn optimized() { let (value_flow, memory_alias) = scope - .iterative::(|scope| { + .iterative::(|inner_scope| { - let nodes = nodes.enter(scope); - let assignment = assignment.enter(scope); - let dereference = dereference.enter(scope); + let nodes = nodes.enter(inner_scope); + let assignment = assignment.enter(inner_scope); + let dereference = dereference.enter(inner_scope); - let (value_flow, value_flow_collection) = Variable::new(scope, Product::new(Default::default(), 1)); - let (memory_alias, memory_alias_collection) = Variable::new(scope, Product::new(Default::default(), 1)); + let (value_flow, value_flow_collection) = Variable::new(inner_scope, Product::new(Default::default(), 1)); + let (memory_alias, memory_alias_collection) = Variable::new(inner_scope, Product::new(Default::default(), 1)); let value_flow_arranged = value_flow_collection.clone().arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>(); let memory_alias_arranged = memory_alias_collection.arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>(); @@ -233,7 +232,7 @@ fn optimized() { value_flow.set(value_flow_next.clone()); memory_alias.set(memory_alias_next.clone()); - (value_flow_next.leave(), memory_alias_next.leave()) + (value_flow_next.leave(&scope), memory_alias_next.leave(&scope)) }); value_flow.map(|_| ()).consolidate().inspect(|x| println!("VF: {:?}", x)); diff --git a/interactive/src/command.rs b/interactive/src/command.rs index 5d110ea70..ba83fca60 100644 --- a/interactive/src/command.rs +++ b/interactive/src/command.rs @@ -4,7 +4,6 @@ use std::hash::Hash; use std::io::Write; use serde::{Deserialize, Serialize}; -use timely::communication::Allocate; use timely::worker::Worker; use timely::logging::TimelyEvent; @@ -48,7 +47,7 @@ where { /// Executes a command. - pub fn execute(self, manager: &mut Manager, worker: &mut Worker) { + pub fn execute(self, manager: &mut Manager, worker: &mut Worker) { match self { diff --git a/interactive/src/logging.rs b/interactive/src/logging.rs index 322dd16d2..0b076f36f 100644 --- a/interactive/src/logging.rs +++ b/interactive/src/logging.rs @@ -3,7 +3,6 @@ use std::hash::Hash; use std::time::Duration; -use timely::communication::Allocate; use timely::worker::Worker; use timely::logging::TimelyEvent; use timely::dataflow::operators::capture::event::EventIterator; @@ -20,16 +19,15 @@ pub trait LoggingValue : VectorFrom+VectorFrom { impl+VectorFrom> LoggingValue for V { } /// Timely logging capture and arrangement. -pub fn publish_timely_logging( +pub fn publish_timely_logging( manager: &mut Manager, - worker: &mut Worker, + worker: &mut Worker, granularity_ns: u64, name: &str, events: I ) where V: ExchangeData+Hash+LoggingValue+Datum, - A: Allocate, I : IntoIterator, ::Item: EventIterator>+'static { @@ -209,16 +207,15 @@ where } /// Timely logging capture and arrangement. -pub fn publish_differential_logging( +pub fn publish_differential_logging( manager: &mut Manager, - worker: &mut Worker, + worker: &mut Worker, granularity_ns: u64, name: &str, events: I ) where V: ExchangeData+Hash+LoggingValue+Datum, - A: Allocate, I : IntoIterator, ::Item: EventIterator>+'static { diff --git a/interactive/src/manager.rs b/interactive/src/manager.rs index 8579a538e..f9a0f3872 100644 --- a/interactive/src/manager.rs +++ b/interactive/src/manager.rs @@ -5,7 +5,6 @@ use std::hash::Hash; // use std::time::Duration; use timely::dataflow::ProbeHandle; -use timely::communication::Allocate; use timely::worker::Worker; use timely::logging::TimelyEventBuilder; @@ -79,7 +78,7 @@ impl Manager // } /// Clear the managed inputs and traces. - pub fn shutdown(&mut self, worker: &mut Worker) { + pub fn shutdown(&mut self, worker: &mut Worker) { self.inputs.sessions.clear(); self.traces.inputs.clear(); self.traces.arrangements.clear(); diff --git a/interactive/src/plan/concat.rs b/interactive/src/plan/concat.rs index 4c27941bb..149b81049 100644 --- a/interactive/src/plan/concat.rs +++ b/interactive/src/plan/concat.rs @@ -19,10 +19,10 @@ impl Render for Concat { type Value = V; - fn render>( + fn render( &self, - scope: &mut S, - arrangements: &mut TraceManager) -> VecCollection, Diff> + scope: &mut Scope