From ce77708a15718d9c65054a8bc4a95ef2d5d2de2f Mon Sep 17 00:00:00 2001 From: Leonid Ryzhyk Date: Tue, 16 Mar 2021 11:04:28 -0700 Subject: [PATCH 01/17] Upgrade to timely and DD v0.12. Recent DD patches include some bug fixes, including at least two bugs that affected our ongoing profiling work. --- CHANGELOG.md | 6 ++++ rust/template/Cargo.toml | 4 +-- rust/template/differential_datalog/Cargo.toml | 6 ++-- .../src/profile_statistics.rs | 33 ------------------- .../differential_datalog/src/program/mod.rs | 8 ++--- .../src/program/worker.rs | 5 ++- .../differential_datalog_test/Cargo.toml | 4 +-- src/Language/DifferentialDatalog/Compile.hs | 4 +-- 8 files changed, 20 insertions(+), 50 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e5d4255c2..115609db4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,6 +3,12 @@ All notable changes to this project will be documented in this file. The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/). +## [Unreleased] + +### Changes + +- Upgraded to timely dataflow and differential dataflow dependencies to v0.12. + ## [0.39.0] - April 11, 2021 ### D3log diff --git a/rust/template/Cargo.toml b/rust/template/Cargo.toml index 88b090ac0..bbece1461 100644 --- a/rust/template/Cargo.toml +++ b/rust/template/Cargo.toml @@ -34,9 +34,9 @@ time = { version = "0.2", features = ["serde"] } ordered-float = { version = "2.0.0", features = ["serde"] } cpuprofiler = { version = "0.0", optional = true } #differential-dataflow = "0.11.0" -differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-2" } +differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-3" } #timely = "0.11" -timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-2" } +timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-3", default-features = false } fnv = "1.0.2" once_cell = "1.4.1" libc = "0.2" diff --git a/rust/template/differential_datalog/Cargo.toml b/rust/template/differential_datalog/Cargo.toml index a8789c016..8b512265c 100644 --- a/rust/template/differential_datalog/Cargo.toml +++ b/rust/template/differential_datalog/Cargo.toml @@ -12,10 +12,10 @@ c_api = [] [dependencies] #differential-dataflow = "0.11.0" -differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-2" } -dogsdogsdogs = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-2" } +differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-3" } +dogsdogsdogs = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-3" } #timely = "0.11" -timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-2" } +timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-3", default-features = false} abomonation = "0.7" ordered-float = { version = "2.0.0", features = ["serde"] } diff --git a/rust/template/differential_datalog/src/profile_statistics.rs b/rust/template/differential_datalog/src/profile_statistics.rs index 11e9b83ff..5bf67ad20 100644 --- a/rust/template/differential_datalog/src/profile_statistics.rs +++ b/rust/template/differential_datalog/src/profile_statistics.rs @@ -189,26 +189,6 @@ impl CSVLogEvent { } } - fn progress_entry( - worker_id: usize, - source_worker: usize, - operator_addr: &[usize], - sequence_number: usize, - is_send: bool, - channel_id: usize, - ) -> CSVLogEvent { - CSVLogEvent { - worker_id, - source_worker: Some(source_worker), - event_type: CSVEventType::Progress, - operator_addr: Some(CSVLogEvent::vec_to_csv_string(operator_addr)), - sequence_number: Some(sequence_number), - is_send: Some(is_send), - channel: Some(channel_id), - ..CSVLogEvent::default() - } - } - fn push_progress(worker_id: usize, operator_id: usize) -> CSVLogEvent { CSVLogEvent { worker_id, @@ -337,19 +317,6 @@ impl Statistics { } } } - TimelyEvent::Progress(p) => { - let e = CSVLogEvent::progress_entry( - worker_index, - p.source, - &p.addr, - p.seq_no, - p.is_send, - p.channel, - ); - self.csv_writer - .serialize(e) - .expect("unable to serialize record"); - } TimelyEvent::PushProgress(p) => { self.csv_writer .serialize(CSVLogEvent::push_progress(worker_index, p.op_id)) diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index 96fa38214..21d213621 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -57,11 +57,9 @@ use dogsdogsdogs::{ calculus::{Differentiate, Integrate}, operators::lookup_map, }; -use timely::communication::{ - initialize::{Configuration, WorkerGuards}, - Allocator, -}; +use timely::communication::{initialize::WorkerGuards, Allocator}; use timely::dataflow::scopes::*; +use timely::execute::Config as TimelyConfig; use timely::order::TotalOrder; use timely::progress::{timestamp::Refines, PathSummary, Timestamp}; use timely::worker::Worker; @@ -999,7 +997,7 @@ impl Program { // Start up timely computation. let worker_guards = timely::execute( - Configuration::Process(number_workers), + TimelyConfig::process(number_workers), move |worker: &mut Worker| -> Result<_, String> { let worker = DDlogWorker::new( worker, diff --git a/rust/template/differential_datalog/src/program/worker.rs b/rust/template/differential_datalog/src/program/worker.rs index f60a214af..5cd93413e 100644 --- a/rust/template/differential_datalog/src/program/worker.rs +++ b/rust/template/differential_datalog/src/program/worker.rs @@ -266,8 +266,8 @@ impl<'a> DDlogWorker<'a> { for trace in session_data.traces.values_mut() { let e = [timestamp]; let ac = AntichainRef::new(&e); - trace.distinguish_since(ac); - trace.advance_by(ac); + trace.set_physical_compaction(ac); + trace.set_logical_compaction(ac); } } @@ -398,7 +398,6 @@ impl<'a> DDlogWorker<'a> { TimelyEvent::GuardedMessage(_) | TimelyEvent::Messages(_) | TimelyEvent::Park(_) - | TimelyEvent::Progress(_) | TimelyEvent::PushProgress(_) => profile_timely, _ => false, diff --git a/rust/template/differential_datalog_test/Cargo.toml b/rust/template/differential_datalog_test/Cargo.toml index 802ee9093..f8fe874b6 100644 --- a/rust/template/differential_datalog_test/Cargo.toml +++ b/rust/template/differential_datalog_test/Cargo.toml @@ -5,11 +5,11 @@ edition = "2018" [dependencies] # differential-dataflow = "0.11.0" -differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-2" } +differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-3" } abomonation = "0.7" fnv = "1.0.2" # timely = "0.11" -timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-2" } +timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-3" } serde = { version = "1.0", features = ["derive"] } erased-serde = "0.3" differential_datalog = { path = "../differential_datalog" } diff --git a/src/Language/DifferentialDatalog/Compile.hs b/src/Language/DifferentialDatalog/Compile.hs index 8fd06bab0..c394a6cb5 100644 --- a/src/Language/DifferentialDatalog/Compile.hs +++ b/src/Language/DifferentialDatalog/Compile.hs @@ -775,8 +775,8 @@ mkCargoToml rs_code crate crate_id = "erased-serde = \"0.3\"" $$ --"differential-dataflow = \"0.11.0\"" $$ --"timely = \"0.11\"" $$ - "differential-dataflow = { git = \"https://github.com/ddlog-dev/differential-dataflow\", branch = \"ddlog-2\" }" $$ - "timely = { git = \"https://github.com/ddlog-dev/timely-dataflow\", branch = \"ddlog-2\" }" $$ + "differential-dataflow = { git = \"https://github.com/ddlog-dev/differential-dataflow\", branch = \"ddlog-3\" }" $$ + "timely = { git = \"https://github.com/ddlog-dev/timely-dataflow\", branch = \"ddlog-3\", default-features = false }" $$ "" $$ dependencies $$ "" $$ From 43348a7599797ac014903dbf474fea24cb604a5a Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Mon, 8 Mar 2021 14:18:11 -0600 Subject: [PATCH 02/17] Split rendering for relations & scc relations into functions --- .../differential_datalog/src/program/mod.rs | 3 + .../src/program/worker.rs | 658 ++++++++++-------- 2 files changed, 382 insertions(+), 279 deletions(-) diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index 21d213621..6ff0e2139 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -83,14 +83,17 @@ const PROF_MSG_BUF_SIZE: usize = 10_000; pub type Response = Result; /// Unique identifier of a DDlog relation. +// TODO: Newtype this for type-safety pub type RelId = usize; /// Unique identifier of an index. +// TODO: Newtype this for type-safety pub type IdxId = usize; /// Unique identifier of an arranged relation. /// The first element of the tuple identifies relation; the second is the index /// of arrangement for the given relation. +// TODO: Newtype this for type-safety pub type ArrId = (RelId, usize); /// Function type used to map the content of a relation diff --git a/rust/template/differential_datalog/src/program/worker.rs b/rust/template/differential_datalog/src/program/worker.rs index 5cd93413e..5d5c36dbb 100644 --- a/rust/template/differential_datalog/src/program/worker.rs +++ b/rust/template/differential_datalog/src/program/worker.rs @@ -5,7 +5,7 @@ use crate::{ arrange::{ArrangedCollection, Arrangements}, ArrId, Dep, Msg, ProgNode, Program, Reply, Update, TS, }, - program::{RelId, Weight}, + program::{RecursiveRelation, RelId, Relation, TKeyAgent, TValAgent, Weight}, variable::Variable, }; use crossbeam_channel::{Receiver, Sender}; @@ -59,6 +59,15 @@ struct SessionData { >, } +type DelayedVarMap = FnvHashMap< + RelId, + ( + RelId, + DDVariable, + Collection, + ), +>; + /// A DDlog timely worker pub struct DDlogWorker<'a> { /// The timely worker instance @@ -435,308 +444,399 @@ impl<'a> DDlogWorker<'a> { fn session_dataflow(&mut self, mut probe: ProbeHandle) -> Result { let program = self.program.clone(); - self.worker.dataflow::(|outer: &mut Child, TS>| -> Result<_, String> { - let mut sessions: FnvHashMap> = FnvHashMap::default(); - let mut collections: FnvHashMap, TS>, DDValue, Weight>> = - HashMap::with_capacity_and_hasher(program.nodes.len(), FnvBuildHasher::default()); - let mut arrangements = FnvHashMap::default(); - - // Create an `Enabled` relation used to enforce the dataflow termination in the - // presence of delayed relations. A delayed relation can potentially generate an - // infinite sequence of outputs for all future timestamps, preventing the - // differential dataflow from terminating. DD can only terminate cleanly when there - // is no data in flight, and will keep spinning forever if new records keep getting - // injected in the dataflow. To prevent this scenario, we join all delayed relations - // with `Enabled`. `Enabled` contains a single record (an empty tuple) as long as - // the program is running. We retract this record on shutdown, hopefully enforcing - // quiescing the dataflow. - // Note: this trick won't be needed once DD supports proactive termination: - // https://github.com/TimelyDataflow/timely-dataflow/issues/306 - let (enabled_session, enabled_collection) = outer.new_collection::<(),Weight>(); - let enabled_arrangement = enabled_collection.arrange_by_self(); - - // Create variables for delayed relations. We will be able to refer to these variables - // inside rules and assign them once all rules have been evaluated. - let delayed_vars: FnvHashMap, TS>, DDValue, Weight>, Collection, TS>, DDValue, Weight>)> = program.delayed_rels.iter().map(|drel| { - let v = DDVariable::new(outer, drel.delay); - - let vcol = with_prof_context( + self.worker.dataflow::( + |outer: &mut Child, TS>| -> Result<_, String> { + let mut sessions: FnvHashMap> = + FnvHashMap::default(); + let mut collections: FnvHashMap< + RelId, + Collection, TS>, DDValue, Weight>, + > = HashMap::with_capacity_and_hasher( + program.nodes.len(), + FnvBuildHasher::default(), + ); + let mut arrangements = FnvHashMap::default(); + + // Create an `Enabled` relation used to enforce the dataflow termination in the + // presence of delayed relations. A delayed relation can potentially generate an + // infinite sequence of outputs for all future timestamps, preventing the + // differential dataflow from terminating. DD can only terminate cleanly when there + // is no data in flight, and will keep spinning forever if new records keep getting + // injected in the dataflow. To prevent this scenario, we join all delayed relations + // with `Enabled`. `Enabled` contains a single record (an empty tuple) as long as + // the program is running. We retract this record on shutdown, hopefully enforcing + // quiescing the dataflow. + // Note: this trick won't be needed once DD supports proactive termination: + // https://github.com/TimelyDataflow/timely-dataflow/issues/306 + let (enabled_session, enabled_collection) = outer.new_collection::<(), Weight>(); + let enabled_arrangement = enabled_collection.arrange_by_self(); + + // Create variables for delayed relations. We will be able to refer to these variables + // inside rules and assign them once all rules have been evaluated. + let delayed_vars: DelayedVarMap<_> = program + .delayed_rels + .iter() + .map(|drel| { + let v = DDVariable::new(outer, drel.delay); + + let vcol = with_prof_context( &format!("join {} with 'Enabled' relation", drel.id), - || lookup_map( - &v, - enabled_arrangement.clone(), - |_: &DDValue, key| *key = (), - move |x, w, _, _| (x.clone(), *w), - (), - (), - (), - ) - ); - (drel.id, (drel.rel_id, v, vcol)) - }).collect(); - - for (nodeid, node) in program.nodes.iter().enumerate() { - match node { - ProgNode::Rel{rel} => { - // Relation may already be in the map if it was created by an `Apply` node - let mut collection = collections - .remove(&rel.id) - .unwrap_or_else(|| { - let (session, collection) = outer.new_collection::(); - sessions.insert(rel.id, session); - - collection - }); - - // apply rules - let rule_collections = rel - .rules - .iter() - .map(|rule| { - program.mk_rule( - rule, - |rid| collections.get(&rid).or_else(|| delayed_vars.get(&rid).map(|v| &v.2)), - Arrangements { - arrangements1: &arrangements, - arrangements2: &FnvHashMap::default(), - }, - true + || { + lookup_map( + &v, + enabled_arrangement.clone(), + |_: &DDValue, key| *key = (), + move |x, w, _, _| (x.clone(), *w), + (), + (), + (), ) - }); - - collection = with_prof_context( - &format!("concatenate rules for {}", rel.name), - || collection.concatenate(rule_collections), + }, ); + (drel.id, (drel.rel_id, v, vcol)) + }) + .collect(); - // don't distinct input collections, as this is already done by the set_update logic - if !rel.input && rel.distinct { - collection = with_prof_context( - &format!("{}.threshold_total", rel.name), - || collection.threshold_total(|_, c| if *c == 0 { 0 } else { 1 }), - ); + for (node_id, node) in program.nodes.iter().enumerate() { + match node { + ProgNode::Rel { rel } => render_relation( + rel, + outer, + &*program, + &mut sessions, + &mut collections, + &mut arrangements, + &delayed_vars, + ), + + &ProgNode::Apply { tfun } => { + // TODO: Add a description field for relation transformers + tfun()(&mut collections); } - // create arrangements - for (i,arr) in rel.arrangements.iter().enumerate() { - with_prof_context( - arr.name(), - || arrangements.insert( - (rel.id, i), - arr.build_arrangement_root(&collection), - ), - ); - } + ProgNode::SCC { rels } => render_scc( + rels, + node_id, + outer, + &*program, + &mut sessions, + &mut collections, + &mut arrangements, + )?, + } + } - collections.insert(rel.id, collection); - }, - &ProgNode::Apply { tfun } => { - tfun()(&mut collections); - }, - ProgNode::SCC { rels } => { - // Preallocate the memory required to store the new relations - sessions.reserve(rels.len()); - collections.reserve(rels.len()); - - // create collections; add them to map; we will overwrite them with - // updated collections returned from the inner scope. - for r in rels.iter() { - let (session, collection) = outer.new_collection::(); - //assert!(!r.rel.input, "input relation in nested scope: {}", r.rel.name); - if r.rel.input { - return Err(format!("input relation in nested scope: {}", r.rel.name)); - } - - sessions.insert(r.rel.id, session); - collections.insert(r.rel.id, collection); - } + for (id, (relid, variable, _)) in delayed_vars.into_iter() { + variable.set( + &collections + .get(&relid) + .ok_or_else(|| { + format!( + "delayed variable {} refers to unknown base relation {}", + id, relid + ) + })? + .consolidate(), + ); + } - // create a nested scope for mutually recursive relations - let new_collections = outer.scoped("recursive component", |inner| -> Result<_, String> { - // create variables for relations defined in the SCC. - let mut vars = HashMap::with_capacity_and_hasher(rels.len(), FnvBuildHasher::default()); - // arrangements created inside the nested scope - let mut local_arrangements = FnvHashMap::default(); - // arrangements entered from global scope - let mut inner_arrangements = FnvHashMap::default(); - - for r in rels.iter() { - let var = Variable::from( - &collections - .get(&r.rel.id) - .ok_or_else(|| format!("failed to find collection with relation ID {}", r.rel.id))? - .enter(inner), - r.distinct, - &r.rel.name, - ); - - vars.insert(r.rel.id, var); - } - - // create arrangements - for rel in rels { - for (i, arr) in rel.rel.arrangements.iter().enumerate() { - // check if arrangement is actually used inside this node - if program.arrangement_used_by_nodes((rel.rel.id, i)).any(|n| n == nodeid) { - with_prof_context( - &format!("local {}", arr.name()), - || local_arrangements.insert( - (rel.rel.id, i), - arr.build_arrangement(&*vars.get(&rel.rel.id)?), - ), - ); - } - } - } - - let dependencies = Program::dependencies(rels.iter().map(|relation| &relation.rel)); - - // collections entered from global scope - let mut inner_collections = HashMap::with_capacity_and_hasher(dependencies.len(), FnvBuildHasher::default()); - - for dep in dependencies { - match dep { - Dep::Rel(relid) => { - assert!(!vars.contains_key(&relid)); - let collection = collections - .get(&relid) - .ok_or_else(|| format!("failed to find collection with relation ID {}", relid))? - .enter(inner); - - inner_collections.insert(relid, collection); - }, - Dep::Arr(arrid) => { - let arrangement = arrangements - .get(&arrid) - .ok_or_else(|| format!("Arr: unknown arrangement {:?}", arrid))? - .enter(inner); - - inner_arrangements.insert(arrid, arrangement); - } - } - } - - // apply rules to variables - for rel in rels { - for rule in &rel.rel.rules { - let c = program.mk_rule( - rule, - |rid| { - vars - .get(&rid) - .map(|v| &(**v)) - .or_else(|| inner_collections.get(&rid)) - }, - Arrangements { - arrangements1: &local_arrangements, - arrangements2: &inner_arrangements, - }, - false - ); - - vars - .get_mut(&rel.rel.id) - .ok_or_else(|| format!("no variable found for relation ID {}", rel.rel.id))? - .add(&c); - } - } - - // bring new relations back to the outer scope - let mut new_collections = HashMap::with_capacity_and_hasher(rels.len(), FnvBuildHasher::default()); - for rel in rels { - let var = vars - .get(&rel.rel.id) - .ok_or_else(|| format!("no variable found for relation ID {}", rel.rel.id))?; - - let mut collection = var.leave(); - // var.distinct() will be called automatically by var.drop() if var has `distinct` flag set - if rel.rel.distinct && !rel.distinct { - collection = with_prof_context( - &format!("{}.distinct_total", rel.rel.name), - || collection.threshold_total(|_,c| if *c == 0 { 0 } else { 1 }), - ); - } - - new_collections.insert(rel.rel.id, collection); - } - - Ok(new_collections) - })?; + for (relid, collection) in collections { + // notify client about changes + if let Some(relation_callback) = &program.get_relation(relid).change_cb { + let relation_callback = relation_callback.clone(); + + let consolidated = + with_prof_context(&format!("consolidate {}", relid), || { + collection.consolidate() + }); - // add new collections to the map - collections.extend(new_collections); - - // create arrangements - for rel in rels { - for (i, arr) in rel.rel.arrangements.iter().enumerate() { - // only if the arrangement is used outside of this node - if arr.queryable() || program.arrangement_used_by_nodes((rel.rel.id, i)).any(|n| n != nodeid) { - with_prof_context( - &format!("global {}", arr.name()), - || -> Result<_, String> { - let collection = collections - .get(&rel.rel.id) - .ok_or_else(|| format!("no collection found for relation ID {}", rel.rel.id))?; - - Ok(arrangements.insert((rel.rel.id, i), arr.build_arrangement(collection))) - } - )?; - } - } + let inspected = with_prof_context(&format!("inspect {}", relid), || { + consolidated.inspect(move |x| { + // assert!(x.2 == 1 || x.2 == -1, "x: {:?}", x); + (relation_callback)(relid, &x.0, x.2) + }) + }); + + with_prof_context(&format!("probe {}", relid), || { + inspected.probe_with(&mut probe) + }); + } + } + + // Attach probes to index arrangements, so we know when all updates + // for a given epoch have been added to the arrangement, and return + // arrangement trace. + let mut traces: BTreeMap = BTreeMap::new(); + for ((relid, arrid), arr) in arrangements.into_iter() { + if let ArrangedCollection::Map(arranged) = arr { + if program.get_relation(relid).arrangements[arrid].queryable() { + arranged + .as_collection(|k, _| k.clone()) + .probe_with(&mut probe); + traces.insert((relid, arrid), arranged.trace.clone()); } } } - }; - for (id, (relid, v, _)) in delayed_vars.into_iter() { - v.set(&collections.get(&relid).ok_or_else(|| format!("delayed variable {} refers to unknown base relation {}", id, relid))?.consolidate()); - }; + Ok(SessionData { + sessions, + enabled_session, + traces, + }) + }, + ) + } +} - for (relid, collection) in collections { - // notify client about changes - if let Some(relation_callback) = &program.get_relation(relid).change_cb { - let relation_callback = relation_callback.clone(); +fn render_relation<'a>( + relation: &Relation, + // TODO: Shift to generic representations for ddflow-related structs + scope: &mut Child<'a, Worker, TS>, + program: &Program, + sessions: &mut FnvHashMap>, + collections: &mut FnvHashMap< + RelId, + Collection, TS>, DDValue, Weight>, + >, + arrangements: &mut FnvHashMap< + ArrId, + ArrangedCollection< + Child<'a, Worker, TS>, + TValAgent, TS>>, + TKeyAgent, TS>>, + >, + >, + delayed_vars: &DelayedVarMap, TS>>, +) { + // Relation may already be in the map if it was created by an `Apply` node + let mut collection = collections.remove(&relation.id).unwrap_or_else(|| { + let (session, collection) = scope.new_collection::(); + sessions.insert(relation.id, session); + + collection + }); + + // apply rules + let rule_collections = relation.rules.iter().map(|rule| { + program.mk_rule( + rule, + |rid| { + collections + .get(&rid) + .or_else(|| delayed_vars.get(&rid).map(|v| &v.2)) + }, + Arrangements { + arrangements1: &arrangements, + arrangements2: &FnvHashMap::default(), + }, + true, + ) + }); + + if rule_collections.len() > 1 { + collection = with_prof_context(&format!("concatenate rules for {}", relation.name), || { + collection.concatenate(rule_collections) + }); + } - let consolidated = with_prof_context( - &format!("consolidate {}", relid), - || collection.consolidate(), - ); + // don't distinct input collections, as this is already done by the set_update logic + if !relation.input && relation.distinct { + collection = with_prof_context(&format!("{}.threshold_total", relation.name), || { + collection.threshold_total(|_, c| if *c == 0 { 0 } else { 1 }) + }); + } - let inspected = with_prof_context( - &format!("inspect {}", relid), - || consolidated.inspect(move |x| { - // assert!(x.2 == 1 || x.2 == -1, "x: {:?}", x); - (relation_callback)(relid, &x.0, x.2) - }), - ); + // create arrangements + for (i, arr) in relation.arrangements.iter().enumerate() { + with_prof_context(arr.name(), || { + arrangements.insert((relation.id, i), arr.build_arrangement_root(&collection)) + }); + } - with_prof_context( - &format!("probe {}", relid), - || inspected.probe_with(&mut probe), - ); + collections.insert(relation.id, collection); +} + +fn render_scc<'a>( + rels: &[RecursiveRelation], + node_id: RelId, + // TODO: Shift to generic representations for ddflow-related structs + scope: &mut Child<'a, Worker, TS>, + program: &Program, + sessions: &mut FnvHashMap>, + collections: &mut FnvHashMap< + RelId, + Collection, TS>, DDValue, Weight>, + >, + arrangements: &mut FnvHashMap< + ArrId, + ArrangedCollection< + Child<'a, Worker, TS>, + TValAgent, TS>>, + TKeyAgent, TS>>, + >, + >, +) -> Result<(), String> { + // Preallocate the memory required to store the new relations + sessions.reserve(rels.len()); + collections.reserve(rels.len()); + + // create collections; add them to map; we will overwrite them with + // updated collections returned from the inner scope. + for r in rels.iter() { + let (session, collection) = scope.new_collection::(); + //assert!(!r.rel.input, "input relation in nested scope: {}", r.rel.name); + if r.rel.input { + return Err(format!("input relation in nested scope: {}", r.rel.name)); + } + + sessions.insert(r.rel.id, session); + collections.insert(r.rel.id, collection); + } + + // create a nested scope for mutually recursive relations + let new_collections = scope.scoped("recursive component", |inner| -> Result<_, String> { + // create variables for relations defined in the SCC. + let mut vars = HashMap::with_capacity_and_hasher(rels.len(), FnvBuildHasher::default()); + // arrangements created inside the nested scope + let mut local_arrangements = FnvHashMap::default(); + // arrangements entered from global scope + let mut inner_arrangements = FnvHashMap::default(); + + for r in rels.iter() { + let var = Variable::from( + &collections + .get(&r.rel.id) + .ok_or_else(|| { + format!("failed to find collection with relation ID {}", r.rel.id) + })? + .enter(inner), + r.distinct, + &r.rel.name, + ); + + vars.insert(r.rel.id, var); + } + + // create arrangements + for rel in rels { + for (i, arr) in rel.rel.arrangements.iter().enumerate() { + // check if arrangement is actually used inside this node + if program + .arrangement_used_by_nodes((rel.rel.id, i)) + .any(|n| n == node_id) + { + with_prof_context(&format!("local {}", arr.name()), || { + local_arrangements.insert( + (rel.rel.id, i), + arr.build_arrangement(&*vars.get(&rel.rel.id)?), + ) + }); } } + } - // Attach probes to index arrangements, so we know when all updates - // for a given epoch have been added to the arrangement, and return - // arrangement trace. - let mut traces: BTreeMap = BTreeMap::new(); - for ((relid, arrid), arr) in arrangements.into_iter() { - if let ArrangedCollection::Map(arranged) = arr { - if program.get_relation(relid).arrangements[arrid].queryable() { - arranged.as_collection(|k,_| k.clone()).probe_with(&mut probe); - traces.insert((relid, arrid), arranged.trace.clone()); - } + let dependencies = Program::dependencies(rels.iter().map(|relation| &relation.rel)); + + // collections entered from global scope + let mut inner_collections = + HashMap::with_capacity_and_hasher(dependencies.len(), FnvBuildHasher::default()); + + for dep in dependencies { + match dep { + Dep::Rel(relid) => { + assert!(!vars.contains_key(&relid)); + let collection = collections + .get(&relid) + .ok_or_else(|| { + format!("failed to find collection with relation ID {}", relid) + })? + .enter(inner); + + inner_collections.insert(relid, collection); } + Dep::Arr(arrid) => { + let arrangement = arrangements + .get(&arrid) + .ok_or_else(|| format!("Arr: unknown arrangement {:?}", arrid))? + .enter(inner); + + inner_arrangements.insert(arrid, arrangement); + } + } + } + + // apply rules to variables + for rel in rels { + for rule in &rel.rel.rules { + let c = program.mk_rule( + rule, + |rid| { + vars.get(&rid) + .map(|v| &(**v)) + .or_else(|| inner_collections.get(&rid)) + }, + Arrangements { + arrangements1: &local_arrangements, + arrangements2: &inner_arrangements, + }, + false, + ); + + vars.get_mut(&rel.rel.id) + .ok_or_else(|| format!("no variable found for relation ID {}", rel.rel.id))? + .add(&c); + } + } + + // bring new relations back to the outer scope + let mut new_collections = + HashMap::with_capacity_and_hasher(rels.len(), FnvBuildHasher::default()); + for rel in rels { + let var = vars + .get(&rel.rel.id) + .ok_or_else(|| format!("no variable found for relation ID {}", rel.rel.id))?; + + let mut collection = var.leave(); + // var.distinct() will be called automatically by var.drop() if var has `distinct` flag set + if rel.rel.distinct && !rel.distinct { + collection = with_prof_context(&format!("{}.distinct_total", rel.rel.name), || { + collection.threshold_total(|_, c| if *c == 0 { 0 } else { 1 }) + }); } - Ok(SessionData{ - sessions, - enabled_session, - traces - }) - }) + new_collections.insert(rel.rel.id, collection); + } + + Ok(new_collections) + })?; + + // add new collections to the map + collections.extend(new_collections); + + // create arrangements + for rel in rels { + for (i, arr) in rel.rel.arrangements.iter().enumerate() { + // only if the arrangement is used outside of this node + if arr.queryable() + || program + .arrangement_used_by_nodes((rel.rel.id, i)) + .any(|n| n != node_id) + { + with_prof_context( + &format!("global {}", arr.name()), + || -> Result<_, String> { + let collection = collections.get(&rel.rel.id).ok_or_else(|| { + format!("no collection found for relation ID {}", rel.rel.id) + })?; + + Ok(arrangements.insert((rel.rel.id, i), arr.build_arrangement(collection))) + }, + )?; + } + } } + + Ok(()) } #[derive(Clone)] From 17ee0e7e4cf38c02251cd742631c5c9925c1479e Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Mon, 8 Mar 2021 17:02:12 -0600 Subject: [PATCH 03/17] Added basic regions for standalone Relations --- .../src/program/arrange.rs | 27 ++++- .../differential_datalog/src/program/mod.rs | 20 ++-- .../src/program/worker.rs | 109 +++++++++++------- 3 files changed, 106 insertions(+), 50 deletions(-) diff --git a/rust/template/differential_datalog/src/program/arrange.rs b/rust/template/differential_datalog/src/program/arrange.rs index 7c06672c9..3c193020f 100644 --- a/rust/template/differential_datalog/src/program/arrange.rs +++ b/rust/template/differential_datalog/src/program/arrange.rs @@ -53,8 +53,31 @@ where TKeyEnter>, > { match self { - ArrangedCollection::Map(arr) => ArrangedCollection::Map(arr.enter(inner)), - ArrangedCollection::Set(arr) => ArrangedCollection::Set(arr.enter(inner)), + Self::Map(arr) => ArrangedCollection::Map(arr.enter(inner)), + Self::Set(arr) => ArrangedCollection::Set(arr.enter(inner)), + } + } + + pub fn enter_region<'a>( + &self, + region: &Child<'a, S, S::Timestamp>, + ) -> ArrangedCollection, TValAgent, TKeyAgent> { + match self { + Self::Map(arr) => ArrangedCollection::Map(arr.enter_region(region)), + Self::Set(arr) => ArrangedCollection::Set(arr.enter_region(region)), + } + } +} + +impl<'a, S> ArrangedCollection, TValAgent, TKeyAgent> +where + S: Scope, + S::Timestamp: Lattice + Ord, +{ + pub fn leave_region(&self) -> ArrangedCollection, TKeyAgent> { + match self { + Self::Map(arr) => ArrangedCollection::Map(arr.leave_region()), + Self::Set(arr) => ArrangedCollection::Set(arr.leave_region()), } } } diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index 6ff0e2139..ee7b1e35a 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -261,6 +261,12 @@ pub struct Relation { pub change_cb: Option>, } +impl Relation { + pub fn name(&self) -> &str { + &*self.name + } +} + /// `DelayedRelation` refers to the contents of a given base relation from /// `delay` epochs ago. /// @@ -1216,7 +1222,7 @@ impl Program { P::Timestamp: Lattice, T: Refines + Lattice + Timestamp + Ord, T: ToTupleTS, - LC: Fn(RelId) -> Option<&'b Collection, DDValue, Weight>>, + LC: Fn(RelId) -> Option, DDValue, Weight>>, { match xform { None => col, @@ -1242,7 +1248,7 @@ impl Program { P::Timestamp: Lattice, T: Refines + Lattice + Timestamp + Ord, T: ToTupleTS, - LC: Fn(RelId) -> Option<&'b Collection, DDValue, Weight>>, + LC: Fn(RelId) -> Option, DDValue, Weight>>, { match *xform { XFormCollection::Arrange { @@ -1458,9 +1464,9 @@ impl Program { ); let arrangements1 = FnvHashMap::default(); let arrangements2 = FnvHashMap::default(); - fn dummy_lookup_collection<'c, S: Scope>( + fn dummy_lookup_collection( _: RelId, - ) -> Option<&'c Collection> { + ) -> Option> { None } let xformed = Self::xform_collection( @@ -1508,7 +1514,7 @@ impl Program { TR: TraceReader + Clone + 'static, TR::Batch: BatchReader, TR::Cursor: Cursor, - LC: Fn(RelId) -> Option<&'b Collection, DDValue, Weight>>, + LC: Fn(RelId) -> Option, DDValue, Weight>>, { match *xform { XFormArrangement::FlatMap { @@ -1831,7 +1837,7 @@ impl Program { P::Timestamp: Lattice, T: Refines + Lattice + Timestamp + Ord, T: ToTupleTS, - F: Fn(RelId) -> Option<&'b Collection, DDValue, Weight>>, + F: Fn(RelId) -> Option, DDValue, Weight>>, 'a: 'b, { match rule { @@ -1852,7 +1858,7 @@ impl Program { xform: Some(x), .. } => Self::xform_collection_ref( - lookup_collection(*rel) + &lookup_collection(*rel) .unwrap_or_else(|| panic!("mk_rule: unknown relation {:?}", rel)), x, &arrangements, diff --git a/rust/template/differential_datalog/src/program/worker.rs b/rust/template/differential_datalog/src/program/worker.rs index 5d5c36dbb..08d813128 100644 --- a/rust/template/differential_datalog/src/program/worker.rs +++ b/rust/template/differential_datalog/src/program/worker.rs @@ -28,6 +28,7 @@ use fnv::{FnvBuildHasher, FnvHashMap}; use std::{ collections::{BTreeMap, BTreeSet, HashMap}, mem, + ops::Deref, rc::Rc, sync::{ atomic::{AtomicBool, Ordering}, @@ -608,54 +609,79 @@ fn render_relation<'a>( >, delayed_vars: &DelayedVarMap, TS>>, ) { - // Relation may already be in the map if it was created by an `Apply` node - let mut collection = collections.remove(&relation.id).unwrap_or_else(|| { - let (session, collection) = scope.new_collection::(); - sessions.insert(relation.id, session); - - collection - }); + scope.clone().region_named(relation.name(), |region| { + // Relation may already be in the map if it was created by an `Apply` node + let mut collection = if let Some(collection) = collections.remove(&relation.id) { + collection.enter_region(region) + } else { + let (session, collection) = scope.new_collection::(); + sessions.insert(relation.id, session); + + // TODO: Find a way to make the collection within the nested region + collection.enter_region(region) + }; - // apply rules - let rule_collections = relation.rules.iter().map(|rule| { - program.mk_rule( - rule, - |rid| { - collections - .get(&rid) - .or_else(|| delayed_vars.get(&rid).map(|v| &v.2)) - }, - Arrangements { - arrangements1: &arrangements, - arrangements2: &FnvHashMap::default(), - }, - true, - ) - }); + let entered_arrangements: FnvHashMap<_, _> = arrangements + .iter() + .map(|(&arr_id, arr)| (arr_id, arr.enter_region(region))) + .collect(); - if rule_collections.len() > 1 { - collection = with_prof_context(&format!("concatenate rules for {}", relation.name), || { - collection.concatenate(rule_collections) + // apply rules + // TODO: Regions for rules + let rule_collections = relation.rules.iter().map(|rule| { + let get_rule_collection = |rule_id| { + if let Some(collection) = collections.get(&rule_id) { + Some(collection.enter_region(region)) + } else { + delayed_vars + .get(&rule_id) + .map(|(_, _, collection)| collection.enter_region(region)) + } + }; + + program.mk_rule( + rule, + get_rule_collection, + Arrangements { + arrangements1: &entered_arrangements, + arrangements2: &FnvHashMap::default(), + }, + true, + ) }); - } - // don't distinct input collections, as this is already done by the set_update logic - if !relation.input && relation.distinct { - collection = with_prof_context(&format!("{}.threshold_total", relation.name), || { - collection.threshold_total(|_, c| if *c == 0 { 0 } else { 1 }) - }); - } + if rule_collections.len() > 1 { + collection = + with_prof_context(&format!("concatenate rules for {}", relation.name), || { + collection.concatenate(rule_collections) + }); + } - // create arrangements - for (i, arr) in relation.arrangements.iter().enumerate() { - with_prof_context(arr.name(), || { - arrangements.insert((relation.id, i), arr.build_arrangement_root(&collection)) - }); - } + // don't distinct input collections, as this is already done by the set_update logic + if !relation.input && relation.distinct { + collection = with_prof_context(&format!("{}.threshold_total", relation.name), || { + collection.threshold_total(|_, c| if *c == 0 { 0 } else { 1 }) + }); + } + + // create arrangements + // TODO: Arrangements have their own shebang, region them off too + for (arr_id, arrangement) in relation.arrangements.iter().enumerate() { + with_prof_context(arrangement.name(), || { + arrangements.insert( + (relation.id, arr_id), + arrangement + .build_arrangement_root(&collection) + .leave_region(), + ) + }); + } - collections.insert(relation.id, collection); + collections.insert(relation.id, collection.leave_region()); + }); } +// TODO: Regions for SCCs fn render_scc<'a>( rels: &[RecursiveRelation], node_id: RelId, @@ -772,8 +798,9 @@ fn render_scc<'a>( rule, |rid| { vars.get(&rid) - .map(|v| &(**v)) + .map(|v| v.deref()) .or_else(|| inner_collections.get(&rid)) + .cloned() }, Arrangements { arrangements1: &local_arrangements, From 69aa2a735273816c41989309a8e2545788d9cf74 Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Tue, 9 Mar 2021 14:27:15 -0600 Subject: [PATCH 04/17] Reworked rendering for root & normal arrangements --- .../src/dataflow/arrange.rs | 99 +++++++ .../src/dataflow/consolidate.rs | 53 ++++ .../src/dataflow/distinct.rs | 69 +++++ .../src/dataflow/filter_map.rs | 69 +++++ .../differential_datalog/src/dataflow/map.rs | 62 +++++ .../differential_datalog/src/dataflow/mod.rs | 10 + rust/template/differential_datalog/src/lib.rs | 2 + .../src/program/arrange.rs | 97 ++++--- .../differential_datalog/src/program/mod.rs | 93 ++++--- .../src/program/worker.rs | 12 +- .../src/render/arrange_by.rs | 250 ++++++++++++++++++ .../differential_datalog/src/render/mod.rs | 15 ++ .../differential_datalog/src/replay.rs | 18 +- src/Language/DifferentialDatalog/Compile.hs | 8 + 14 files changed, 771 insertions(+), 86 deletions(-) create mode 100644 rust/template/differential_datalog/src/dataflow/arrange.rs create mode 100644 rust/template/differential_datalog/src/dataflow/consolidate.rs create mode 100644 rust/template/differential_datalog/src/dataflow/distinct.rs create mode 100644 rust/template/differential_datalog/src/dataflow/filter_map.rs create mode 100644 rust/template/differential_datalog/src/dataflow/map.rs create mode 100644 rust/template/differential_datalog/src/dataflow/mod.rs create mode 100644 rust/template/differential_datalog/src/render/arrange_by.rs create mode 100644 rust/template/differential_datalog/src/render/mod.rs diff --git a/rust/template/differential_datalog/src/dataflow/arrange.rs b/rust/template/differential_datalog/src/dataflow/arrange.rs new file mode 100644 index 000000000..12dca3f4f --- /dev/null +++ b/rust/template/differential_datalog/src/dataflow/arrange.rs @@ -0,0 +1,99 @@ +use differential_dataflow::{ + difference::Semigroup, + lattice::Lattice, + operators::arrange::{Arrange, Arranged, TraceAgent}, + trace::implementations::ord::{OrdKeySpine, OrdValSpine}, + Collection, ExchangeData, Hashable, +}; +use timely::dataflow::{ + channels::pact::{Exchange, Pipeline}, + Scope, +}; + +pub trait ArrangeByKeyExt { + type Output; + + fn arrange_by_key_exchange(&self, route: F) -> Self::Output + where + F: Fn(&K, &V) -> u64 + 'static, + { + self.arrange_by_key_exchange_named("ArrangeByKeyExchange", route) + } + + fn arrange_by_key_exchange_named(&self, name: &str, route: F) -> Self::Output + where + F: Fn(&K, &V) -> u64 + 'static; + + fn arrange_by_key_pipelined(&self) -> Self::Output { + self.arrange_by_key_pipelined_named("ArrangeByKeyPipelined") + } + + fn arrange_by_key_pipelined_named(&self, name: &str) -> Self::Output; +} + +impl ArrangeByKeyExt for Collection +where + S: Scope, + S::Timestamp: Lattice, + K: ExchangeData + Hashable, + V: ExchangeData, + R: Semigroup + ExchangeData, +{ + #[allow(clippy::type_complexity)] + type Output = Arranged>>; + + fn arrange_by_key_exchange_named(&self, name: &str, route: F) -> Self::Output + where + F: Fn(&K, &V) -> u64 + 'static, + { + let exchange = Exchange::new(move |((key, value), _time, _diff)| route(key, value)); + self.arrange_core(exchange, name) + } + + fn arrange_by_key_pipelined_named(&self, name: &str) -> Self::Output { + self.arrange_core(Pipeline, name) + } +} + +pub trait ArrangeBySelfExt { + type Output; + + fn arrange_by_self_exchange(&self, route: F) -> Self::Output + where + F: Fn(&K) -> u64 + 'static, + { + self.arrange_by_self_exchange_named("ArrangeBySelfExchange", route) + } + + fn arrange_by_self_exchange_named(&self, name: &str, route: F) -> Self::Output + where + F: Fn(&K) -> u64 + 'static; + + fn arrange_by_self_pipelined(&self) -> Self::Output { + self.arrange_by_self_pipelined_named("ArrangeBySelfPipelined") + } + + fn arrange_by_self_pipelined_named(&self, name: &str) -> Self::Output; +} + +impl ArrangeBySelfExt for Collection +where + S: Scope, + S::Timestamp: Lattice, + K: ExchangeData + Hashable, + R: Semigroup + ExchangeData, +{ + type Output = Arranged>>; + + fn arrange_by_self_exchange_named(&self, name: &str, route: F) -> Self::Output + where + F: Fn(&K) -> u64 + 'static, + { + let exchange = Exchange::new(move |((key, ()), _time, _diff)| route(key)); + self.map(|key| (key, ())).arrange_core(exchange, name) + } + + fn arrange_by_self_pipelined_named(&self, name: &str) -> Self::Output { + self.map(|key| (key, ())).arrange_core(Pipeline, name) + } +} diff --git a/rust/template/differential_datalog/src/dataflow/consolidate.rs b/rust/template/differential_datalog/src/dataflow/consolidate.rs new file mode 100644 index 000000000..77fccbe2e --- /dev/null +++ b/rust/template/differential_datalog/src/dataflow/consolidate.rs @@ -0,0 +1,53 @@ +use differential_dataflow::{ + difference::Semigroup, + lattice::Lattice, + operators::arrange::{Arrange, Arranged, TraceAgent}, + trace::{implementations::ord::OrdKeySpine, layers::ordered::OrdOffset}, + Collection, ExchangeData, Hashable, +}; +use std::{ + convert::{TryFrom, TryInto}, + fmt::Debug, +}; +use timely::dataflow::Scope; + +pub trait ConsolidateExt +where + S: Scope, + S::Timestamp: Lattice, + D: ExchangeData, + R: Semigroup + ExchangeData, + O: OrdOffset, + >::Error: Debug, + >::Error: Debug, +{ + /// A `.consolidate()` that returns its internal arrangement + fn consolidate_arranged(&self) -> Arranged>> { + self.consolidate_arranged_named("ConsolidateArranged") + } + + /// The same as `.consolidate_arranged()` but with the ability to name the operator. + fn consolidate_arranged_named( + &self, + name: &str, + ) -> Arranged>>; +} + +impl ConsolidateExt for Collection +where + S: Scope, + S::Timestamp: Lattice, + D: ExchangeData + Hashable, + R: Semigroup + ExchangeData, + O: OrdOffset, + >::Error: Debug, + >::Error: Debug, +{ + fn consolidate_arranged_named( + &self, + name: &str, + ) -> Arranged>> { + // TODO: Name this map? + self.map(|key| (key, ())).arrange_named(name) + } +} diff --git a/rust/template/differential_datalog/src/dataflow/distinct.rs b/rust/template/differential_datalog/src/dataflow/distinct.rs new file mode 100644 index 000000000..c27ebe4db --- /dev/null +++ b/rust/template/differential_datalog/src/dataflow/distinct.rs @@ -0,0 +1,69 @@ +use crate::dataflow::{ConsolidateExt, MapExt}; +use differential_dataflow::{ + difference::Abelian, + lattice::Lattice, + operators::{ + arrange::{ArrangeBySelf, Arranged, TraceAgent}, + Reduce, + }, + trace::{implementations::ord::OrdKeySpine, layers::ordered::OrdOffset}, + Collection, ExchangeData, Hashable, +}; +use std::{ + convert::{TryFrom, TryInto}, + fmt::Debug, + ops::Add, +}; +use timely::dataflow::Scope; + +/// An alternative implementation of `distinct`. +/// +/// The implementation of `distinct` in differential dataflow maintains both its input and output +/// arrangements. This implementation, suggested by @frankmcsherry instead uses a single +/// arrangement that produces the number of "surplus" records, which are then subtracted from the +/// input to get an output with distinct records. This has the advantage that for keys that are +/// already distinct, there is no additional memory used in the output (nothing to subtract). It +/// has the downside that if the input changes a lot, the output may have more changes (to track +/// the input changes) than if it just recorded distinct records (which is pretty stable). +pub fn diff_distinct( + relation_name: &str, + collection: &Collection, +) -> Arranged>> +where + S: Scope, + S::Timestamp: Lattice, + D: ExchangeData + Hashable, + R: Abelian + ExchangeData + Add + From, + O: OrdOffset, + >::Error: Debug, + >::Error: Debug, +{ + // For each value with weight w != 1, compute an adjustment record with the same value and + // weight (1-w) + // TODO: What happens when negative weights get into this? + let negated = collection + .arrange_by_self_named(&format!( + "ArrangeBySelf: DiffDistinct for {}", + relation_name + )) + .reduce_named( + &format!("Reduce: DiffDistinct for {}", relation_name), + |_, src, dst| { + // If the input weight is 1, don't produce a surplus record. + if src[0].1 != R::from(1) { + dst.push(((), R::from(1) + src[0].1.clone().neg())) + } + }, + ) + .map_named(&format!("Map: DiffDistinct for {}", relation_name), |x| x.0); + + collection + // TODO: `.concat_named()`? + .concat(&negated) + // We directly return the consolidation arrangement, + // allowing us to potentially skip re-arranging it later + .consolidate_arranged_named(&format!( + "ConsolidateArranged: DiffDistinct for {}", + relation_name, + )) +} diff --git a/rust/template/differential_datalog/src/dataflow/filter_map.rs b/rust/template/differential_datalog/src/dataflow/filter_map.rs new file mode 100644 index 000000000..86b8b3bb7 --- /dev/null +++ b/rust/template/differential_datalog/src/dataflow/filter_map.rs @@ -0,0 +1,69 @@ +use differential_dataflow::{collection::AsCollection, difference::Semigroup, Collection}; +use timely::{ + dataflow::{channels::pact::Pipeline, operators::Operator, Scope, Stream}, + Data, +}; + +pub trait FilterMap { + type Output; + + fn filter_map(&self, logic: L) -> Self::Output + where + L: FnMut(D) -> Option + 'static, + { + self.filter_map_named("FilterMap", logic) + } + + fn filter_map_named(&self, name: &str, logic: L) -> Self::Output + where + L: FnMut(D) -> Option + 'static; +} + +impl FilterMap for Stream +where + S: Scope, + D: Data, + D2: Data, +{ + type Output = Stream; + + fn filter_map_named(&self, name: &str, mut logic: L) -> Self::Output + where + L: FnMut(D) -> Option + 'static, + { + let mut buffer = Vec::new(); + + self.unary(Pipeline, name, move |_capability, _info| { + move |input, output| { + input.for_each(|capability, data| { + data.swap(&mut buffer); + + output + .session(&capability) + .give_iterator(buffer.drain(..).filter_map(|data| logic(data))); + }); + } + }) + } +} + +impl FilterMap for Collection +where + S: Scope, + D: Data, + D2: Data, + R: Semigroup, +{ + type Output = Collection; + + fn filter_map_named(&self, name: &str, mut logic: L) -> Self::Output + where + L: FnMut(D) -> Option + 'static, + { + self.inner + .filter_map_named(name, move |(data, time, diff)| { + logic(data).map(|data| (data, time, diff)) + }) + .as_collection() + } +} diff --git a/rust/template/differential_datalog/src/dataflow/map.rs b/rust/template/differential_datalog/src/dataflow/map.rs new file mode 100644 index 000000000..5bfc5387c --- /dev/null +++ b/rust/template/differential_datalog/src/dataflow/map.rs @@ -0,0 +1,62 @@ +use differential_dataflow::{difference::Semigroup, AsCollection, Collection}; +use timely::{ + dataflow::{channels::pact::Pipeline, operators::Operator, Scope, Stream}, + Data, +}; + +pub trait MapExt { + type Output; + + fn map_named(&self, name: &str, logic: L) -> Self::Output + where + D1: Data, + D2: Data, + L: FnMut(D1) -> D2 + 'static; +} + +impl MapExt for Stream +where + S: Scope, + D1: Data, + D2: Data, +{ + type Output = Stream; + + fn map_named(&self, name: &str, mut logic: L) -> Self::Output + where + L: FnMut(D1) -> D2 + 'static, + { + let mut buffer = Vec::new(); + + self.unary(Pipeline, name, move |_, _| { + move |input, output| { + input.for_each(|time, data| { + data.swap(&mut buffer); + + output + .session(&time) + .give_iterator(buffer.drain(..).map(|x| logic(x))); + }); + } + }) + } +} + +impl MapExt for Collection +where + S: Scope, + D1: Data, + D2: Data, + R: Semigroup, +{ + type Output = Collection; + + fn map_named(&self, name: &str, mut logic: L) -> Self::Output + where + L: FnMut(D1) -> D2 + 'static, + { + self.inner + .map_named(name, move |(data, time, diff)| (logic(data), time, diff)) + .as_collection() + } +} diff --git a/rust/template/differential_datalog/src/dataflow/mod.rs b/rust/template/differential_datalog/src/dataflow/mod.rs new file mode 100644 index 000000000..9a9d693b9 --- /dev/null +++ b/rust/template/differential_datalog/src/dataflow/mod.rs @@ -0,0 +1,10 @@ +mod arrange; +mod consolidate; +mod distinct; +mod filter_map; +mod map; + +pub use consolidate::ConsolidateExt; +pub use distinct::diff_distinct; +pub use filter_map::FilterMap; +pub use map::MapExt; diff --git a/rust/template/differential_datalog/src/lib.rs b/rust/template/differential_datalog/src/lib.rs index bf25d1b36..4de13329e 100644 --- a/rust/template/differential_datalog/src/lib.rs +++ b/rust/template/differential_datalog/src/lib.rs @@ -6,9 +6,11 @@ )] mod callback; +mod dataflow; mod ddlog; mod profile; mod profile_statistics; +mod render; pub mod replay; mod valmap; mod variable; diff --git a/rust/template/differential_datalog/src/program/arrange.rs b/rust/template/differential_datalog/src/program/arrange.rs index 3c193020f..ebb290981 100644 --- a/rust/template/differential_datalog/src/program/arrange.rs +++ b/rust/template/differential_datalog/src/program/arrange.rs @@ -2,7 +2,7 @@ use crate::{ ddval::DDValue, - program::{ArrId, TKeyAgent, TKeyEnter, TSNested, TValAgent, TValEnter, Weight}, + program::{ArrId, TKeyAgent, TKeyEnter, TValAgent, TValEnter, Weight}, }; use differential_dataflow::{ difference::{Diff, Monoid}, @@ -12,72 +12,85 @@ use differential_dataflow::{ arrange::arrangement::{ArrangeBySelf, Arranged}, Consolidate, JoinCore, Reduce, }, - trace::{BatchReader, Cursor, TraceReader}, + trace::{wrappers::enter::TraceEnter, BatchReader, Cursor, TraceReader}, Collection, Data, ExchangeData, }; use fnv::FnvHashMap; -use num::One; use std::ops::{Add, Mul, Neg}; use timely::{ dataflow::scopes::{Child, Scope, ScopeParent}, - order::Product, progress::{timestamp::Refines, Timestamp}, }; -pub(super) enum ArrangedCollection +pub enum Arrangement where S: Scope, - S::Timestamp: Lattice + Ord, - T1: TraceReader + Clone, - T1::Batch: BatchReader, - T1::Cursor: Cursor, - T2: TraceReader + Clone, - T2::Batch: BatchReader, - T2::Cursor: Cursor, + S::Timestamp: Lattice, + Map: TraceReader + Clone + 'static, + Map::Batch: BatchReader + Clone + 'static, + Map::Cursor: Cursor, + Set: TraceReader + Clone + 'static, + Set::Batch: BatchReader + Clone + 'static, + Set::Cursor: Cursor, { - Map(Arranged), - Set(Arranged), + Map(Arranged), + Set(Arranged), } -impl ArrangedCollection, TKeyAgent> +impl Arrangement where S: Scope, S::Timestamp: Lattice + Ord, + Map: TraceReader + Clone + 'static, + Map::Batch: BatchReader + Clone + 'static, + Map::Cursor: Cursor, + Set: TraceReader + Clone + 'static, + Set::Batch: BatchReader + Clone + 'static, + Set::Cursor: Cursor, { - pub(super) fn enter<'a>( + pub(super) fn enter<'a, TInner>( &self, - inner: &Child<'a, S, Product>, - ) -> ArrangedCollection< - Child<'a, S, Product>, - TValEnter>, - TKeyEnter>, - > { + inner: &Child<'a, S, TInner>, + ) -> Arrangement, R, TraceEnter, TraceEnter> + where + R: 'static, + TInner: Refines + Lattice + Timestamp + Clone + 'static, + { match self { - Self::Map(arr) => ArrangedCollection::Map(arr.enter(inner)), - Self::Set(arr) => ArrangedCollection::Set(arr.enter(inner)), + Self::Map(arr) => Arrangement::Map(arr.enter(inner)), + Self::Set(arr) => Arrangement::Set(arr.enter(inner)), } } pub fn enter_region<'a>( &self, region: &Child<'a, S, S::Timestamp>, - ) -> ArrangedCollection, TValAgent, TKeyAgent> { + ) -> Arrangement, R, Map, Set> + where + R: 'static, + { match self { - Self::Map(arr) => ArrangedCollection::Map(arr.enter_region(region)), - Self::Set(arr) => ArrangedCollection::Set(arr.enter_region(region)), + Self::Map(arr) => Arrangement::Map(arr.enter_region(region)), + Self::Set(arr) => Arrangement::Set(arr.enter_region(region)), } } } -impl<'a, S> ArrangedCollection, TValAgent, TKeyAgent> +impl<'a, S, R, Map, Set> Arrangement, R, Map, Set> where S: Scope, S::Timestamp: Lattice + Ord, + Map: TraceReader + Clone + 'static, + Map::Batch: BatchReader + Clone + 'static, + Map::Cursor: Cursor, + Set: TraceReader + Clone + 'static, + Set::Batch: BatchReader + Clone + 'static, + Set::Cursor: Cursor, { - pub fn leave_region(&self) -> ArrangedCollection, TKeyAgent> { + pub fn leave_region(&self) -> Arrangement { match self { - Self::Map(arr) => ArrangedCollection::Map(arr.leave_region()), - Self::Set(arr) => ArrangedCollection::Set(arr.leave_region()), + Self::Map(arr) => Arrangement::Map(arr.leave_region()), + Self::Set(arr) => Arrangement::Set(arr.leave_region()), } } } @@ -93,13 +106,16 @@ where 'a: 'b, { Arrangement1( - &'b ArrangedCollection< + &'b Arrangement< Child<'a, P, T>, + Weight, TValAgent>, TKeyAgent>, >, ), - Arrangement2(&'b ArrangedCollection, TValEnter<'a, P, T>, TKeyEnter<'a, P, T>>), + Arrangement2( + &'b Arrangement, Weight, TValEnter<'a, P, T>, TKeyEnter<'a, P, T>>, + ), } pub(super) struct Arrangements<'a, 'b, P, T> @@ -111,11 +127,16 @@ where { pub(super) arrangements1: &'b FnvHashMap< ArrId, - ArrangedCollection, TValAgent>, TKeyAgent>>, + Arrangement< + Child<'a, P, T>, + Weight, + TValAgent>, + TKeyAgent>, + >, >, pub(super) arrangements2: &'b FnvHashMap< ArrId, - ArrangedCollection, TValEnter<'a, P, T>, TKeyEnter<'a, P, T>>, + Arrangement, Weight, TValEnter<'a, P, T>, TKeyEnter<'a, P, T>>, >, } @@ -199,7 +220,7 @@ where G: Scope, G::Timestamp: Lattice, D: ExchangeData + Hashable, - R: Monoid + ExchangeData + One + Neg + Add, + R: Monoid + ExchangeData + Neg + Add + From, { collection .concat( @@ -209,8 +230,8 @@ where .arrange_by_self() .reduce(|_, src, dst| { // If the input weight is 1, don't produce a surplus record. - if !src[0].1.is_one() { - dst.push(((), ::one() + src[0].1.clone().neg())) + if src[0].1 != R::from(1) { + dst.push(((), R::from(1) + src[0].1.clone().neg())) } }) .map(|x| x.0), diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index ee7b1e35a..318d11de3 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -21,8 +21,13 @@ pub use arrange::diff_distinct; pub use timestamp::{TSNested, TupleTS, TS}; pub use update::Update; -use crate::{ddval::*, profile::*, record::Mutator}; -use arrange::{antijoin_arranged, ArrangedCollection, Arrangements, A}; +use crate::{ + ddval::*, + profile::*, + record::Mutator, + render::arrange_by::{ArrangeBy, ArrangementKind}, +}; +use arrange::{antijoin_arranged, Arrangement as DataflowArrangement, Arrangements, A}; use crossbeam_channel::{Receiver, Sender}; use fnv::{FnvHashMap, FnvHashSet}; use std::{ @@ -42,7 +47,6 @@ use std::{ use timestamp::ToTupleTS; use worker::{DDlogWorker, ProfilingData}; -use differential_dataflow::difference::Semigroup; use differential_dataflow::lattice::Lattice; use differential_dataflow::operators::arrange::arrangement::Arranged; use differential_dataflow::operators::arrange::*; @@ -768,57 +772,66 @@ impl Arrangement { fn build_arrangement_root( &self, collection: &Collection, - ) -> ArrangedCollection, TKeyAgent> + ) -> DataflowArrangement, TKeyAgent> where S: Scope, Collection: ThresholdTotal, S::Timestamp: Lattice + Ord + TotalOrder, { - match *self { - Arrangement::Map { afun, .. } => { - ArrangedCollection::Map(collection.flat_map(afun).arrange()) - } + let kind = match *self { + Arrangement::Map { afun, .. } => ArrangementKind::Map { + value_function: afun, + }, Arrangement::Set { fmfun, distinct, .. } => { - let filtered = collection.flat_map(fmfun); - if distinct { - ArrangedCollection::Set( - filtered - .threshold_total(|_, c| if c.is_zero() { 0 } else { 1 }) - .map(|k| (k, ())) - .arrange(), /* arrange_by_self() */ - ) - } else { - ArrangedCollection::Set(filtered.map(|k| (k, ())).arrange()) + // TODO: We don't currently produce a `None` as the key extraction + // function, but doing so will simplify the dataflow graph + // in instances where a function isn't needed + ArrangementKind::Set { + key_function: Some(fmfun), + distinct, } } + }; + + ArrangeBy { + kind, + target_relation: self.name().into(), } + .render_root(collection) } fn build_arrangement( &self, collection: &Collection, - ) -> ArrangedCollection, TKeyAgent> + ) -> DataflowArrangement, TKeyAgent> where S: Scope, S::Timestamp: Lattice + Ord, { - match *self { - Arrangement::Map { afun, .. } => { - ArrangedCollection::Map(collection.flat_map(afun).arrange()) - } + let kind = match *self { + Arrangement::Map { afun, .. } => ArrangementKind::Map { + value_function: afun, + }, Arrangement::Set { fmfun, distinct, .. } => { - let filtered = collection.flat_map(fmfun); - if distinct { - ArrangedCollection::Set(diff_distinct(&filtered).map(|k| (k, ())).arrange()) - } else { - ArrangedCollection::Set(filtered.map(|k| (k, ())).arrange()) + // TODO: We don't currently produce a `None` as the key extraction + // function, but doing so will simplify the dataflow graph + // in instances where a function isn't needed + ArrangementKind::Set { + key_function: Some(fmfun), + distinct, } } + }; + + ArrangeBy { + kind, + target_relation: self.name().into(), } + .render(collection) } } @@ -1369,8 +1382,8 @@ impl Program { // arrange input collection let collection_with_keys = col.flat_map(afun); let arr = match arrangements.lookup_arr(arrangement) { - A::Arrangement1(ArrangedCollection::Map(arranged)) => arranged.clone(), - A::Arrangement1(ArrangedCollection::Set(_)) => { + A::Arrangement1(DataflowArrangement::Map(arranged)) => arranged.clone(), + A::Arrangement1(DataflowArrangement::Set(_)) => { panic!("StreamJoin: not a map arrangement {:?}", arrangement) } _ => panic!("StreamJoin in nested scope: {}", description), @@ -1408,8 +1421,8 @@ impl Program { // arrange input collection let collection_with_keys = col.flat_map(afun); let arr = match arrangements.lookup_arr(arrangement) { - A::Arrangement1(ArrangedCollection::Set(arranged)) => arranged.clone(), - A::Arrangement1(ArrangedCollection::Map(_)) => { + A::Arrangement1(DataflowArrangement::Set(arranged)) => arranged.clone(), + A::Arrangement1(DataflowArrangement::Map(_)) => { panic!("StreamSemijoin: not a set arrangement {:?}", arrangement) } _ => panic!("StreamSemijoin in nested scope: {}", description), @@ -1588,7 +1601,7 @@ impl Program { jfun, ref next, } => match arrangements.lookup_arr(arrangement) { - A::Arrangement1(ArrangedCollection::Map(arranged)) => { + A::Arrangement1(DataflowArrangement::Map(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( || arr.join_core(arranged, jfun), @@ -1603,7 +1616,7 @@ impl Program { is_top_level_scope, ) } - A::Arrangement2(ArrangedCollection::Map(arranged)) => { + A::Arrangement2(DataflowArrangement::Map(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( || arr.join_core(arranged, jfun), @@ -1628,7 +1641,7 @@ impl Program { jfun, ref next, } => match arrangements.lookup_arr(arrangement) { - A::Arrangement1(ArrangedCollection::Set(arranged)) => { + A::Arrangement1(DataflowArrangement::Set(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( || arr.join_core(arranged, jfun), @@ -1643,7 +1656,7 @@ impl Program { is_top_level_scope, ) } - A::Arrangement2(ArrangedCollection::Set(arranged)) => { + A::Arrangement2(DataflowArrangement::Set(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( || arr.join_core(arranged, jfun), @@ -1666,7 +1679,7 @@ impl Program { arrangement, ref next, } => match arrangements.lookup_arr(arrangement) { - A::Arrangement1(ArrangedCollection::Set(arranged)) => { + A::Arrangement1(DataflowArrangement::Set(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( || antijoin_arranged(&arr, arranged).map(|(_, v)| v), @@ -1684,7 +1697,7 @@ impl Program { is_top_level_scope, ) } - A::Arrangement2(ArrangedCollection::Set(arranged)) => { + A::Arrangement2(DataflowArrangement::Set(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( || antijoin_arranged(&arr, arranged).map(|(_, v)| v), @@ -1866,14 +1879,14 @@ impl Program { is_top_level_scope, ), Rule::ArrangementRule { arr, xform, .. } => match arrangements.lookup_arr(*arr) { - A::Arrangement1(ArrangedCollection::Map(arranged)) => Self::xform_arrangement( + A::Arrangement1(DataflowArrangement::Map(arranged)) => Self::xform_arrangement( arranged, xform, &arrangements, &lookup_collection, is_top_level_scope, ), - A::Arrangement2(ArrangedCollection::Map(arranged)) => Self::xform_arrangement( + A::Arrangement2(DataflowArrangement::Map(arranged)) => Self::xform_arrangement( arranged, xform, &arrangements, diff --git a/rust/template/differential_datalog/src/program/worker.rs b/rust/template/differential_datalog/src/program/worker.rs index 08d813128..3a6c1a8fd 100644 --- a/rust/template/differential_datalog/src/program/worker.rs +++ b/rust/template/differential_datalog/src/program/worker.rs @@ -2,7 +2,7 @@ use crate::{ ddval::DDValue, profile::{get_prof_context, with_prof_context, ProfMsg}, program::{ - arrange::{ArrangedCollection, Arrangements}, + arrange::{Arrangement, Arrangements}, ArrId, Dep, Msg, ProgNode, Program, Reply, Update, TS, }, program::{RecursiveRelation, RelId, Relation, TKeyAgent, TValAgent, Weight}, @@ -569,7 +569,7 @@ impl<'a> DDlogWorker<'a> { // arrangement trace. let mut traces: BTreeMap = BTreeMap::new(); for ((relid, arrid), arr) in arrangements.into_iter() { - if let ArrangedCollection::Map(arranged) = arr { + if let Arrangement::Map(arranged) = arr { if program.get_relation(relid).arrangements[arrid].queryable() { arranged .as_collection(|k, _| k.clone()) @@ -601,8 +601,9 @@ fn render_relation<'a>( >, arrangements: &mut FnvHashMap< ArrId, - ArrangedCollection< + Arrangement< Child<'a, Worker, TS>, + Weight, TValAgent, TS>>, TKeyAgent, TS>>, >, @@ -650,7 +651,7 @@ fn render_relation<'a>( ) }); - if rule_collections.len() > 1 { + if rule_collections.len() > 0 { collection = with_prof_context(&format!("concatenate rules for {}", relation.name), || { collection.concatenate(rule_collections) @@ -695,8 +696,9 @@ fn render_scc<'a>( >, arrangements: &mut FnvHashMap< ArrId, - ArrangedCollection< + Arrangement< Child<'a, Worker, TS>, + Weight, TValAgent, TS>>, TKeyAgent, TS>>, >, diff --git a/rust/template/differential_datalog/src/render/arrange_by.rs b/rust/template/differential_datalog/src/render/arrange_by.rs new file mode 100644 index 000000000..2f6169881 --- /dev/null +++ b/rust/template/differential_datalog/src/render/arrange_by.rs @@ -0,0 +1,250 @@ +use crate::{ + dataflow::{diff_distinct, FilterMap, MapExt}, + ddval::DDValue, + program::arrange::Arrangement, + render::{Offset, Str, TraceKey, TraceValue}, +}; +use differential_dataflow::{ + difference::Abelian, + lattice::Lattice, + operators::{ + arrange::{Arrange, TraceAgent}, + ThresholdTotal, + }, + trace::implementations::ord::OrdKeySpine, + Collection, ExchangeData, +}; +use std::ops::Add; +use timely::{dataflow::Scope, order::TotalOrder}; + +// TODO: Allow dynamic functions +pub type KeyFunc = fn(DDValue) -> Option; +pub type ValueFunc = fn(DDValue) -> Option<(DDValue, DDValue)>; + +#[derive(Clone)] +pub struct ArrangeBy<'a> { + pub kind: ArrangementKind, + pub target_relation: Str<'a>, + // TODO: Add source file location +} + +#[derive(Clone)] +pub enum ArrangementKind { + Set { + /// The function for extracting the relation's key value, + /// can be `None` if the value is already the key + key_function: Option, + /// Whether or not the set should be distinct + distinct: bool, + }, + Map { + value_function: ValueFunc, + }, +} + +impl ArrangementKind { + pub const fn is_set(&self) -> bool { + matches!(self, Self::Set { .. }) + } +} + +type Arranged = + Arrangement>, TraceAgent>>; + +impl<'a> ArrangeBy<'a> { + pub fn render(&self, collection: &Collection) -> Arranged + where + S: Scope, + S::Timestamp: Lattice, + R: Abelian + ExchangeData + Add + From, + { + // Name for the arrangement operation + let arrangement_name = format!( + "Arrange{}: {}", + if self.kind.is_set() { "Set" } else { "Map" }, + self.target_relation, + ); + + match self.kind { + ArrangementKind::Set { + key_function, + distinct, + } => { + // Arranges a collection by its key + let arrange_set = |keyed: &Collection| { + let arranged = + keyed.arrange_named::>(&arrangement_name); + + Arrangement::Set(arranged) + }; + + // The keyed relation + let keyed = match self.key_collection( + collection, + key_function, + distinct, + &arrangement_name, + ) { + Ok(keyed) => keyed, + Err(arranged) => return arranged, + }; + + if distinct { + // Our `diff_distinct()` impl returns an arrangement which is + // ready to be used as an arranged set + Arrangement::Set(diff_distinct(&*self.target_relation, &keyed)) + } else { + // The name mapping the collection from `key` to `(key, ())` so we can arrange it + let mapped_name = format!("Map: Map to key for {}", self.target_relation); + arrange_set(&keyed.map_named(&mapped_name, |key| (key, ()))) + } + } + + ArrangementKind::Map { value_function } => { + self.render_map(&collection, value_function, &arrangement_name) + } + } + } + + pub fn render_root( + &self, + collection: &Collection, + ) -> Arrangement>, TraceAgent>> + where + S: Scope, + // TODO: The `TotalOrder` bound really puts a damper on + // code reuse, refactor things some more + S::Timestamp: Lattice + TotalOrder, + R: Abelian + ExchangeData + Add + From, + { + // Name for the arrangement operation + let arrangement_name = format!( + "Arrange{}: {}", + if self.kind.is_set() { "Set" } else { "Map" }, + self.target_relation, + ); + + match self.kind { + ArrangementKind::Set { + key_function, + distinct, + } => { + // Arranges a collection by its key + let arrange_set = |keyed: &Collection| { + let arranged = + keyed.arrange_named::>(&arrangement_name); + + Arrangement::Set(arranged) + }; + + // The keyed relation + let mut keyed = match self.key_collection( + collection, + key_function, + distinct, + &arrangement_name, + ) { + Ok(keyed) => keyed, + Err(arranged) => return arranged, + }; + + if distinct { + // Note: `Collection::threshold_total()` makes an arrangement, this just makes it explicit + // and allows us to name it. + // TODO: Insert this arrangement into the collection of shared arrangements so + // that we can try to re-use it if the opportunity arises + // TODO: Note that this `.arrange_named()` doesn't specify the `O` offset type. + // this is because `.threshold_total()` only accepts traces that use `usize` + // as the offset, whereas we'd prefer to use `Offset` + let threshold_arranged = keyed + .arrange_named::>(&format!( + "ArrangeBySelf: ThresholdTotal for {}", + self.target_relation, + )); + + // FIXME: If `diff` is negative, this will promote it to `1` + // TODO: `.threshold_total()` doesn't allow renaming the operator, + // make a custom op or PR differential-dataflow + keyed = threshold_arranged.threshold_total(|_, diff| { + if diff.is_zero() { + R::from(0) + } else { + R::from(1) + } + }); + } + + // The name mapping the collection from `key` to `(key, ())` so we can arrange it + let mapped_name = format!("Map: Map to key for {}", self.target_relation); + arrange_set(&keyed.map_named(&mapped_name, |key| (key, ()))) + } + + ArrangementKind::Map { value_function } => { + self.render_map(&collection, value_function, &arrangement_name) + } + } + } + + fn render_map( + &self, + collection: &Collection, + value_function: ValueFunc, + arrangement_name: &str, + ) -> Arranged + where + S: Scope, + S::Timestamp: Lattice, + R: Abelian + ExchangeData + Add + From, + { + // Extract the relation's key and value tuple before arranging it + let arranged = collection + .filter_map_named( + &format!( + "FilterMap: Extract key and value for {}", + self.target_relation, + ), + value_function, + ) + .arrange_named(arrangement_name); + + Arrangement::Map(arranged) + } + + fn key_collection( + &self, + collection: &Collection, + key_function: Option, + distinct: bool, + arrangement_name: &str, + ) -> Result, Arranged> + where + S: Scope, + S::Timestamp: Lattice, + R: Abelian + ExchangeData + Add + From, + { + if let Some(key_function) = key_function { + // The name for extracting the set's key out of the relation + let keyed_name = format!("FilterMap: Extract key for {}", self.target_relation); + + if distinct { + Ok(collection.filter_map_named(&keyed_name, key_function)) + + // If our set is filtered and is not distinct we can skip a redundant map + // operation by mapping into a `(key, ())` within the filter itself + } else { + let keyed = collection.filter_map_named(&keyed_name, move |value| { + key_function(value).map(|key| (key, ())) + }); + + let arranged = + keyed.arrange_named::>(&arrangement_name); + + Err(Arrangement::Set(arranged)) + } + + // If the collection has no key function then the current value is the key + } else { + Ok(collection.clone()) + } + } +} diff --git a/rust/template/differential_datalog/src/render/mod.rs b/rust/template/differential_datalog/src/render/mod.rs new file mode 100644 index 000000000..9fb4bdec7 --- /dev/null +++ b/rust/template/differential_datalog/src/render/mod.rs @@ -0,0 +1,15 @@ +use crate::ddval::DDValue; +use differential_dataflow::trace::implementations::ord::{OrdKeySpine, OrdValSpine}; +use std::borrow::Cow; +use timely::dataflow::ScopeParent; + +pub mod arrange_by; + +pub type Str<'a> = Cow<'a, str>; + +pub type Offset = u32; + +pub type TraceValue = + OrdValSpine::Timestamp, R, O>; + +pub type TraceKey = OrdKeySpine::Timestamp, R, O>; diff --git a/rust/template/differential_datalog/src/replay.rs b/rust/template/differential_datalog/src/replay.rs index 26f785643..572fe92e7 100644 --- a/rust/template/differential_datalog/src/replay.rs +++ b/rust/template/differential_datalog/src/replay.rs @@ -388,10 +388,12 @@ where #[cfg(test)] mod tests { use super::*; + #[cfg(feature = "c_api")] + use std::ffi::CStr; - struct DummyIntentory; + struct DummyInventory; - impl DDlogInventory for DummyIntentory { + impl DDlogInventory for DummyInventory { fn get_table_id(&self, _tname: &str) -> Result { unimplemented!() } @@ -407,6 +409,16 @@ mod tests { fn get_index_name(&self, _iid: IdxId) -> Result<&'static str, String> { unimplemented!() } + + #[cfg(feature = "c_api")] + fn get_table_cname(&self, _tid: RelId) -> Result<&'static CStr, String> { + unimplemented!() + } + + #[cfg(feature = "c_api")] + fn get_index_cname(&self, _iid: IdxId) -> Result<&'static CStr, String> { + unimplemented!() + } } /// Test recording of "updates" using `record_updates`. @@ -418,7 +430,7 @@ mod tests { let recorder = CommandRecorder::new( &mut buf, - Box::new(DummyIntentory) as Box, + Box::new(DummyInventory) as Box, ); recorder .do_record_updates(iter, |_, w, r| write!(w, "update {}", r)) diff --git a/src/Language/DifferentialDatalog/Compile.hs b/src/Language/DifferentialDatalog/Compile.hs index c394a6cb5..f421c8e08 100644 --- a/src/Language/DifferentialDatalog/Compile.hs +++ b/src/Language/DifferentialDatalog/Compile.hs @@ -217,6 +217,14 @@ rustLibFiles = , ("differential_datalog/src/test_record.rs" , $(embedFile "rust/template/differential_datalog/src/test_record.rs")) , ("differential_datalog/src/valmap.rs" , $(embedFile "rust/template/differential_datalog/src/valmap.rs")) , ("differential_datalog/src/variable.rs" , $(embedFile "rust/template/differential_datalog/src/variable.rs")) + , ("differential_datalog/src/render/mod.rs" , $(embedFile "rust/template/differential_datalog/src/render/mod.rs")) + , ("differential_datalog/src/render/arrange_by.rs" , $(embedFile "rust/template/differential_datalog/src/render/arrange_by.rs")) + , ("differential_datalog/src/dataflow/mod.rs" , $(embedFile "rust/template/differential_datalog/src/dataflow/mod.rs")) + , ("differential_datalog/src/dataflow/arrange.rs" , $(embedFile "rust/template/differential_datalog/src/dataflow/arrange.rs")) + , ("differential_datalog/src/dataflow/consolidate.rs" , $(embedFile "rust/template/differential_datalog/src/dataflow/consolidate.rs")) + , ("differential_datalog/src/dataflow/distinct.rs" , $(embedFile "rust/template/differential_datalog/src/dataflow/distinct.rs")) + , ("differential_datalog/src/dataflow/filter_map.rs" , $(embedFile "rust/template/differential_datalog/src/dataflow/filter_map.rs")) + , ("differential_datalog/src/dataflow/map.rs" , $(embedFile "rust/template/differential_datalog/src/dataflow/map.rs")) , ("differential_datalog_test/Cargo.toml" , $(embedFile "rust/template/differential_datalog_test/Cargo.toml")) , ("differential_datalog_test/lib.rs" , $(embedFile "rust/template/differential_datalog_test/lib.rs")) , ("differential_datalog_test/test_value.rs" , $(embedFile "rust/template/differential_datalog_test/test_value.rs")) From fea9d60e5b28884d15b2d364c8548b790f85f6a8 Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Tue, 16 Mar 2021 13:03:41 -0500 Subject: [PATCH 05/17] Updated to ddlog-3 branches for timely-dataflow, differential-dataflow and dogsdogsdogs --- rust/template/Cargo.toml | 2 +- rust/template/differential_datalog/Cargo.toml | 2 +- rust/template/differential_datalog/src/program/mod.rs | 6 ++++-- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/rust/template/Cargo.toml b/rust/template/Cargo.toml index bbece1461..be23652f3 100644 --- a/rust/template/Cargo.toml +++ b/rust/template/Cargo.toml @@ -36,7 +36,7 @@ cpuprofiler = { version = "0.0", optional = true } #differential-dataflow = "0.11.0" differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-3" } #timely = "0.11" -timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-3", default-features = false } +timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-3", default-features = false } fnv = "1.0.2" once_cell = "1.4.1" libc = "0.2" diff --git a/rust/template/differential_datalog/Cargo.toml b/rust/template/differential_datalog/Cargo.toml index 8b512265c..60d350a31 100644 --- a/rust/template/differential_datalog/Cargo.toml +++ b/rust/template/differential_datalog/Cargo.toml @@ -15,7 +15,7 @@ c_api = [] differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-3" } dogsdogsdogs = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-3" } #timely = "0.11" -timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-3", default-features = false} +timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-3", default-features = false } abomonation = "0.7" ordered-float = { version = "2.0.0", features = ["serde"] } diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index 318d11de3..2114f3847 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -61,12 +61,14 @@ use dogsdogsdogs::{ calculus::{Differentiate, Integrate}, operators::lookup_map, }; -use timely::communication::{initialize::WorkerGuards, Allocator}; use timely::dataflow::scopes::*; -use timely::execute::Config as TimelyConfig; use timely::order::TotalOrder; use timely::progress::{timestamp::Refines, PathSummary, Timestamp}; use timely::worker::Worker; +use timely::{ + communication::{initialize::WorkerGuards, Allocator}, + execute::Config as TimelyConfig, +}; type ValTrace = DefaultValTrace::Timestamp, Weight, u32>; type KeyTrace = DefaultKeyTrace::Timestamp, Weight, u32>; From 1095896ee83ae6afa9d4d1fa9a6d0faab42039ec Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Tue, 16 Mar 2021 14:23:03 -0500 Subject: [PATCH 06/17] Respect DIFFERENTIAL_EAGER_MERGE variable --- .../differential_datalog/src/program/mod.rs | 27 +++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index 2114f3847..db6f1ed1c 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -35,6 +35,7 @@ use std::{ borrow::Cow, cmp, collections::{hash_map, BTreeSet}, + env, fmt::{self, Debug, Formatter}, iter::{self, Cycle, Skip}, ops::Range, @@ -55,7 +56,7 @@ use differential_dataflow::trace::implementations::ord::OrdKeySpine as DefaultKe use differential_dataflow::trace::implementations::ord::OrdValSpine as DefaultValTrace; use differential_dataflow::trace::wrappers::enter::TraceEnter; use differential_dataflow::trace::{BatchReader, Cursor, TraceReader}; -use differential_dataflow::Collection; +use differential_dataflow::{Collection, Config as DDFlowConfig}; use dogsdogsdogs::{ altneu::AltNeu, calculus::{Differentiate, Integrate}, @@ -1019,9 +1020,31 @@ impl Program { let program = Arc::new(self.clone()); let profiling = ProfilingData::new(profile_cpu.clone(), profile_timely.clone(), prof_send); + let mut config = Config::process(number_workers); + + // Allow configuring the merge behavior of ddflow + // FIXME: Expose the merge behavior to all apis and deprecate the env var + if let Ok(value) = env::var("DIFFERENTIAL_EAGER_MERGE") { + let idle_merge_effort = if value.is_empty() { + None + } else { + let merge_effort: isize = value.parse().map_err(|_| { + "the `DIFFERENTIAL_EAGER_MERGE` variable must be set to an integer value" + .to_owned() + })?; + + Some(merge_effort) + }; + + differential_dataflow::configure( + &mut config.worker, + &DDFlowConfig { idle_merge_effort }, + ); + } + // Start up timely computation. let worker_guards = timely::execute( - TimelyConfig::process(number_workers), + config, move |worker: &mut Worker| -> Result<_, String> { let worker = DDlogWorker::new( worker, From 21e2ca12ddf45f0841482932ebaa08e2f9980eb3 Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Tue, 16 Mar 2021 14:33:19 -0500 Subject: [PATCH 07/17] Fixed infinitely recursing generic instantiation --- CHANGELOG.md | 6 + rust/template/Cargo.toml | 28 +- .../src/program/arrange.rs | 84 ++-- .../differential_datalog/src/program/mod.rs | 382 ++++++++++-------- .../src/program/worker.rs | 39 +- test/datalog_tests/rust_api_test/Cargo.toml | 8 +- test/datalog_tests/rust_api_test/src/main.rs | 41 +- 7 files changed, 310 insertions(+), 278 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 115609db4..95a0b0cbf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ # Changelog + All notable changes to this project will be documented in this file. The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/). @@ -8,6 +9,11 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/). ### Changes - Upgraded to timely dataflow and differential dataflow dependencies to v0.12. +- Worked on improving the debuggability of ddlog dataflow graphs + +### Bug Fixes + +- Fixed crashes due to misused unsafe code ## [0.39.0] - April 11, 2021 diff --git a/rust/template/Cargo.toml b/rust/template/Cargo.toml index be23652f3..8e1ffb0f2 100644 --- a/rust/template/Cargo.toml +++ b/rust/template/Cargo.toml @@ -14,20 +14,6 @@ command-line = ["cmd_parser", "rustop"] nested_ts_32 = ["differential_datalog/nested_ts_32"] c_api = ["differential_datalog/c_api"] -[target.'cfg(not(windows))'.build-dependencies] -libtool = "0.1" - -[dependencies.differential_datalog] -path = "./differential_datalog" - -[dependencies.cmd_parser] -path = "./cmd_parser" -optional = true - -[dependencies.ddlog_ovsdb_adapter] -path = "./ovsdb" -optional = true - [dependencies] abomonation = "0.7" time = { version = "0.2", features = ["serde"] } @@ -53,6 +39,20 @@ enum-primitive-derive = "0.2.1" # libraries: flatbuffers "0.6" <-> FlatBuffers "1.11.0". flatbuffers = { version = "0.6", optional = true } +[dependencies.differential_datalog] +path = "./differential_datalog" + +[dependencies.cmd_parser] +path = "./cmd_parser" +optional = true + +[dependencies.ddlog_ovsdb_adapter] +path = "./ovsdb" +optional = true + +[target.'cfg(not(windows))'.build-dependencies] +libtool = "0.1" + [[bin]] name = "datalog_example_cli" path = "src/main.rs" diff --git a/rust/template/differential_datalog/src/program/arrange.rs b/rust/template/differential_datalog/src/program/arrange.rs index ebb290981..f1caa3865 100644 --- a/rust/template/differential_datalog/src/program/arrange.rs +++ b/rust/template/differential_datalog/src/program/arrange.rs @@ -18,10 +18,22 @@ use differential_dataflow::{ use fnv::FnvHashMap; use std::ops::{Add, Mul, Neg}; use timely::{ - dataflow::scopes::{Child, Scope, ScopeParent}, + dataflow::scopes::{Child, Scope}, progress::{timestamp::Refines, Timestamp}, }; +#[derive(Clone)] +pub enum ArrangementFlavor +where + S: Scope, + S::Timestamp: Lattice + Refines, + T: Lattice + Timestamp, +{ + Local(Arrangement, TKeyAgent>), + Foreign(Arrangement, TKeyEnter>), +} + +#[derive(Clone)] pub enum Arrangement where S: Scope, @@ -95,68 +107,26 @@ where } } -/// Helper type that represents an arranged collection of one of two -/// types (e.g., an arrangement created in a local scope or entered from -/// the parent scope) -pub(super) enum A<'a, 'b, P, T> -where - P: ScopeParent, - P::Timestamp: Lattice + Ord, - T: Refines + Lattice + Timestamp + Ord, - 'a: 'b, -{ - Arrangement1( - &'b Arrangement< - Child<'a, P, T>, - Weight, - TValAgent>, - TKeyAgent>, - >, - ), - Arrangement2( - &'b Arrangement, Weight, TValEnter<'a, P, T>, TKeyEnter<'a, P, T>>, - ), -} - -pub(super) struct Arrangements<'a, 'b, P, T> +pub(super) struct Arrangements<'a, S, T> where - P: ScopeParent, - P::Timestamp: Lattice + Ord, - T: Refines + Lattice + Timestamp + Ord, - 'a: 'b, + S: Scope, + S::Timestamp: Lattice + Refines, + T: Lattice + Timestamp, { - pub(super) arrangements1: &'b FnvHashMap< - ArrId, - Arrangement< - Child<'a, P, T>, - Weight, - TValAgent>, - TKeyAgent>, - >, - >, - pub(super) arrangements2: &'b FnvHashMap< - ArrId, - Arrangement, Weight, TValEnter<'a, P, T>, TKeyEnter<'a, P, T>>, - >, + pub(super) arrangements: &'a FnvHashMap>, } -impl<'a, 'b, P, T> Arrangements<'a, 'b, P, T> +impl<'a, S, T> Arrangements<'a, S, T> where - P: ScopeParent, - P::Timestamp: Lattice + Ord, - T: Refines + Lattice + Timestamp + Ord, - 'a: 'b, + S: Scope, + S::Timestamp: Lattice + Refines, + T: Lattice + Timestamp, { - pub(super) fn lookup_arr(&self, arrid: ArrId) -> A<'a, 'b, P, T> { - self.arrangements1.get(&arrid).map_or_else( - || { - self.arrangements2 - .get(&arrid) - .map(|arr| A::Arrangement2(arr)) - .unwrap_or_else(|| panic!("mk_rule: unknown arrangement {:?}", arrid)) - }, - |arr| A::Arrangement1(arr), - ) + pub(super) fn lookup_arr(&self, arrid: ArrId) -> ArrangementFlavor { + self.arrangements + .get(&arrid) + .cloned() + .unwrap_or_else(|| panic!("mk_rule: unknown arrangement {:?}", arrid)) } } diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index db6f1ed1c..2f98e9efe 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -27,7 +27,9 @@ use crate::{ record::Mutator, render::arrange_by::{ArrangeBy, ArrangementKind}, }; -use arrange::{antijoin_arranged, Arrangement as DataflowArrangement, Arrangements, A}; +use arrange::{ + antijoin_arranged, Arrangement as DataflowArrangement, ArrangementFlavor, Arrangements, +}; use crossbeam_channel::{Receiver, Sender}; use fnv::{FnvHashMap, FnvHashSet}; use std::{ @@ -71,14 +73,14 @@ use timely::{ execute::Config as TimelyConfig, }; -type ValTrace = DefaultValTrace::Timestamp, Weight, u32>; -type KeyTrace = DefaultKeyTrace::Timestamp, Weight, u32>; +type ValTrace = DefaultValTrace; +type KeyTrace = DefaultKeyTrace; type TValAgent = TraceAgent>; type TKeyAgent = TraceAgent>; -type TValEnter<'a, P, T> = TraceEnter, T>; -type TKeyEnter<'a, P, T> = TraceEnter, T>; +type TValEnter = TraceEnter, T>; +type TKeyEnter = TraceEnter, T>; /// Diff associated with records in differential dataflow pub type Weight = i32; @@ -775,7 +777,7 @@ impl Arrangement { fn build_arrangement_root( &self, collection: &Collection, - ) -> DataflowArrangement, TKeyAgent> + ) -> DataflowArrangement, TKeyAgent> where S: Scope, Collection: ThresholdTotal, @@ -808,7 +810,7 @@ impl Arrangement { fn build_arrangement( &self, collection: &Collection, - ) -> DataflowArrangement, TKeyAgent> + ) -> DataflowArrangement, TKeyAgent> where S: Scope, S::Timestamp: Lattice + Ord, @@ -1020,7 +1022,7 @@ impl Program { let program = Arc::new(self.clone()); let profiling = ProfilingData::new(profile_cpu.clone(), profile_timely.clone(), prof_send); - let mut config = Config::process(number_workers); + let mut config = TimelyConfig::process(number_workers); // Allow configuring the merge behavior of ddflow // FIXME: Expose the merge behavior to all apis and deprecate the env var @@ -1248,19 +1250,21 @@ impl Program { /// TODO: Allow this to return an error, so we can replace `expect`'s below /// with proper error handling. - fn xform_collection<'a, 'b, P, T, LC>( - col: Collection, DDValue, Weight>, + // TODO: Much of this logic would be vastly simplified if we used a + // combination of traits and `Vec`s (as opposed to + // what we do now with a linked list of them) + fn xform_collection<'a, S, T, Lookup>( + col: Collection, xform: &Option, - arrangements: &Arrangements<'a, 'b, P, T>, - lookup_collection: &LC, + arrangements: &Arrangements<'a, S, T>, + lookup_collection: Lookup, is_top_level_scope: bool, - ) -> Collection, DDValue, Weight> + ) -> Collection where - P: ScopeParent, - P::Timestamp: Lattice, - T: Refines + Lattice + Timestamp + Ord, - T: ToTupleTS, - LC: Fn(RelId) -> Option, DDValue, Weight>>, + S: Scope, + S::Timestamp: Lattice + Refines + ToTupleTS, + T: Lattice + Timestamp, + Lookup: Fn(RelId) -> Option>, { match xform { None => col, @@ -1274,19 +1278,118 @@ impl Program { } } - fn xform_collection_ref<'a, 'b, P, T, LC>( - col: &Collection, DDValue, Weight>, + fn xform_collection_ref<'a, S, T, Lookup>( + col: &Collection, + xform: &XFormCollection, + arrangements: &Arrangements<'a, S, T>, + lookup_collection: Lookup, + is_top_level_scope: bool, + ) -> Collection + where + S: Scope, + S::Timestamp: Lattice + Refines + ToTupleTS, + T: Lattice + Timestamp, + Lookup: Fn(RelId) -> Option>, + { + if let XFormCollection::StreamXForm { + description, + xform, + next, + } = xform + { + let xformed = col + .scope() + .scoped::, _, _>(description, |inner| { + let d_col = with_prof_context( + format!("differentiate stream before: {}", description).as_ref(), + || col.differentiate(inner), + ); + + fn dummy_lookup_collection( + _: RelId, + ) -> Option> { + None + } + + // We must call the streamless variant within the nested scope + // otherwise we force rustc to instantiate an infinitely long type + // since the function calls itself (a potentially infinite number of times), + // each requiring further nesting of the scopes (and their types) + let xformed = Self::streamless_xform_collection::< + Child>, + S::Timestamp, + _, + >( + d_col, + &*xform, + &Arrangements { + arrangements: &FnvHashMap::default(), + }, + dummy_lookup_collection, + false, + ); + + with_prof_context( + format!("integrate stream after: {}", description).as_ref(), + || xformed.integrate(), + ) + }); + + Self::xform_collection( + xformed.clone(), + &*next, + arrangements, + lookup_collection, + is_top_level_scope, + ) + } else { + Self::streamless_xform_collection_ref( + col, + xform, + arrangements, + lookup_collection, + is_top_level_scope, + ) + } + } + + fn streamless_xform_collection<'a, S, T, Lookup>( + col: Collection, + xform: &Option, + arrangements: &Arrangements<'a, S, T>, + lookup_collection: Lookup, + is_top_level_scope: bool, + ) -> Collection + where + S: Scope, + S::Timestamp: Lattice + Refines + ToTupleTS, + T: Lattice + Timestamp, + Lookup: Fn(RelId) -> Option>, + { + match xform { + None => col, + Some(ref x) => Self::streamless_xform_collection_ref( + &col, + x, + arrangements, + lookup_collection, + is_top_level_scope, + ), + } + } + + fn streamless_xform_collection_ref<'a, S, T, Lookup>( + col: &Collection, xform: &XFormCollection, - arrangements: &Arrangements<'a, 'b, P, T>, - lookup_collection: &LC, + arrangements: &Arrangements<'a, S, T>, + lookup_collection: Lookup, is_top_level_scope: bool, - ) -> Collection, DDValue, Weight> + ) -> Collection where - P: ScopeParent, - P::Timestamp: Lattice, - T: Refines + Lattice + Timestamp + Ord, - T: ToTupleTS, - LC: Fn(RelId) -> Option, DDValue, Weight>>, + S: Scope, + S::Timestamp: Lattice + Refines + ToTupleTS, + T: Lattice + Timestamp, + Lookup: Fn(RelId) -> Option>, { match *xform { XFormCollection::Arrange { @@ -1308,13 +1411,15 @@ impl Program { ref next, } => { #[allow(clippy::unnecessary_cast)] - let one = Any::downcast_ref::(&(1 as TS)) + let one = Any::downcast_ref::<::Summary>(&(1 as TS)) .expect("Differentiate operator used in recursive context"); + let diff = with_prof_context(&description, || { col.concat( &col.delay(move |t| one.results_in(t).expect("Integer overflow in Differentiate: maximal number of transactions exceeded")).negate()) }); - Self::xform_collection( + + Self::streamless_xform_collection( diff, &*next, arrangements, @@ -1328,7 +1433,7 @@ impl Program { ref next, } => { let mapped = with_prof_context(&description, || col.map(mfun)); - Self::xform_collection( + Self::streamless_xform_collection( mapped, &*next, arrangements, @@ -1344,7 +1449,7 @@ impl Program { let flattened = with_prof_context(&description, || { col.flat_map(move |x| fmfun(x).into_iter().flatten()) }); - Self::xform_collection( + Self::streamless_xform_collection( flattened, &*next, arrangements, @@ -1358,7 +1463,7 @@ impl Program { ref next, } => { let filtered = with_prof_context(&description, || col.filter(ffun)); - Self::xform_collection( + Self::streamless_xform_collection( filtered, &*next, arrangements, @@ -1372,7 +1477,7 @@ impl Program { ref next, } => { let flattened = with_prof_context(&description, || col.flat_map(fmfun)); - Self::xform_collection( + Self::streamless_xform_collection( flattened, &*next, arrangements, @@ -1388,7 +1493,7 @@ impl Program { let inspect = with_prof_context(&description, || { col.inspect(move |(v, ts, w)| ifun(v, ts.to_tuple_ts(), *w)) }); - Self::xform_collection( + Self::streamless_xform_collection( inspect, &*next, arrangements, @@ -1407,8 +1512,10 @@ impl Program { // arrange input collection let collection_with_keys = col.flat_map(afun); let arr = match arrangements.lookup_arr(arrangement) { - A::Arrangement1(DataflowArrangement::Map(arranged)) => arranged.clone(), - A::Arrangement1(DataflowArrangement::Set(_)) => { + ArrangementFlavor::Local(DataflowArrangement::Map(arranged)) => { + arranged.clone() + } + ArrangementFlavor::Local(DataflowArrangement::Set(_)) => { panic!("StreamJoin: not a map arrangement {:?}", arrangement) } _ => panic!("StreamJoin in nested scope: {}", description), @@ -1427,7 +1534,7 @@ impl Program { // to return `Option`. .flat_map(|v| v) }); - Self::xform_collection( + Self::streamless_xform_collection( join, &*next, arrangements, @@ -1446,8 +1553,10 @@ impl Program { // arrange input collection let collection_with_keys = col.flat_map(afun); let arr = match arrangements.lookup_arr(arrangement) { - A::Arrangement1(DataflowArrangement::Set(arranged)) => arranged.clone(), - A::Arrangement1(DataflowArrangement::Map(_)) => { + ArrangementFlavor::Local(DataflowArrangement::Set(arranged)) => { + arranged.clone() + } + ArrangementFlavor::Local(DataflowArrangement::Map(_)) => { panic!("StreamSemijoin: not a set arrangement {:?}", arrangement) } _ => panic!("StreamSemijoin in nested scope: {}", description), @@ -1466,7 +1575,7 @@ impl Program { // to return `Option`. .flat_map(|v| v) }); - Self::xform_collection( + Self::streamless_xform_collection( join, &*next, arrangements, @@ -1474,85 +1583,32 @@ impl Program { is_top_level_scope, ) } + XFormCollection::StreamXForm { ref description, .. - } if !is_top_level_scope => { - panic!("StreamXForm in nested scope: {}", description); - } - XFormCollection::StreamXForm { - ref description, - ref xform, - ref next, } => { - // `StreamXForm` can in principle be applied in any scope, but it causes `rustc` - // to go into an infinite recursion instantiating infinitely nested `AltNeu` - // scopes and eventually crashing. In practice, we only need to apply - // `StreamXForm` in the top-level scope (and possibly its children in the - // future). So we brutally transmute the scope to the top-level scope. - let col = unsafe { - &*(col as *const Collection, DDValue, Weight> - as *const Collection, TS>, DDValue, Weight>) - }; - let xformed = col - .scope() - .scoped::, _, _>(description, |inner| { - let d_col = with_prof_context( - format!("differentiate stream before: {}", description).as_ref(), - || col.differentiate(inner), - ); - let arrangements1 = FnvHashMap::default(); - let arrangements2 = FnvHashMap::default(); - fn dummy_lookup_collection( - _: RelId, - ) -> Option> { - None - } - let xformed = Self::xform_collection( - d_col, - &*xform, - &Arrangements { - arrangements1: &arrangements1, - arrangements2: &arrangements2, - }, - &dummy_lookup_collection, - false, - ); - with_prof_context( - format!("integrate stream after: {}", description).as_ref(), - || xformed.integrate(), - ) - }); - let xformed = unsafe { - &*(&xformed as *const Collection, TS>, DDValue, Weight> - as *const Collection, DDValue, Weight>) - }; - Self::xform_collection( - xformed.clone(), - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + panic!("StreamXForm in nested scope: {}", description); } } } - fn xform_arrangement<'a, 'b, P, T, TR, LC>( - arr: &Arranged, TR>, + fn xform_arrangement<'a, S, T, TR, LC>( + arr: &Arranged, xform: &XFormArrangement, - arrangements: &Arrangements<'a, 'b, P, T>, - lookup_collection: &LC, + arrangements: &Arrangements<'a, S, T>, + lookup_collection: LC, is_top_level_scope: bool, - ) -> Collection, DDValue, Weight> + ) -> Collection where - P: ScopeParent, - P::Timestamp: Lattice, - T: Refines + Lattice + Timestamp + Ord, - T: ToTupleTS, - TR: TraceReader + Clone + 'static, - TR::Batch: BatchReader, - TR::Cursor: Cursor, - LC: Fn(RelId) -> Option, DDValue, Weight>>, + S: Scope, + S::Timestamp: Lattice + Refines + ToTupleTS, + T: Lattice + Timestamp, + TR: TraceReader + + Clone + + 'static, + TR::Batch: BatchReader, + TR::Cursor: Cursor, + LC: Fn(RelId) -> Option>, { match *xform { XFormArrangement::FlatMap { @@ -1560,7 +1616,7 @@ impl Program { fmfun, ref next, } => with_prof_context(&description, || { - Self::xform_collection( + Self::streamless_xform_collection( arr.flat_map_ref(move |_, v| match fmfun(v.clone()) { Some(iter) => iter, None => Box::new(None.into_iter()), @@ -1576,7 +1632,7 @@ impl Program { fmfun, ref next, } => with_prof_context(&description, || { - Self::xform_collection( + Self::streamless_xform_collection( arr.flat_map_ref(move |_, v| fmfun(v.clone())), &*next, arrangements, @@ -1611,7 +1667,7 @@ impl Program { }, ) }); - Self::xform_collection( + Self::streamless_xform_collection( col, &*next, arrangements, @@ -1626,14 +1682,14 @@ impl Program { jfun, ref next, } => match arrangements.lookup_arr(arrangement) { - A::Arrangement1(DataflowArrangement::Map(arranged)) => { + ArrangementFlavor::Local(DataflowArrangement::Map(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( - || arr.join_core(arranged, jfun), - |f| arr.filter(move |_, v| f(v)).join_core(arranged, jfun), + || arr.join_core(&arranged, jfun), + |f| arr.filter(move |_, v| f(v)).join_core(&arranged, jfun), ) }); - Self::xform_collection( + Self::streamless_xform_collection( col, &*next, arrangements, @@ -1641,14 +1697,14 @@ impl Program { is_top_level_scope, ) } - A::Arrangement2(DataflowArrangement::Map(arranged)) => { + ArrangementFlavor::Foreign(DataflowArrangement::Map(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( - || arr.join_core(arranged, jfun), - |f| arr.filter(move |_, v| f(v)).join_core(arranged, jfun), + || arr.join_core(&arranged, jfun), + |f| arr.filter(move |_, v| f(v)).join_core(&arranged, jfun), ) }); - Self::xform_collection( + Self::streamless_xform_collection( col, &*next, arrangements, @@ -1666,14 +1722,14 @@ impl Program { jfun, ref next, } => match arrangements.lookup_arr(arrangement) { - A::Arrangement1(DataflowArrangement::Set(arranged)) => { + ArrangementFlavor::Local(DataflowArrangement::Set(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( - || arr.join_core(arranged, jfun), - |f| arr.filter(move |_, v| f(v)).join_core(arranged, jfun), + || arr.join_core(&arranged, jfun), + |f| arr.filter(move |_, v| f(v)).join_core(&arranged, jfun), ) }); - Self::xform_collection( + Self::streamless_xform_collection( col, &*next, arrangements, @@ -1681,14 +1737,14 @@ impl Program { is_top_level_scope, ) } - A::Arrangement2(DataflowArrangement::Set(arranged)) => { + ArrangementFlavor::Foreign(DataflowArrangement::Set(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( - || arr.join_core(arranged, jfun), - |f| arr.filter(move |_, v| f(v)).join_core(arranged, jfun), + || arr.join_core(&arranged, jfun), + |f| arr.filter(move |_, v| f(v)).join_core(&arranged, jfun), ) }); - Self::xform_collection( + Self::streamless_xform_collection( col, &*next, arrangements, @@ -1704,17 +1760,17 @@ impl Program { arrangement, ref next, } => match arrangements.lookup_arr(arrangement) { - A::Arrangement1(DataflowArrangement::Set(arranged)) => { + ArrangementFlavor::Local(DataflowArrangement::Set(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( - || antijoin_arranged(&arr, arranged).map(|(_, v)| v), + || antijoin_arranged(&arr, &arranged).map(|(_, v)| v), |f| { - antijoin_arranged(&arr.filter(move |_, v| f(v)), arranged) + antijoin_arranged(&arr.filter(move |_, v| f(v)), &arranged) .map(|(_, v)| v) }, ) }); - Self::xform_collection( + Self::streamless_xform_collection( col, &*next, arrangements, @@ -1722,17 +1778,17 @@ impl Program { is_top_level_scope, ) } - A::Arrangement2(DataflowArrangement::Set(arranged)) => { + ArrangementFlavor::Foreign(DataflowArrangement::Set(arranged)) => { let col = with_prof_context(&description, || { ffun.map_or_else( - || antijoin_arranged(&arr, arranged).map(|(_, v)| v), + || antijoin_arranged(&arr, &arranged).map(|(_, v)| v), |f| { - antijoin_arranged(&arr.filter(move |_, v| f(v)), arranged) + antijoin_arranged(&arr.filter(move |_, v| f(v)), &arranged) .map(|(_, v)| v) }, ) }); - Self::xform_collection( + Self::streamless_xform_collection( col, &*next, arrangements, @@ -1792,7 +1848,7 @@ impl Program { // to return `Option`. join.flat_map(|v| v) }); - Self::xform_collection( + Self::streamless_xform_collection( col, &*next, arrangements, @@ -1849,7 +1905,7 @@ impl Program { // to return `Option`. join.flat_map(|v| v) }); - Self::xform_collection( + Self::streamless_xform_collection( col, &*next, arrangements, @@ -1863,20 +1919,18 @@ impl Program { /// Compile right-hand-side of a rule to a collection /// /// * `is_top_level_scope` - `true` when evaluating a rule in the top-level scope. - fn mk_rule<'a, 'b, P, T, F>( + fn mk_rule<'a, S, T, F>( &self, rule: &Rule, lookup_collection: F, - arrangements: Arrangements<'a, 'b, P, T>, + arrangements: Arrangements<'a, S, T>, is_top_level_scope: bool, - ) -> Collection, DDValue, Weight> + ) -> Collection where - P: ScopeParent + 'a, - P::Timestamp: Lattice, - T: Refines + Lattice + Timestamp + Ord, - T: ToTupleTS, - F: Fn(RelId) -> Option, DDValue, Weight>>, - 'a: 'b, + S: Scope, + S::Timestamp: Lattice + Refines + ToTupleTS, + T: Lattice + Timestamp, + F: Fn(RelId) -> Option>, { match rule { Rule::CollectionRule { @@ -1904,20 +1958,24 @@ impl Program { is_top_level_scope, ), Rule::ArrangementRule { arr, xform, .. } => match arrangements.lookup_arr(*arr) { - A::Arrangement1(DataflowArrangement::Map(arranged)) => Self::xform_arrangement( - arranged, - xform, - &arrangements, - &lookup_collection, - is_top_level_scope, - ), - A::Arrangement2(DataflowArrangement::Map(arranged)) => Self::xform_arrangement( - arranged, - xform, - &arrangements, - &lookup_collection, - is_top_level_scope, - ), + ArrangementFlavor::Local(DataflowArrangement::Map(arranged)) => { + Self::xform_arrangement( + &arranged, + xform, + &arrangements, + &lookup_collection, + is_top_level_scope, + ) + } + ArrangementFlavor::Foreign(DataflowArrangement::Map(arranged)) => { + Self::xform_arrangement( + &arranged, + xform, + &arrangements, + &lookup_collection, + is_top_level_scope, + ) + } _ => panic!("Rule starts with a set arrangement {:?}", *arr), }, } diff --git a/rust/template/differential_datalog/src/program/worker.rs b/rust/template/differential_datalog/src/program/worker.rs index 3a6c1a8fd..7cf5636fa 100644 --- a/rust/template/differential_datalog/src/program/worker.rs +++ b/rust/template/differential_datalog/src/program/worker.rs @@ -1,11 +1,14 @@ use crate::{ ddval::DDValue, profile::{get_prof_context, with_prof_context, ProfMsg}, + program::{ + arrange::ArrangementFlavor, RecursiveRelation, RelId, Relation, TKeyAgent, TValAgent, + Weight, + }, program::{ arrange::{Arrangement, Arrangements}, ArrId, Dep, Msg, ProgNode, Program, Reply, Update, TS, }, - program::{RecursiveRelation, RelId, Relation, TKeyAgent, TValAgent, Weight}, variable::Variable, }; use crossbeam_channel::{Receiver, Sender}; @@ -601,12 +604,7 @@ fn render_relation<'a>( >, arrangements: &mut FnvHashMap< ArrId, - Arrangement< - Child<'a, Worker, TS>, - Weight, - TValAgent, TS>>, - TKeyAgent, TS>>, - >, + Arrangement, TS>, Weight, TValAgent, TKeyAgent>, >, delayed_vars: &DelayedVarMap, TS>>, ) { @@ -622,9 +620,9 @@ fn render_relation<'a>( collection.enter_region(region) }; - let entered_arrangements: FnvHashMap<_, _> = arrangements + let entered_arrangements: FnvHashMap<_, ArrangementFlavor<_, TS>> = arrangements .iter() - .map(|(&arr_id, arr)| (arr_id, arr.enter_region(region))) + .map(|(&arr_id, arr)| (arr_id, ArrangementFlavor::Local(arr.enter_region(region)))) .collect(); // apply rules @@ -644,8 +642,7 @@ fn render_relation<'a>( rule, get_rule_collection, Arrangements { - arrangements1: &entered_arrangements, - arrangements2: &FnvHashMap::default(), + arrangements: &entered_arrangements, }, true, ) @@ -696,12 +693,7 @@ fn render_scc<'a>( >, arrangements: &mut FnvHashMap< ArrId, - Arrangement< - Child<'a, Worker, TS>, - Weight, - TValAgent, TS>>, - TKeyAgent, TS>>, - >, + Arrangement, TS>, Weight, TValAgent, TKeyAgent>, >, ) -> Result<(), String> { // Preallocate the memory required to store the new relations @@ -793,6 +785,16 @@ fn render_scc<'a>( } } + let arrangements = local_arrangements + .into_iter() + .map(|(id, arr)| (id, ArrangementFlavor::Local(arr))) + .chain( + inner_arrangements + .into_iter() + .map(|(id, arr)| (id, ArrangementFlavor::Foreign(arr))), + ) + .collect(); + // apply rules to variables for rel in rels { for rule in &rel.rel.rules { @@ -805,8 +807,7 @@ fn render_scc<'a>( .cloned() }, Arrangements { - arrangements1: &local_arrangements, - arrangements2: &inner_arrangements, + arrangements: &arrangements, }, false, ); diff --git a/test/datalog_tests/rust_api_test/Cargo.toml b/test/datalog_tests/rust_api_test/Cargo.toml index 857e7447c..585b2c951 100644 --- a/test/datalog_tests/rust_api_test/Cargo.toml +++ b/test/datalog_tests/rust_api_test/Cargo.toml @@ -7,7 +7,7 @@ edition = "2018" # The generated Rust project contains several crates that must be imported # by the client program. -differential_datalog = {path = "../tutorial_ddlog/differential_datalog"} -tutorial = {path = "../tutorial_ddlog"} -ddlog_rt = {path = "../tutorial_ddlog/types/ddlog_rt"} -types = {path = "../tutorial_ddlog/types"} +differential_datalog = { path = "../tutorial_ddlog/differential_datalog" } +tutorial = { path = "../tutorial_ddlog" } +ddlog_rt = { path = "../tutorial_ddlog/types/ddlog_rt" } +types = { path = "../tutorial_ddlog/types" } diff --git a/test/datalog_tests/rust_api_test/src/main.rs b/test/datalog_tests/rust_api_test/src/main.rs index 8bc475ad6..421cc0927 100644 --- a/test/datalog_tests/rust_api_test/src/main.rs +++ b/test/datalog_tests/rust_api_test/src/main.rs @@ -16,16 +16,16 @@ use tutorial_ddlog::relid2name; // Type and function definitions. use tutorial_ddlog::typedefs::*; - // The differential_datalog crate contains the DDlog runtime that is +// The differential_datalog crate contains the DDlog runtime that is // the same for all DDlog programs and simply gets copied to each generated // DDlog workspace unmodified (this will change in future releases). -use differential_datalog::{DDlog, DDlogInventory, DDlogDynamic}; // Trait that must be implemented by an instance of a DDlog program. -use differential_datalog::DeltaMap; // Type that represents a set of changes to DDlog relations. - // Returned by `DDlog::transaction_commit_dump_changes()`. -use differential_datalog::ddval::DDValue; // Generic type that wraps all DDlog value. -use differential_datalog::ddval::DDValConvert; // Trait to convert Rust types to/from DDValue. - // All types used in input and output relations, indexes, and - // primary keys implement this trait. +use differential_datalog::DeltaMap; +use differential_datalog::{DDlog, DDlogDynamic, DDlogInventory}; // Trait that must be implemented by an instance of a DDlog program. // Type that represents a set of changes to DDlog relations. + // Returned by `DDlog::transaction_commit_dump_changes()`. +use differential_datalog::ddval::DDValConvert; +use differential_datalog::ddval::DDValue; // Generic type that wraps all DDlog value. // Trait to convert Rust types to/from DDValue. + // All types used in input and output relations, indexes, and + // primary keys implement this trait. use differential_datalog::program::RelId; // Numeric relations id. use differential_datalog::program::Update; // Type-safe representation of a DDlog command (insert/delete_val/delete_key/...) @@ -34,9 +34,7 @@ use differential_datalog::record::Record; // Dynamically typed representation of use differential_datalog::record::RelIdentifier; // Relation identifier: either `RelId` or `Cow`g. use differential_datalog::record::UpdCmd; // Dynamically typed representation of DDlog command. - fn main() -> Result<(), String> { - // Instantiate a DDlog program. // Returns a handle to the program and initial contents of output relations. // Arguments @@ -45,7 +43,7 @@ fn main() -> Result<(), String> { // of output relations. Should only be used for debugging in order to dump // the contents of output tables using `HDDlog::dump_table()`. Otherwise, // indexes are the preferred way to achieve this. - let (mut hddlog, init_state) = HDDlog::run(1, false)?; + let (hddlog, init_state) = HDDlog::run(1, false)?; println!("Initial state"); dump_delta(&init_state); @@ -73,7 +71,8 @@ fn main() -> Result<(), String> { // multiple updates. An update inserts, deletes or modifies a record in a DDlog // relation. let updates = vec![ - Update::Insert { // We are going to insert.. + Update::Insert { + // We are going to insert.. relid: Relations::Word1 as RelId, // .. into relation with this Id. // `Word1` type, declared in the `types` crate has the same fields as // the corresponding DDlog type. @@ -111,12 +110,9 @@ fn main() -> Result<(), String> { // weight = 1 - insert. // weight = -1 - delete. assert_eq!(*weight, 1); - // `val` has type `DDValue`; converting it to a concrete Rust - // type is an unsafe operation: specifying the wrong Rust type - // will lead to undefined behavior. - let phrase: &Phrases = unsafe { Phrases::from_ddvalue_ref(val) }; + let phrase: &Phrases = Phrases::from_ddvalue_ref(val); println!("New phrase: {}", phrase.phrase); - }; + } hddlog.transaction_start()?; @@ -129,11 +125,12 @@ fn main() -> Result<(), String> { // DDlog values. let commands = vec![UpdCmd::Insert( RelIdentifier::RelId(relid_word1), - Record::PosStruct( // Positional struct consists of constructor name - // and a vector of arguments whose number and - // types must match those of the DDlog constructor. - // The alternative is `NamedStruct` where arguments - // are represented as (name, value) pairs. + Record::PosStruct( + // Positional struct consists of constructor name + // and a vector of arguments whose number and + // types must match those of the DDlog constructor. + // The alternative is `NamedStruct` where arguments + // are represented as (name, value) pairs. Cow::from("Word1"), // Constructor name. // Constructor arguments. vec![ From 2cc516f1c9e7123f16df3ca75499eb47d7f1c7c6 Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Sat, 10 Apr 2021 13:21:43 -0500 Subject: [PATCH 08/17] Fixed clippy lints --- Cargo.toml | 2 +- rust/template/differential_datalog/src/lib.rs | 2 +- .../differential_datalog/src/program/mod.rs | 10 ++---- .../src/program/update.rs | 31 ++++--------------- 4 files changed, 11 insertions(+), 34 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 3a4ae884b..90a871434 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -4,7 +4,7 @@ members = [ "rust/template/ovsdb", "rust/template/cmd_parser", "rust/template/ddlog_derive", - "rust/template/distributed_datalog", + #"rust/template/distributed_datalog", "rust/template/differential_datalog", "rust/template/differential_datalog_test", ] diff --git a/rust/template/differential_datalog/src/lib.rs b/rust/template/differential_datalog/src/lib.rs index 4de13329e..916bcacf8 100644 --- a/rust/template/differential_datalog/src/lib.rs +++ b/rust/template/differential_datalog/src/lib.rs @@ -2,7 +2,7 @@ clippy::get_unwrap, clippy::missing_safety_doc, clippy::type_complexity, - clippy::match_like_matches_macro + clippy::upper_case_acronyms )] mod callback; diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index 2f98e9efe..3cb703651 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -1336,7 +1336,7 @@ impl Program { }); Self::xform_collection( - xformed.clone(), + xformed, &*next, arrangements, lookup_collection, @@ -1512,9 +1512,7 @@ impl Program { // arrange input collection let collection_with_keys = col.flat_map(afun); let arr = match arrangements.lookup_arr(arrangement) { - ArrangementFlavor::Local(DataflowArrangement::Map(arranged)) => { - arranged.clone() - } + ArrangementFlavor::Local(DataflowArrangement::Map(arranged)) => arranged, ArrangementFlavor::Local(DataflowArrangement::Set(_)) => { panic!("StreamJoin: not a map arrangement {:?}", arrangement) } @@ -1553,9 +1551,7 @@ impl Program { // arrange input collection let collection_with_keys = col.flat_map(afun); let arr = match arrangements.lookup_arr(arrangement) { - ArrangementFlavor::Local(DataflowArrangement::Set(arranged)) => { - arranged.clone() - } + ArrangementFlavor::Local(DataflowArrangement::Set(arranged)) => arranged, ArrangementFlavor::Local(DataflowArrangement::Map(_)) => { panic!("StreamSemijoin: not a set arrangement {:?}", arrangement) } diff --git a/rust/template/differential_datalog/src/program/update.rs b/rust/template/differential_datalog/src/program/update.rs index b2f241a0f..aa06f357c 100644 --- a/rust/template/differential_datalog/src/program/update.rs +++ b/rust/template/differential_datalog/src/program/update.rs @@ -52,53 +52,34 @@ impl Update { /// Returns `true` if the the current update is an Insert. pub fn is_insert(&self) -> bool { - match self { - Update::Insert { .. } => true, - _ => false, - } + matches!(self, Update::Insert { .. }) } /// Returns `true` if the the current update is an InsertOrUpdate. pub fn is_insert_or_update(&self) -> bool { - match self { - Update::InsertOrUpdate { .. } => true, - _ => false, - } + matches!(self, Update::InsertOrUpdate { .. }) } /// Returns `true` if the the current update is a DeleteValue. pub fn is_delete_value(&self) -> bool { - match self { - Update::DeleteValue { .. } => true, - _ => false, - } + matches!(self, Update::DeleteValue { .. }) } /// Returns `true` if the the current update is a DeleteKey. pub fn is_delete_key(&self) -> bool { - match self { - Update::DeleteKey { .. } => true, - _ => false, - } + matches!(self, Update::DeleteKey { .. }) } /// Returns `true` if the the current update is a Modify. pub fn is_modify(&self) -> bool { - match self { - Update::Modify { .. } => true, - _ => false, - } + matches!(self, Update::Modify { .. }) } /// Returns whether the current update has a key of some sort /// /// Returns `true` if the update is a `DeleteKey` or a `Modify` pub fn has_key(&self) -> bool { - match self { - Update::DeleteKey { .. } => true, - Update::Modify { .. } => true, - _ => false, - } + matches!(self, Update::DeleteKey { .. } | Update::Modify { .. }) } /// Attempts to get the key of the current update From adcf46dbeb7f58b91f6f4e3b93b71ae98b346628 Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Sat, 10 Apr 2021 13:43:10 -0500 Subject: [PATCH 09/17] Upgraded timely/ddflow to ddlog-4 --- rust/template/Cargo.toml | 4 ++-- rust/template/differential_datalog/Cargo.toml | 6 +++--- rust/template/differential_datalog_test/Cargo.toml | 4 ++-- src/Language/DifferentialDatalog/Compile.hs | 4 ++-- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/rust/template/Cargo.toml b/rust/template/Cargo.toml index 8e1ffb0f2..a63f58f7e 100644 --- a/rust/template/Cargo.toml +++ b/rust/template/Cargo.toml @@ -20,9 +20,9 @@ time = { version = "0.2", features = ["serde"] } ordered-float = { version = "2.0.0", features = ["serde"] } cpuprofiler = { version = "0.0", optional = true } #differential-dataflow = "0.11.0" -differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-3" } +differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-4" } #timely = "0.11" -timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-3", default-features = false } +timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-4", default-features = false } fnv = "1.0.2" once_cell = "1.4.1" libc = "0.2" diff --git a/rust/template/differential_datalog/Cargo.toml b/rust/template/differential_datalog/Cargo.toml index 60d350a31..77d8e11f6 100644 --- a/rust/template/differential_datalog/Cargo.toml +++ b/rust/template/differential_datalog/Cargo.toml @@ -12,10 +12,10 @@ c_api = [] [dependencies] #differential-dataflow = "0.11.0" -differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-3" } -dogsdogsdogs = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-3" } +differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-4" } +dogsdogsdogs = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-4" } #timely = "0.11" -timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-3", default-features = false } +timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-4", default-features = false } abomonation = "0.7" ordered-float = { version = "2.0.0", features = ["serde"] } diff --git a/rust/template/differential_datalog_test/Cargo.toml b/rust/template/differential_datalog_test/Cargo.toml index f8fe874b6..c6cbd1f8d 100644 --- a/rust/template/differential_datalog_test/Cargo.toml +++ b/rust/template/differential_datalog_test/Cargo.toml @@ -5,11 +5,11 @@ edition = "2018" [dependencies] # differential-dataflow = "0.11.0" -differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-3" } +differential-dataflow = { git = "https://github.com/ddlog-dev/differential-dataflow", branch = "ddlog-4" } abomonation = "0.7" fnv = "1.0.2" # timely = "0.11" -timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-3" } +timely = { git = "https://github.com/ddlog-dev/timely-dataflow", branch = "ddlog-4" } serde = { version = "1.0", features = ["derive"] } erased-serde = "0.3" differential_datalog = { path = "../differential_datalog" } diff --git a/src/Language/DifferentialDatalog/Compile.hs b/src/Language/DifferentialDatalog/Compile.hs index f421c8e08..b5ed191d2 100644 --- a/src/Language/DifferentialDatalog/Compile.hs +++ b/src/Language/DifferentialDatalog/Compile.hs @@ -783,8 +783,8 @@ mkCargoToml rs_code crate crate_id = "erased-serde = \"0.3\"" $$ --"differential-dataflow = \"0.11.0\"" $$ --"timely = \"0.11\"" $$ - "differential-dataflow = { git = \"https://github.com/ddlog-dev/differential-dataflow\", branch = \"ddlog-3\" }" $$ - "timely = { git = \"https://github.com/ddlog-dev/timely-dataflow\", branch = \"ddlog-3\", default-features = false }" $$ + "differential-dataflow = { git = \"https://github.com/ddlog-dev/differential-dataflow\", branch = \"ddlog-4\" }" $$ + "timely = { git = \"https://github.com/ddlog-dev/timely-dataflow\", branch = \"ddlog-4\", default-features = false }" $$ "" $$ dependencies $$ "" $$ From d7e7c2f60025af7314306267b40caf92fcf475bc Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Sat, 10 Apr 2021 13:52:33 -0500 Subject: [PATCH 10/17] Documented Arrangement and ArrangementFlavor --- .../differential_datalog/src/program/arrange.rs | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/rust/template/differential_datalog/src/program/arrange.rs b/rust/template/differential_datalog/src/program/arrange.rs index f1caa3865..d0e9894e1 100644 --- a/rust/template/differential_datalog/src/program/arrange.rs +++ b/rust/template/differential_datalog/src/program/arrange.rs @@ -22,6 +22,7 @@ use timely::{ progress::{timestamp::Refines, Timestamp}, }; +/// An arrangement originating either in the current scope or a higher one #[derive(Clone)] pub enum ArrangementFlavor where @@ -29,10 +30,15 @@ where S::Timestamp: Lattice + Refines, T: Lattice + Timestamp, { + /// An arrangement created within the current scope, usually created by + /// [`Arrangement::enter_region()`] or made directly from the output of + /// [`ArrangeBySelf::arrange_by_self()`] or similar methods Local(Arrangement, TKeyAgent>), + /// An arrangement imported from a higher scope, usually created by [`Arrangement::enter()`] Foreign(Arrangement, TKeyEnter>), } +/// A single [arrangement](Arranged), either arranged as a map of keys to values or as a set of keys #[derive(Clone)] pub enum Arrangement where @@ -45,7 +51,9 @@ where Set::Batch: BatchReader + Clone + 'static, Set::Cursor: Cursor, { + /// An [arrangement](Arranged) of keys to associated values Map(Arranged), + /// An [arrangement](Arranged) of keys Set(Arranged), } @@ -60,7 +68,8 @@ where Set::Batch: BatchReader + Clone + 'static, Set::Cursor: Cursor, { - pub(super) fn enter<'a, TInner>( + /// Brings an arranged collection out of a nested scope, see [`Arranged::enter()`] + pub fn enter<'a, TInner>( &self, inner: &Child<'a, S, TInner>, ) -> Arrangement, R, TraceEnter, TraceEnter> @@ -74,6 +83,7 @@ where } } + /// Brings an arranged collection into a nested region, see [`Arranged::enter_region()`] pub fn enter_region<'a>( &self, region: &Child<'a, S, S::Timestamp>, @@ -99,6 +109,7 @@ where Set::Batch: BatchReader + Clone + 'static, Set::Cursor: Cursor, { + /// Brings an arranged collection out of a nested region, see [`Arranged::leave_region()`] pub fn leave_region(&self) -> Arrangement { match self { Self::Map(arr) => Arrangement::Map(arr.leave_region()), From 470f12c6f10b1800451e5dfceb2f44dbc68d8f5a Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Sat, 10 Apr 2021 14:15:29 -0500 Subject: [PATCH 11/17] Removed is_top_level_scope --- Cargo.toml | 2 +- .../differential_datalog/src/program/mod.rs | 192 +++--------------- .../src/program/worker.rs | 8 +- 3 files changed, 28 insertions(+), 174 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 90a871434..3a4ae884b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -4,7 +4,7 @@ members = [ "rust/template/ovsdb", "rust/template/cmd_parser", "rust/template/ddlog_derive", - #"rust/template/distributed_datalog", + "rust/template/distributed_datalog", "rust/template/differential_datalog", "rust/template/differential_datalog_test", ] diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index 3cb703651..cbc6f470f 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -1258,7 +1258,6 @@ impl Program { xform: &Option, arrangements: &Arrangements<'a, S, T>, lookup_collection: Lookup, - is_top_level_scope: bool, ) -> Collection where S: Scope, @@ -1268,13 +1267,7 @@ impl Program { { match xform { None => col, - Some(ref x) => Self::xform_collection_ref( - &col, - x, - arrangements, - lookup_collection, - is_top_level_scope, - ), + Some(ref x) => Self::xform_collection_ref(&col, x, arrangements, lookup_collection), } } @@ -1283,7 +1276,6 @@ impl Program { xform: &XFormCollection, arrangements: &Arrangements<'a, S, T>, lookup_collection: Lookup, - is_top_level_scope: bool, ) -> Collection where S: Scope, @@ -1326,7 +1318,6 @@ impl Program { arrangements: &FnvHashMap::default(), }, dummy_lookup_collection, - false, ); with_prof_context( @@ -1335,21 +1326,9 @@ impl Program { ) }); - Self::xform_collection( - xformed, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::xform_collection(xformed, &*next, arrangements, lookup_collection) } else { - Self::streamless_xform_collection_ref( - col, - xform, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection_ref(col, xform, arrangements, lookup_collection) } } @@ -1358,7 +1337,6 @@ impl Program { xform: &Option, arrangements: &Arrangements<'a, S, T>, lookup_collection: Lookup, - is_top_level_scope: bool, ) -> Collection where S: Scope, @@ -1368,13 +1346,9 @@ impl Program { { match xform { None => col, - Some(ref x) => Self::streamless_xform_collection_ref( - &col, - x, - arrangements, - lookup_collection, - is_top_level_scope, - ), + Some(ref x) => { + Self::streamless_xform_collection_ref(&col, x, arrangements, lookup_collection) + } } } @@ -1383,7 +1357,6 @@ impl Program { xform: &XFormCollection, arrangements: &Arrangements<'a, S, T>, lookup_collection: Lookup, - is_top_level_scope: bool, ) -> Collection where S: Scope, @@ -1398,13 +1371,7 @@ impl Program { ref next, } => { let arr = with_prof_context(&description, || col.flat_map(afun).arrange_by_key()); - Self::xform_arrangement( - &arr, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::xform_arrangement(&arr, &*next, arrangements, lookup_collection) } XFormCollection::Differentiate { ref description, @@ -1419,13 +1386,7 @@ impl Program { &col.delay(move |t| one.results_in(t).expect("Integer overflow in Differentiate: maximal number of transactions exceeded")).negate()) }); - Self::streamless_xform_collection( - diff, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(diff, &*next, arrangements, lookup_collection) } XFormCollection::Map { ref description, @@ -1433,13 +1394,7 @@ impl Program { ref next, } => { let mapped = with_prof_context(&description, || col.map(mfun)); - Self::streamless_xform_collection( - mapped, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(mapped, &*next, arrangements, lookup_collection) } XFormCollection::FlatMap { ref description, @@ -1454,7 +1409,6 @@ impl Program { &*next, arrangements, lookup_collection, - is_top_level_scope, ) } XFormCollection::Filter { @@ -1463,13 +1417,7 @@ impl Program { ref next, } => { let filtered = with_prof_context(&description, || col.filter(ffun)); - Self::streamless_xform_collection( - filtered, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(filtered, &*next, arrangements, lookup_collection) } XFormCollection::FilterMap { ref description, @@ -1482,7 +1430,6 @@ impl Program { &*next, arrangements, lookup_collection, - is_top_level_scope, ) } XFormCollection::Inspect { @@ -1493,13 +1440,7 @@ impl Program { let inspect = with_prof_context(&description, || { col.inspect(move |(v, ts, w)| ifun(v, ts.to_tuple_ts(), *w)) }); - Self::streamless_xform_collection( - inspect, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(inspect, &*next, arrangements, lookup_collection) } XFormCollection::StreamJoin { ref description, @@ -1532,13 +1473,7 @@ impl Program { // to return `Option`. .flat_map(|v| v) }); - Self::streamless_xform_collection( - join, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(join, &*next, arrangements, lookup_collection) } XFormCollection::StreamSemijoin { ref description, @@ -1571,13 +1506,7 @@ impl Program { // to return `Option`. .flat_map(|v| v) }); - Self::streamless_xform_collection( - join, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(join, &*next, arrangements, lookup_collection) } XFormCollection::StreamXForm { @@ -1593,7 +1522,6 @@ impl Program { xform: &XFormArrangement, arrangements: &Arrangements<'a, S, T>, lookup_collection: LC, - is_top_level_scope: bool, ) -> Collection where S: Scope, @@ -1620,7 +1548,6 @@ impl Program { &*next, arrangements, lookup_collection, - is_top_level_scope, ) }), XFormArrangement::FilterMap { @@ -1633,7 +1560,6 @@ impl Program { &*next, arrangements, lookup_collection, - is_top_level_scope, ) }), XFormArrangement::Aggregate { @@ -1663,13 +1589,7 @@ impl Program { }, ) }); - Self::streamless_xform_collection( - col, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(col, &*next, arrangements, lookup_collection) } XFormArrangement::Join { ref description, @@ -1685,13 +1605,7 @@ impl Program { |f| arr.filter(move |_, v| f(v)).join_core(&arranged, jfun), ) }); - Self::streamless_xform_collection( - col, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(col, &*next, arrangements, lookup_collection) } ArrangementFlavor::Foreign(DataflowArrangement::Map(arranged)) => { let col = with_prof_context(&description, || { @@ -1700,13 +1614,7 @@ impl Program { |f| arr.filter(move |_, v| f(v)).join_core(&arranged, jfun), ) }); - Self::streamless_xform_collection( - col, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(col, &*next, arrangements, lookup_collection) } _ => panic!("Join: not a map arrangement {:?}", arrangement), @@ -1725,13 +1633,7 @@ impl Program { |f| arr.filter(move |_, v| f(v)).join_core(&arranged, jfun), ) }); - Self::streamless_xform_collection( - col, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(col, &*next, arrangements, lookup_collection) } ArrangementFlavor::Foreign(DataflowArrangement::Set(arranged)) => { let col = with_prof_context(&description, || { @@ -1740,13 +1642,7 @@ impl Program { |f| arr.filter(move |_, v| f(v)).join_core(&arranged, jfun), ) }); - Self::streamless_xform_collection( - col, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(col, &*next, arrangements, lookup_collection) } _ => panic!("Semijoin: not a set arrangement {:?}", arrangement), }, @@ -1766,13 +1662,7 @@ impl Program { }, ) }); - Self::streamless_xform_collection( - col, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(col, &*next, arrangements, lookup_collection) } ArrangementFlavor::Foreign(DataflowArrangement::Set(arranged)) => { let col = with_prof_context(&description, || { @@ -1784,13 +1674,7 @@ impl Program { }, ) }); - Self::streamless_xform_collection( - col, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(col, &*next, arrangements, lookup_collection) } _ => panic!("Antijoin: not a set arrangement {:?}", arrangement), }, @@ -1844,13 +1728,7 @@ impl Program { // to return `Option`. join.flat_map(|v| v) }); - Self::streamless_xform_collection( - col, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(col, &*next, arrangements, lookup_collection) } XFormArrangement::StreamSemijoin { ref description, @@ -1901,26 +1779,17 @@ impl Program { // to return `Option`. join.flat_map(|v| v) }); - Self::streamless_xform_collection( - col, - &*next, - arrangements, - lookup_collection, - is_top_level_scope, - ) + Self::streamless_xform_collection(col, &*next, arrangements, lookup_collection) } } } /// Compile right-hand-side of a rule to a collection - /// - /// * `is_top_level_scope` - `true` when evaluating a rule in the top-level scope. fn mk_rule<'a, S, T, F>( &self, rule: &Rule, lookup_collection: F, arrangements: Arrangements<'a, S, T>, - is_top_level_scope: bool, ) -> Collection where S: Scope, @@ -1951,26 +1820,13 @@ impl Program { x, &arrangements, &lookup_collection, - is_top_level_scope, ), Rule::ArrangementRule { arr, xform, .. } => match arrangements.lookup_arr(*arr) { ArrangementFlavor::Local(DataflowArrangement::Map(arranged)) => { - Self::xform_arrangement( - &arranged, - xform, - &arrangements, - &lookup_collection, - is_top_level_scope, - ) + Self::xform_arrangement(&arranged, xform, &arrangements, &lookup_collection) } ArrangementFlavor::Foreign(DataflowArrangement::Map(arranged)) => { - Self::xform_arrangement( - &arranged, - xform, - &arrangements, - &lookup_collection, - is_top_level_scope, - ) + Self::xform_arrangement(&arranged, xform, &arrangements, &lookup_collection) } _ => panic!("Rule starts with a set arrangement {:?}", *arr), }, diff --git a/rust/template/differential_datalog/src/program/worker.rs b/rust/template/differential_datalog/src/program/worker.rs index 7cf5636fa..e8099b9ce 100644 --- a/rust/template/differential_datalog/src/program/worker.rs +++ b/rust/template/differential_datalog/src/program/worker.rs @@ -628,12 +628,12 @@ fn render_relation<'a>( // apply rules // TODO: Regions for rules let rule_collections = relation.rules.iter().map(|rule| { - let get_rule_collection = |rule_id| { - if let Some(collection) = collections.get(&rule_id) { + let get_rule_collection = |relation_id| { + if let Some(collection) = collections.get(&relation_id) { Some(collection.enter_region(region)) } else { delayed_vars - .get(&rule_id) + .get(&relation_id) .map(|(_, _, collection)| collection.enter_region(region)) } }; @@ -644,7 +644,6 @@ fn render_relation<'a>( Arrangements { arrangements: &entered_arrangements, }, - true, ) }); @@ -809,7 +808,6 @@ fn render_scc<'a>( Arrangements { arrangements: &arrangements, }, - false, ); vars.get_mut(&rel.rel.id) From df7761c779342859eb6c398efd7dd797ecfe281a Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Sat, 10 Apr 2021 14:32:11 -0500 Subject: [PATCH 12/17] Duplication really sucks --- .../differential_datalog/src/program/mod.rs | 216 ++++++++++++++---- 1 file changed, 175 insertions(+), 41 deletions(-) diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index cbc6f470f..7f9da1089 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -1283,52 +1283,186 @@ impl Program { T: Lattice + Timestamp, Lookup: Fn(RelId) -> Option>, { - if let XFormCollection::StreamXForm { - description, - xform, - next, - } = xform - { - let xformed = col - .scope() - .scoped::, _, _>(description, |inner| { - let d_col = with_prof_context( - format!("differentiate stream before: {}", description).as_ref(), - || col.differentiate(inner), - ); - - fn dummy_lookup_collection( - _: RelId, - ) -> Option> { - None - } + match *xform { + XFormCollection::Arrange { + ref description, + afun, + ref next, + } => { + let arr = with_prof_context(&description, || col.flat_map(afun).arrange_by_key()); + Self::xform_arrangement(&arr, &*next, arrangements, lookup_collection) + } + XFormCollection::Differentiate { + ref description, + ref next, + } => { + #[allow(clippy::unnecessary_cast)] + let one = Any::downcast_ref::<::Summary>(&(1 as TS)) + .expect("Differentiate operator used in recursive context"); - // We must call the streamless variant within the nested scope - // otherwise we force rustc to instantiate an infinitely long type - // since the function calls itself (a potentially infinite number of times), - // each requiring further nesting of the scopes (and their types) - let xformed = Self::streamless_xform_collection::< - Child>, - S::Timestamp, - _, - >( - d_col, - &*xform, - &Arrangements { - arrangements: &FnvHashMap::default(), - }, - dummy_lookup_collection, - ); + let diff = with_prof_context(&description, || { + col.concat( + &col.delay(move |t| one.results_in(t).expect("Integer overflow in Differentiate: maximal number of transactions exceeded")).negate()) + }); - with_prof_context( - format!("integrate stream after: {}", description).as_ref(), - || xformed.integrate(), + Self::xform_collection(diff, &*next, arrangements, lookup_collection) + } + XFormCollection::Map { + ref description, + mfun, + ref next, + } => { + let mapped = with_prof_context(&description, || col.map(mfun)); + Self::xform_collection(mapped, &*next, arrangements, lookup_collection) + } + XFormCollection::FlatMap { + ref description, + fmfun, + ref next, + } => { + let flattened = with_prof_context(&description, || { + col.flat_map(move |x| fmfun(x).into_iter().flatten()) + }); + Self::xform_collection(flattened, &*next, arrangements, lookup_collection) + } + XFormCollection::Filter { + ref description, + ffun, + ref next, + } => { + let filtered = with_prof_context(&description, || col.filter(ffun)); + Self::xform_collection(filtered, &*next, arrangements, lookup_collection) + } + XFormCollection::FilterMap { + ref description, + fmfun, + ref next, + } => { + let flattened = with_prof_context(&description, || col.flat_map(fmfun)); + Self::xform_collection(flattened, &*next, arrangements, lookup_collection) + } + XFormCollection::Inspect { + ref description, + ifun, + ref next, + } => { + let inspect = with_prof_context(&description, || { + col.inspect(move |(v, ts, w)| ifun(v, ts.to_tuple_ts(), *w)) + }); + Self::xform_collection(inspect, &*next, arrangements, lookup_collection) + } + XFormCollection::StreamJoin { + ref description, + afun, + arrangement, + jfun, + ref next, + } => { + let join = with_prof_context(&description, || { + // arrange input collection + let collection_with_keys = col.flat_map(afun); + let arr = match arrangements.lookup_arr(arrangement) { + ArrangementFlavor::Local(DataflowArrangement::Map(arranged)) => arranged, + ArrangementFlavor::Local(DataflowArrangement::Set(_)) => { + panic!("StreamJoin: not a map arrangement {:?}", arrangement) + } + _ => panic!("StreamJoin in nested scope: {}", description), + }; + lookup_map( + &collection_with_keys, + arr, + |(k, _), key| *key = k.clone(), + move |v1, w1, v2, w2| (jfun(&v1.1, v2), w1 * w2), + ().into_ddvalue(), + ().into_ddvalue(), + ().into_ddvalue(), + ) + // Filter out `None`'s. + // FIXME: We wouldn't need this if `lookup_map` allowed `output_func` + // to return `Option`. + .flat_map(|v| v) + }); + Self::xform_collection(join, &*next, arrangements, lookup_collection) + } + XFormCollection::StreamSemijoin { + ref description, + afun, + arrangement, + jfun, + ref next, + } => { + let join = with_prof_context(&description, || { + // arrange input collection + let collection_with_keys = col.flat_map(afun); + let arr = match arrangements.lookup_arr(arrangement) { + ArrangementFlavor::Local(DataflowArrangement::Set(arranged)) => arranged, + ArrangementFlavor::Local(DataflowArrangement::Map(_)) => { + panic!("StreamSemijoin: not a set arrangement {:?}", arrangement) + } + _ => panic!("StreamSemijoin in nested scope: {}", description), + }; + lookup_map( + &collection_with_keys, + arr, + |(k, _), key| *key = k.clone(), + move |v1, w1, _, w2| (jfun(&v1.1), w1 * w2), + ().into_ddvalue(), + ().into_ddvalue(), + ().into_ddvalue(), ) + // Filter out `None`'s. + // FIXME: We wouldn't need this if `lookup_map` allowed `output_func` + // to return `Option`. + .flat_map(|v| v) }); + Self::xform_collection(join, &*next, arrangements, lookup_collection) + } - Self::xform_collection(xformed, &*next, arrangements, lookup_collection) - } else { - Self::streamless_xform_collection_ref(col, xform, arrangements, lookup_collection) + XFormCollection::StreamXForm { + ref description, + ref xform, + ref next, + } => { + let xformed = + col.scope() + .scoped::, _, _>(description, |inner| { + let d_col = with_prof_context( + format!("differentiate stream before: {}", description).as_ref(), + || col.differentiate(inner), + ); + + fn dummy_lookup_collection( + _: RelId, + ) -> Option> + { + None + } + + // We must call the streamless variant within the nested scope + // otherwise we force rustc to instantiate an infinitely long type + // since the function calls itself (a potentially infinite number of times), + // each requiring further nesting of the scopes (and their types) + let xformed = Self::streamless_xform_collection::< + Child>, + S::Timestamp, + _, + >( + d_col, + &*xform, + &Arrangements { + arrangements: &FnvHashMap::default(), + }, + dummy_lookup_collection, + ); + + with_prof_context( + format!("integrate stream after: {}", description).as_ref(), + || xformed.integrate(), + ) + }); + + Self::xform_collection(xformed, &*next, arrangements, lookup_collection) + } } } From e2a0641ce82a3011192dbdce1f766505c8312687 Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Sat, 10 Apr 2021 15:33:33 -0500 Subject: [PATCH 13/17] Fixed clippy lints --- rust/template/differential_datalog/src/lib.rs | 3 +- .../differential_datalog/src/record/mod.rs | 45 ++++++++++--------- 2 files changed, 25 insertions(+), 23 deletions(-) diff --git a/rust/template/differential_datalog/src/lib.rs b/rust/template/differential_datalog/src/lib.rs index 916bcacf8..3cc9b6b15 100644 --- a/rust/template/differential_datalog/src/lib.rs +++ b/rust/template/differential_datalog/src/lib.rs @@ -1,8 +1,7 @@ #![allow( clippy::get_unwrap, clippy::missing_safety_doc, - clippy::type_complexity, - clippy::upper_case_acronyms + clippy::type_complexity )] mod callback; diff --git a/rust/template/differential_datalog/src/record/mod.rs b/rust/template/differential_datalog/src/record/mod.rs index d89a837e4..93ddc0cd5 100644 --- a/rust/template/differential_datalog/src/record/mod.rs +++ b/rust/template/differential_datalog/src/record/mod.rs @@ -67,30 +67,42 @@ pub enum Record { } impl Record { - pub fn is_bool(&self) -> bool { + pub const fn is_bool(&self) -> bool { matches!(self, Self::Bool(_)) } - pub fn is_int(&self) -> bool { + pub const fn is_int(&self) -> bool { matches!(self, Self::Int(_)) } - pub fn is_float(&self) -> bool { + pub const fn is_float(&self) -> bool { matches!(self, Self::Float(_)) } - pub fn is_double(&self) -> bool { + pub const fn is_double(&self) -> bool { matches!(self, Self::Double(_)) } - pub fn is_struct(&self) -> bool { + pub const fn is_struct(&self) -> bool { matches!(self, Self::PosStruct(_, _) | Self::NamedStruct(_, _)) } - pub fn is_named_struct(&self) -> bool { + pub const fn is_named_struct(&self) -> bool { matches!(self, Self::NamedStruct(_, _)) } + pub const fn is_vector(&self) -> bool { + matches!(self, Self::Array(CollectionKind::Vector, _)) + } + + pub const fn is_map(&self) -> bool { + matches!(self, Self::Array(CollectionKind::Map, _)) + } + + pub const fn is_string(&self) -> bool { + matches!(self, Self::String(_)) + } + pub fn as_int(&self) -> Option<&BigInt> { match self { Self::Int(int) => Some(int), @@ -1076,11 +1088,8 @@ pub unsafe extern "C" fn ddlog_string_with_length( #[no_mangle] #[cfg(feature = "c_api")] -pub unsafe extern "C" fn ddlog_is_string(rec: *const Record) -> bool { - match rec.as_ref() { - Some(Record::String(_)) => true, - _ => false, - } +pub unsafe extern "C" fn ddlog_is_string(record: *const Record) -> bool { + record.as_ref().map(Record::is_string).unwrap_or_default() } #[cfg(feature = "c_api")] @@ -1237,11 +1246,8 @@ pub unsafe extern "C" fn ddlog_vector( #[cfg(feature = "c_api")] #[no_mangle] -pub unsafe extern "C" fn ddlog_is_vector(rec: *const Record) -> bool { - match rec.as_ref() { - Some(Record::Array(CollectionKind::Vector, _)) => true, - _ => false, - } +pub unsafe extern "C" fn ddlog_is_vector(record: *const Record) -> bool { + record.as_ref().map(Record::is_vector).unwrap_or_default() } #[cfg(feature = "c_api")] @@ -1336,11 +1342,8 @@ pub unsafe extern "C" fn ddlog_map(fields: *const *mut Record, len: libc::size_t #[cfg(feature = "c_api")] #[no_mangle] -pub unsafe extern "C" fn ddlog_is_map(rec: *const Record) -> bool { - match rec.as_ref() { - Some(Record::Array(CollectionKind::Map, _)) => true, - _ => false, - } +pub unsafe extern "C" fn ddlog_is_map(record: *const Record) -> bool { + record.as_ref().map(Record::is_map).unwrap_or_default() } #[cfg(feature = "c_api")] From f3759786997ee57b4e231cb01adad274b9415b44 Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Tue, 13 Apr 2021 16:48:15 -0500 Subject: [PATCH 14/17] Config options --- Cargo.toml | 2 +- .../src/program/config.rs | 174 ++++++++++ .../differential_datalog/src/program/mod.rs | 110 +++---- .../src/program/worker.rs | 298 ++++++++++-------- .../src/render/arrange_by.rs | 9 +- .../differential_datalog/src/render/mod.rs | 13 +- rust/template/src/api/mod.rs | 7 +- src/Language/DifferentialDatalog/Compile.hs | 1 + 8 files changed, 415 insertions(+), 199 deletions(-) create mode 100644 rust/template/differential_datalog/src/program/config.rs diff --git a/Cargo.toml b/Cargo.toml index 3a4ae884b..90a871434 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -4,7 +4,7 @@ members = [ "rust/template/ovsdb", "rust/template/cmd_parser", "rust/template/ddlog_derive", - "rust/template/distributed_datalog", + #"rust/template/distributed_datalog", "rust/template/differential_datalog", "rust/template/differential_datalog_test", ] diff --git a/rust/template/differential_datalog/src/program/config.rs b/rust/template/differential_datalog/src/program/config.rs new file mode 100644 index 000000000..6be13821c --- /dev/null +++ b/rust/template/differential_datalog/src/program/config.rs @@ -0,0 +1,174 @@ +//! Configuration for DDlog programs + +use crate::{ + profile::Profile, + program::{worker::ProfilingData, Program, PROF_MSG_BUF_SIZE}, +}; +use differential_dataflow::Config as DDFlowConfig; +use std::{ + env, + sync::{atomic::AtomicBool, Arc, Mutex}, + thread::{self, JoinHandle}, +}; +use timely::Config as TimelyConfig; + +/// The configuration for a DDlog program +#[derive(Debug, Clone, Copy)] +pub struct Config { + /// The number of timely + pub num_timely_workers: usize, + /// Whether extra regions should be added to the dataflow + /// + /// These extra regions *significantly* help with the readability + /// of the generated dataflows at the cost of a minor performance + /// penalty. Best used with [`ProfilingKind::TimelyProfiling`] + /// in order to see the + pub enable_debug_regions: bool, + /// The kind of profiling to enable + pub profiling_kind: ProfilingKind, + /// An amount of arrangement effort to spend each scheduling quantum + /// + /// See [`differential_dataflow::Config`] + pub differential_idle_merge_effort: Option, +} + +impl Config { + /// Create a new [`Config`] with the default settings + pub fn new() -> Self { + Self { + num_timely_workers: 1, + enable_debug_regions: false, + profiling_kind: ProfilingKind::default(), + differential_idle_merge_effort: None, + } + } + + pub(super) fn timely_config(&self) -> Result { + let mut config = TimelyConfig::process(self.num_timely_workers); + + // Allow configuring the merge behavior of ddflow + let idle_merge_effort = if self.differential_idle_merge_effort.is_some() { + self.differential_idle_merge_effort + + // Support for previous users who rely on the `DIFFERENTIAL_EAGER_MERGE` variable + // TODO: Remove the env var and expose this in all user apis + } else if let Ok(value) = env::var("DIFFERENTIAL_EAGER_MERGE") { + if value.is_empty() { + None + } else { + let merge_effort: isize = value.parse().map_err(|_| { + "the `DIFFERENTIAL_EAGER_MERGE` variable must be set to an integer value" + .to_owned() + })?; + + Some(merge_effort) + } + } else { + None + }; + + differential_dataflow::configure(&mut config.worker, &DDFlowConfig { idle_merge_effort }); + + Ok(config) + } +} + +impl Default for Config { + fn default() -> Self { + Self::new() + } +} + +/// The kind of profiling to be enabled for DDlog +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum ProfilingKind { + /// Disable all profiling + None, + /// Enable self-profiling + /// + /// Note: This spawns an additional thread and can have a + /// performance impact on the target program and also disables + /// general-purpose Timely Dataflow and Differential Dataflow + /// profiling + SelfProfiling, + /// Enable profiling for Timely Dataflow + TimelyProfiling { + /// Enable profiling for Differential Dataflow as well as Timely + differential_dataflow: bool, + }, +} + +impl ProfilingKind { + /// Returns `true` if the profiling_kind is [`None`] + pub const fn is_none(&self) -> bool { + matches!(self, Self::None) + } + + /// Returns `true` if the profiling_kind is [`SelfProfiling`] + pub const fn is_self_profiling(&self) -> bool { + matches!(self, Self::SelfProfiling) + } + + /// Returns `true` if the profiling_kind is [`TimelyProfiling`] + pub const fn is_timely_profiling(&self) -> bool { + matches!(self, Self::TimelyProfiling { .. }) + } +} + +impl Default for ProfilingKind { + fn default() -> Self { + Self::None + } +} + +#[derive(Debug)] +pub(super) struct SelfProfilingRig { + pub(super) profile: Option>>, + pub(super) profile_thread: Option>, + pub(super) profiling_data: Option, + pub(super) profile_cpu: Option>, + pub(super) profile_timely: Option>, +} + +impl SelfProfilingRig { + /// Create a new self profiling rig + /// + /// Note: Spawns a worker thread to process profiling messages + pub(super) fn new(config: &Config) -> Self { + if config.profiling_kind.is_self_profiling() { + let (profile_send, profile_recv) = crossbeam_channel::bounded(PROF_MSG_BUF_SIZE); + + // Profiling data structure + let profile = Arc::new(Mutex::new(Profile::new())); + + let (profile_cpu, profile_timely) = ( + Arc::new(AtomicBool::new(false)), + Arc::new(AtomicBool::new(false)), + ); + + // Thread to collect profiling data. + let cloned_profile = profile.clone(); + let profile_thread = + thread::spawn(move || Program::prof_thread_func(profile_recv, cloned_profile)); + + let profiling_data = + ProfilingData::new(profile_cpu.clone(), profile_timely.clone(), profile_send); + + Self { + profile: Some(profile), + profile_thread: Some(profile_thread), + profiling_data: Some(profiling_data), + profile_cpu: Some(profile_cpu), + profile_timely: Some(profile_timely), + } + } else { + Self { + profile: None, + profile_thread: None, + profiling_data: None, + profile_cpu: None, + profile_timely: None, + } + } + } +} diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index 7f9da1089..7fb2fef5d 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -13,6 +13,7 @@ // TODO: single input relation pub mod arrange; +pub mod config; mod timestamp; mod update; mod worker; @@ -25,11 +26,15 @@ use crate::{ ddval::*, profile::*, record::Mutator, - render::arrange_by::{ArrangeBy, ArrangementKind}, + render::{ + arrange_by::{ArrangeBy, ArrangementKind}, + RenderContext, + }, }; use arrange::{ antijoin_arranged, Arrangement as DataflowArrangement, ArrangementFlavor, Arrangements, }; +use config::{Config, SelfProfilingRig}; use crossbeam_channel::{Receiver, Sender}; use fnv::{FnvHashMap, FnvHashSet}; use std::{ @@ -37,7 +42,6 @@ use std::{ borrow::Cow, cmp, collections::{hash_map, BTreeSet}, - env, fmt::{self, Debug, Formatter}, iter::{self, Cycle, Skip}, ops::Range, @@ -45,10 +49,10 @@ use std::{ atomic::{AtomicBool, Ordering}, Arc, Mutex, }, - thread::{self, JoinHandle}, + thread::JoinHandle, }; use timestamp::ToTupleTS; -use worker::{DDlogWorker, ProfilingData}; +use worker::DDlogWorker; use differential_dataflow::lattice::Lattice; use differential_dataflow::operators::arrange::arrangement::Arranged; @@ -58,20 +62,17 @@ use differential_dataflow::trace::implementations::ord::OrdKeySpine as DefaultKe use differential_dataflow::trace::implementations::ord::OrdValSpine as DefaultValTrace; use differential_dataflow::trace::wrappers::enter::TraceEnter; use differential_dataflow::trace::{BatchReader, Cursor, TraceReader}; -use differential_dataflow::{Collection, Config as DDFlowConfig}; +use differential_dataflow::Collection; use dogsdogsdogs::{ altneu::AltNeu, calculus::{Differentiate, Integrate}, operators::lookup_map, }; +use timely::communication::{initialize::WorkerGuards, Allocator}; use timely::dataflow::scopes::*; use timely::order::TotalOrder; use timely::progress::{timestamp::Refines, PathSummary, Timestamp}; use timely::worker::Worker; -use timely::{ - communication::{initialize::WorkerGuards, Allocator}, - execute::Config as TimelyConfig, -}; type ValTrace = DefaultValTrace; type KeyTrace = DefaultKeyTrace; @@ -776,6 +777,7 @@ impl Arrangement { fn build_arrangement_root( &self, + render_context: &RenderContext, collection: &Collection, ) -> DataflowArrangement, TKeyAgent> where @@ -804,11 +806,12 @@ impl Arrangement { kind, target_relation: self.name().into(), } - .render_root(collection) + .render_root(render_context, collection) } fn build_arrangement( &self, + render_context: &RenderContext, collection: &Collection, ) -> DataflowArrangement, TKeyAgent> where @@ -836,7 +839,7 @@ impl Arrangement { kind, target_relation: self.name().into(), } - .render(collection) + .render(render_context, collection) } } @@ -873,13 +876,13 @@ pub struct RunningProgram { need_to_flush: bool, timestamp: TS, /// CPU profiling enabled (can be expensive). - profile_cpu: Arc, + profile_cpu: Option>, /// Consume timely_events and output them to CSV file. Can be expensive. - profile_timely: Arc, + profile_timely: Option>, /// Profiling thread. prof_thread_handle: Option>, /// Profiling statistics. - pub profile: Arc>, + pub profile: Option>>, worker_round_robbin: Skip>>, } @@ -988,70 +991,49 @@ enum Reply { } impl Program { - /// Instantiate the program with `nworkers` timely threads. + /// Instantiate the program with `number_workers` timely threads. pub fn run(&self, number_workers: usize) -> Result { + let config = Config { + num_timely_workers: number_workers, + ..Default::default() + }; + + self.run_with_config(config) + } + + /// Initialize the program with the given configuration + pub fn run_with_config(&self, config: Config) -> Result { // Setup channels to communicate with the dataflow. // We use async channels to avoid deadlocks when workers are parked in // `step_or_park`. This has the downside of introducing an unbounded buffer // that is only guaranteed to be fully flushed when the transaction commits. - let (request_send, request_recv): (Vec<_>, Vec<_>) = (0..number_workers) + let (request_send, request_recv): (Vec<_>, Vec<_>) = (0..config.num_timely_workers) .map(|_| crossbeam_channel::unbounded::()) .unzip(); let request_recv = Arc::from(request_recv); // Channels for responses from worker threads. - let (reply_send, reply_recv): (Vec<_>, Vec<_>) = (0..number_workers) + let (reply_send, reply_recv): (Vec<_>, Vec<_>) = (0..config.num_timely_workers) .map(|_| crossbeam_channel::unbounded::()) .unzip(); let reply_send = Arc::from(reply_send); - let (prof_send, prof_recv) = crossbeam_channel::bounded(PROF_MSG_BUF_SIZE); - - // Profile data structure - let profile = Arc::new(Mutex::new(Profile::new())); - let (profile_cpu, profile_timely) = ( - Arc::new(AtomicBool::new(false)), - Arc::new(AtomicBool::new(false)), - ); - - // Thread to collect profiling data. - let cloned_profile = profile.clone(); - let prof_thread = thread::spawn(move || Self::prof_thread_func(prof_recv, cloned_profile)); + let profiling_rig = SelfProfilingRig::new(&config); // Clone the program so that it can be moved into the timely computation let program = Arc::new(self.clone()); - let profiling = ProfilingData::new(profile_cpu.clone(), profile_timely.clone(), prof_send); - - let mut config = TimelyConfig::process(number_workers); - - // Allow configuring the merge behavior of ddflow - // FIXME: Expose the merge behavior to all apis and deprecate the env var - if let Ok(value) = env::var("DIFFERENTIAL_EAGER_MERGE") { - let idle_merge_effort = if value.is_empty() { - None - } else { - let merge_effort: isize = value.parse().map_err(|_| { - "the `DIFFERENTIAL_EAGER_MERGE` variable must be set to an integer value" - .to_owned() - })?; - - Some(merge_effort) - }; - - differential_dataflow::configure( - &mut config.worker, - &DDFlowConfig { idle_merge_effort }, - ); - } + let timely_config = config.timely_config()?; + let (worker_config, profiling_data) = (config, profiling_rig.profiling_data.clone()); // Start up timely computation. let worker_guards = timely::execute( - config, + timely_config, move |worker: &mut Worker| -> Result<_, String> { let worker = DDlogWorker::new( worker, + worker_config, program.clone(), - profiling.clone(), + profiling_data.clone(), Arc::clone(&request_recv), Arc::clone(&reply_send), ); @@ -1116,11 +1098,11 @@ impl Program { transaction_in_progress: false, need_to_flush: false, timestamp: 1, - profile_cpu, - profile_timely, - prof_thread_handle: Some(prof_thread), - profile, - worker_round_robbin: (0..number_workers).cycle().skip(0), + profile_cpu: profiling_rig.profile_cpu, + profile_timely: profiling_rig.profile_timely, + prof_thread_handle: profiling_rig.profile_thread, + profile: profiling_rig.profile, + worker_round_robbin: (0..config.num_timely_workers).cycle().skip(0), }; // Wait for the initial transaction to complete. running_program.await_flush_ack()?; @@ -1977,11 +1959,17 @@ impl RunningProgram { /// `enable = true` - enables forwarding. This can be expensive in large dataflows. /// `enable = false` - disables forwarding. pub fn enable_cpu_profiling(&self, enable: bool) { - self.profile_cpu.store(enable, Ordering::SeqCst); + if let Some(profile_cpu) = self.profile_cpu.as_ref() { + profile_cpu.store(enable, Ordering::SeqCst); + } + // TODO: Log warning if self profiling is disabled } pub fn enable_timely_profiling(&self, enable: bool) { - self.profile_timely.store(enable, Ordering::SeqCst); + if let Some(profile_timely) = self.profile_timely.as_ref() { + profile_timely.store(enable, Ordering::SeqCst); + } + // TODO: Log warning if self profiling is disabled } /// Terminate program, killing all worker threads. diff --git a/rust/template/differential_datalog/src/program/worker.rs b/rust/template/differential_datalog/src/program/worker.rs index e8099b9ce..b3af3249a 100644 --- a/rust/template/differential_datalog/src/program/worker.rs +++ b/rust/template/differential_datalog/src/program/worker.rs @@ -7,8 +7,10 @@ use crate::{ }, program::{ arrange::{Arrangement, Arrangements}, + config::{Config, ProfilingKind}, ArrId, Dep, Msg, ProgNode, Program, Reply, Update, TS, }, + render::RenderContext, variable::Variable, }; use crossbeam_channel::{Receiver, Sender}; @@ -31,6 +33,7 @@ use fnv::{FnvBuildHasher, FnvHashMap}; use std::{ collections::{BTreeMap, BTreeSet, HashMap}, mem, + net::TcpStream, ops::Deref, rc::Rc, sync::{ @@ -76,11 +79,13 @@ type DelayedVarMap = FnvHashMap< pub struct DDlogWorker<'a> { /// The timely worker instance worker: &'a mut Worker, + /// The DDlog program's configuration settings + config: Config, /// The program this worker is executing program: Arc, /// Information on which metrics are enabled and a /// channel for sending profiling data - profiling: ProfilingData, + profiling: Option, /// The current worker's receiver for receiving messages request_receiver: Receiver, /// The current worker's sender for sending messages @@ -92,8 +97,9 @@ impl<'a> DDlogWorker<'a> { #[allow(clippy::too_many_arguments)] pub(super) fn new( worker: &'a mut Worker, + config: Config, program: Arc, - profiling: ProfilingData, + profiling: Option, request_receivers: Arc<[Receiver]>, reply_senders: Arc<[Sender]>, ) -> Self { @@ -101,6 +107,7 @@ impl<'a> DDlogWorker<'a> { Self { worker, + config, program, profiling, request_receiver: request_receivers[worker_index].clone(), @@ -387,66 +394,89 @@ impl<'a> DDlogWorker<'a> { } /// Initialize timely and differential profiling logging hooks - fn init_profiling(&self) { - let profiling = self.profiling.clone(); - self.worker - .log_register() - .insert::("timely", move |_time, data| { - let profile_cpu = profiling.is_cpu_enabled(); - let profile_timely = profiling.is_timely_enabled(); - - // Filter out events we don't care about to avoid the overhead of sending - // the event around just to drop it eventually. - let filtered: Vec<((Duration, usize, TimelyEvent), Option)> = data - .drain(..) - .filter(|event| { - match event.2 { - // Always send Operates events as they're used for always-on memory profiling. - TimelyEvent::Operates(_) => true, - - // Send scheduling events if profiling is enabled - TimelyEvent::Schedule(_) => profile_cpu || profile_timely, - - // Send timely events if timely profiling is enabled - TimelyEvent::GuardedMessage(_) - | TimelyEvent::Messages(_) - | TimelyEvent::Park(_) - | TimelyEvent::PushProgress(_) => profile_timely, - - _ => false, - } - }) - .map(|(d, s, e)| match e { - // Only Operate events care about the context string. - TimelyEvent::Operates(_) => ((d, s, e), Some(get_prof_context())), - _ => ((d, s, e), None), - }) - .collect(); + fn init_profiling(&mut self) { + if let Some(profiling) = self.profiling.clone() { + let timely_profiling = profiling.clone(); + self.worker + .log_register() + .insert::("timely", move |_time, data| { + let profile_cpu = timely_profiling.is_cpu_enabled(); + let profile_timely = timely_profiling.is_timely_enabled(); + + // Filter out events we don't care about to avoid the overhead of sending + // the event around just to drop it eventually. + let filtered: Vec<((Duration, usize, TimelyEvent), Option)> = data + .drain(..) + .filter(|event| { + match event.2 { + // Always send Operates events as they're used for always-on memory profiling. + TimelyEvent::Operates(_) => true, + + // Send scheduling events if profiling is enabled + TimelyEvent::Schedule(_) => profile_cpu || profile_timely, + + // Send timely events if timely profiling is enabled + TimelyEvent::GuardedMessage(_) + | TimelyEvent::Messages(_) + | TimelyEvent::Park(_) + | TimelyEvent::PushProgress(_) => profile_timely, + + _ => false, + } + }) + .map(|(d, s, e)| match e { + // Only Operate events care about the context string. + TimelyEvent::Operates(_) => ((d, s, e), Some(get_prof_context())), + _ => ((d, s, e), None), + }) + .collect(); + + // If there are any profiling events, record them + if !filtered.is_empty() { + timely_profiling.record(ProfMsg::TimelyMessage( + filtered, + profile_cpu, + profile_timely, + )); + } + }); - // If there are any profiling events, record them - if !filtered.is_empty() { - profiling.record(ProfMsg::TimelyMessage( - filtered, - profile_cpu, - profile_timely, - )); - } - }); - - let profiling = self.profiling.clone(); - self.worker.log_register().insert::( - "differential/arrange", - move |_time, data| { - // If there are events, send them through the profiling channel - if !data.is_empty() { - profiling.record(ProfMsg::DifferentialMessage(mem::take(data))); + self.worker.log_register().insert::( + "differential/arrange", + move |_time, data| { + // If there are events, send them through the profiling channel + if !data.is_empty() { + profiling.record(ProfMsg::DifferentialMessage(mem::take(data))); + } + }, + ); + } else if let ProfilingKind::TimelyProfiling { + differential_dataflow, + } = self.config.profiling_kind + { + if differential_dataflow { + if let Ok(addr) = ::std::env::var("DIFFERENTIAL_LOG_ADDR") { + if !addr.is_empty() { + if let Ok(stream) = TcpStream::connect(&addr) { + differential_dataflow::logging::enable(self.worker, stream); + // TODO: Use tracing to log that logging connected successfully + } else { + panic!("Could not connect to differential log address: {:?}", addr); + } + } } - }, - ); + } + + // Timely already has its logging hooks set by default + } else if self.config.profiling_kind.is_none() { + self.worker.log_register().remove("timely"); + self.worker.log_register().remove("differential/arrange"); + } } fn session_dataflow(&mut self, mut probe: ProbeHandle) -> Result { let program = self.program.clone(); + let render_context = RenderContext::new(self.config); self.worker.dataflow::( |outer: &mut Child, TS>| -> Result<_, String> { @@ -459,7 +489,10 @@ impl<'a> DDlogWorker<'a> { program.nodes.len(), FnvBuildHasher::default(), ); - let mut arrangements = FnvHashMap::default(); + let mut arrangements: FnvHashMap< + ArrId, + Arrangement<_, Weight, TValAgent, TKeyAgent>, + > = FnvHashMap::default(); // Create an `Enabled` relation used to enforce the dataflow termination in the // presence of delayed relations. A delayed relation can potentially generate an @@ -507,6 +540,7 @@ impl<'a> DDlogWorker<'a> { rel, outer, &*program, + &render_context, &mut sessions, &mut collections, &mut arrangements, @@ -523,6 +557,7 @@ impl<'a> DDlogWorker<'a> { node_id, outer, &*program, + &render_context, &mut sessions, &mut collections, &mut arrangements, @@ -592,90 +627,83 @@ impl<'a> DDlogWorker<'a> { } } -fn render_relation<'a>( +// TODO: Add back regions for relations +fn render_relation<'a, S>( relation: &Relation, // TODO: Shift to generic representations for ddflow-related structs - scope: &mut Child<'a, Worker, TS>, + scope: &mut S, program: &Program, + render_context: &RenderContext, sessions: &mut FnvHashMap>, - collections: &mut FnvHashMap< - RelId, - Collection, TS>, DDValue, Weight>, - >, - arrangements: &mut FnvHashMap< - ArrId, - Arrangement, TS>, Weight, TValAgent, TKeyAgent>, - >, - delayed_vars: &DelayedVarMap, TS>>, -) { - scope.clone().region_named(relation.name(), |region| { - // Relation may already be in the map if it was created by an `Apply` node - let mut collection = if let Some(collection) = collections.remove(&relation.id) { - collection.enter_region(region) - } else { - let (session, collection) = scope.new_collection::(); - sessions.insert(relation.id, session); - - // TODO: Find a way to make the collection within the nested region - collection.enter_region(region) + collections: &mut FnvHashMap>, + arrangements: &mut FnvHashMap, TKeyAgent>>, + delayed_vars: &DelayedVarMap, +) where + S: Scope, +{ + // Relation may already be in the map if it was created by an `Apply` node + let mut collection = if let Some(collection) = collections.remove(&relation.id) { + collection + } else { + let (session, collection) = scope.new_collection::(); + sessions.insert(relation.id, session); + + // TODO: Find a way to make the collection within the nested region + collection + }; + + let entered_arrangements: FnvHashMap<_, ArrangementFlavor<_, TS>> = arrangements + .iter() + .map(|(&arr_id, arr)| (arr_id, ArrangementFlavor::Local(arr.clone()))) + .collect(); + + // apply rules + // TODO: Regions for rules + let rule_collections = relation.rules.iter().map(|rule| { + let get_rule_collection = |relation_id| { + if let Some(collection) = collections.get(&relation_id) { + Some(collection.clone()) + } else { + delayed_vars + .get(&relation_id) + .map(|(_, _, collection)| collection.clone()) + } }; - let entered_arrangements: FnvHashMap<_, ArrangementFlavor<_, TS>> = arrangements - .iter() - .map(|(&arr_id, arr)| (arr_id, ArrangementFlavor::Local(arr.enter_region(region)))) - .collect(); - - // apply rules - // TODO: Regions for rules - let rule_collections = relation.rules.iter().map(|rule| { - let get_rule_collection = |relation_id| { - if let Some(collection) = collections.get(&relation_id) { - Some(collection.enter_region(region)) - } else { - delayed_vars - .get(&relation_id) - .map(|(_, _, collection)| collection.enter_region(region)) - } - }; + program.mk_rule( + rule, + get_rule_collection, + Arrangements { + arrangements: &entered_arrangements, + }, + ) + }); - program.mk_rule( - rule, - get_rule_collection, - Arrangements { - arrangements: &entered_arrangements, - }, - ) + if rule_collections.len() > 0 { + collection = with_prof_context(&format!("concatenate rules for {}", relation.name), || { + collection.concatenate(rule_collections) }); + } - if rule_collections.len() > 0 { - collection = - with_prof_context(&format!("concatenate rules for {}", relation.name), || { - collection.concatenate(rule_collections) - }); - } - - // don't distinct input collections, as this is already done by the set_update logic - if !relation.input && relation.distinct { - collection = with_prof_context(&format!("{}.threshold_total", relation.name), || { - collection.threshold_total(|_, c| if *c == 0 { 0 } else { 1 }) - }); - } + // don't distinct input collections, as this is already done by the set_update logic + if !relation.input && relation.distinct { + collection = with_prof_context(&format!("{}.threshold_total", relation.name), || { + collection.threshold_total(|_, c| if *c == 0 { 0 } else { 1 }) + }); + } - // create arrangements - // TODO: Arrangements have their own shebang, region them off too - for (arr_id, arrangement) in relation.arrangements.iter().enumerate() { - with_prof_context(arrangement.name(), || { - arrangements.insert( - (relation.id, arr_id), - arrangement - .build_arrangement_root(&collection) - .leave_region(), - ) - }); - } + // create arrangements + // TODO: Arrangements have their own shebang, region them off too + for (arr_id, arrangement) in relation.arrangements.iter().enumerate() { + with_prof_context(arrangement.name(), || { + arrangements.insert( + (relation.id, arr_id), + arrangement.build_arrangement_root(&render_context, &collection), + ) + }); + } - collections.insert(relation.id, collection.leave_region()); - }); + collections.insert(relation.id, collection); } // TODO: Regions for SCCs @@ -685,6 +713,7 @@ fn render_scc<'a>( // TODO: Shift to generic representations for ddflow-related structs scope: &mut Child<'a, Worker, TS>, program: &Program, + render_context: &RenderContext, sessions: &mut FnvHashMap>, collections: &mut FnvHashMap< RelId, @@ -747,7 +776,7 @@ fn render_scc<'a>( with_prof_context(&format!("local {}", arr.name()), || { local_arrangements.insert( (rel.rel.id, i), - arr.build_arrangement(&*vars.get(&rel.rel.id)?), + arr.build_arrangement(render_context, &*vars.get(&rel.rel.id)?), ) }); } @@ -857,7 +886,10 @@ fn render_scc<'a>( format!("no collection found for relation ID {}", rel.rel.id) })?; - Ok(arrangements.insert((rel.rel.id, i), arr.build_arrangement(collection))) + Ok(arrangements.insert( + (rel.rel.id, i), + arr.build_arrangement(render_context, collection), + )) }, )?; } @@ -867,7 +899,7 @@ fn render_scc<'a>( Ok(()) } -#[derive(Clone)] +#[derive(Debug, Clone)] pub struct ProfilingData { /// Whether CPU profiling is enabled cpu_enabled: Arc, diff --git a/rust/template/differential_datalog/src/render/arrange_by.rs b/rust/template/differential_datalog/src/render/arrange_by.rs index 2f6169881..fa704c009 100644 --- a/rust/template/differential_datalog/src/render/arrange_by.rs +++ b/rust/template/differential_datalog/src/render/arrange_by.rs @@ -2,7 +2,7 @@ use crate::{ dataflow::{diff_distinct, FilterMap, MapExt}, ddval::DDValue, program::arrange::Arrangement, - render::{Offset, Str, TraceKey, TraceValue}, + render::{Offset, RenderContext, Str, TraceKey, TraceValue}, }; use differential_dataflow::{ difference::Abelian, @@ -52,7 +52,11 @@ type Arranged = Arrangement>, TraceAgent>>; impl<'a> ArrangeBy<'a> { - pub fn render(&self, collection: &Collection) -> Arranged + pub fn render( + &self, + _context: &RenderContext, + collection: &Collection, + ) -> Arranged where S: Scope, S::Timestamp: Lattice, @@ -108,6 +112,7 @@ impl<'a> ArrangeBy<'a> { pub fn render_root( &self, + _context: &RenderContext, collection: &Collection, ) -> Arrangement>, TraceAgent>> where diff --git a/rust/template/differential_datalog/src/render/mod.rs b/rust/template/differential_datalog/src/render/mod.rs index 9fb4bdec7..19d2157b5 100644 --- a/rust/template/differential_datalog/src/render/mod.rs +++ b/rust/template/differential_datalog/src/render/mod.rs @@ -1,4 +1,4 @@ -use crate::ddval::DDValue; +use crate::{ddval::DDValue, program::config::Config}; use differential_dataflow::trace::implementations::ord::{OrdKeySpine, OrdValSpine}; use std::borrow::Cow; use timely::dataflow::ScopeParent; @@ -13,3 +13,14 @@ pub type TraceValue = OrdValSpine::Timestamp, R, O>; pub type TraceKey = OrdKeySpine::Timestamp, R, O>; + +#[derive(Debug)] +pub struct RenderContext { + pub config: Config, +} + +impl RenderContext { + pub const fn new(config: Config) -> Self { + Self { config } + } +} diff --git a/rust/template/src/api/mod.rs b/rust/template/src/api/mod.rs index 899deb8a7..7a6d2eeb4 100644 --- a/rust/template/src/api/mod.rs +++ b/rust/template/src/api/mod.rs @@ -244,7 +244,12 @@ impl DDlogProfiling for HDDlog { fn profile(&self) -> Result { self.record_command(|r| r.profile()); let rprog = self.prog.lock().unwrap(); - let profile: String = rprog.profile.lock().unwrap().to_string(); + let profile: String = rprog + .profile + .as_ref() + .map(|profile| profile.lock().unwrap().to_string()) + .unwrap_or_else(String::new); + Ok(profile) } } diff --git a/src/Language/DifferentialDatalog/Compile.hs b/src/Language/DifferentialDatalog/Compile.hs index b5ed191d2..59485daa4 100644 --- a/src/Language/DifferentialDatalog/Compile.hs +++ b/src/Language/DifferentialDatalog/Compile.hs @@ -210,6 +210,7 @@ rustLibFiles = , ("differential_datalog/src/program/arrange.rs" , $(embedFile "rust/template/differential_datalog/src/program/arrange.rs")) , ("differential_datalog/src/program/timestamp.rs" , $(embedFile "rust/template/differential_datalog/src/program/timestamp.rs")) , ("differential_datalog/src/program/worker.rs" , $(embedFile "rust/template/differential_datalog/src/program/worker.rs")) + , ("differential_datalog/src/program/config.rs" , $(embedFile "rust/template/differential_datalog/src/program/config.rs")) , ("differential_datalog/src/record/mod.rs" , $(embedFile "rust/template/differential_datalog/src/record/mod.rs")) , ("differential_datalog/src/record/tuples.rs" , $(embedFile "rust/template/differential_datalog/src/record/tuples.rs")) , ("differential_datalog/src/record/arrays.rs" , $(embedFile "rust/template/differential_datalog/src/record/arrays.rs")) From bdf6c4dd4d6c12ff7d8021e651fa1c02e357572a Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Mon, 19 Apr 2021 12:03:02 -0500 Subject: [PATCH 15/17] Set default profiling mode in Program::run to be backwards compatable --- Cargo.toml | 2 +- rust/template/differential_datalog/src/program/mod.rs | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 90a871434..3a4ae884b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -4,7 +4,7 @@ members = [ "rust/template/ovsdb", "rust/template/cmd_parser", "rust/template/ddlog_derive", - #"rust/template/distributed_datalog", + "rust/template/distributed_datalog", "rust/template/differential_datalog", "rust/template/differential_datalog_test", ] diff --git a/rust/template/differential_datalog/src/program/mod.rs b/rust/template/differential_datalog/src/program/mod.rs index 7fb2fef5d..9914c091b 100644 --- a/rust/template/differential_datalog/src/program/mod.rs +++ b/rust/template/differential_datalog/src/program/mod.rs @@ -25,6 +25,7 @@ pub use update::Update; use crate::{ ddval::*, profile::*, + program::config::ProfilingKind, record::Mutator, render::{ arrange_by::{ArrangeBy, ArrangementKind}, @@ -995,6 +996,7 @@ impl Program { pub fn run(&self, number_workers: usize) -> Result { let config = Config { num_timely_workers: number_workers, + profiling_kind: ProfilingKind::SelfProfiling, ..Default::default() }; From 0ac353b74da7fa74858dfeba16180ad24484a9fe Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Mon, 19 Apr 2021 12:03:46 -0500 Subject: [PATCH 16/17] Finished thought --- rust/template/differential_datalog/src/program/config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/template/differential_datalog/src/program/config.rs b/rust/template/differential_datalog/src/program/config.rs index 6be13821c..6698533c1 100644 --- a/rust/template/differential_datalog/src/program/config.rs +++ b/rust/template/differential_datalog/src/program/config.rs @@ -22,7 +22,7 @@ pub struct Config { /// These extra regions *significantly* help with the readability /// of the generated dataflows at the cost of a minor performance /// penalty. Best used with [`ProfilingKind::TimelyProfiling`] - /// in order to see the + /// in order to visualize the dataflow graph pub enable_debug_regions: bool, /// The kind of profiling to enable pub profiling_kind: ProfilingKind, From 6bb3fe71b2a3a6ab8d515229cb7e408d04dd98d5 Mon Sep 17 00:00:00 2001 From: Chase Wilson Date: Mon, 19 Apr 2021 13:09:09 -0500 Subject: [PATCH 17/17] Fixed clippy lints & formatting --- rust/template/differential_datalog/src/program/worker.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/rust/template/differential_datalog/src/program/worker.rs b/rust/template/differential_datalog/src/program/worker.rs index b3af3249a..e5f286815 100644 --- a/rust/template/differential_datalog/src/program/worker.rs +++ b/rust/template/differential_datalog/src/program/worker.rs @@ -458,8 +458,8 @@ impl<'a> DDlogWorker<'a> { if let Ok(addr) = ::std::env::var("DIFFERENTIAL_LOG_ADDR") { if !addr.is_empty() { if let Ok(stream) = TcpStream::connect(&addr) { - differential_dataflow::logging::enable(self.worker, stream); // TODO: Use tracing to log that logging connected successfully + differential_dataflow::logging::enable(self.worker, stream); } else { panic!("Could not connect to differential log address: {:?}", addr); } @@ -467,7 +467,7 @@ impl<'a> DDlogWorker<'a> { } } - // Timely already has its logging hooks set by default + // Timely already has its logging hooks set by default } else if self.config.profiling_kind.is_none() { self.worker.log_register().remove("timely"); self.worker.log_register().remove("differential/arrange"); @@ -628,7 +628,8 @@ impl<'a> DDlogWorker<'a> { } // TODO: Add back regions for relations -fn render_relation<'a, S>( +#[allow(clippy::clippy::too_many_arguments)] +fn render_relation( relation: &Relation, // TODO: Shift to generic representations for ddflow-related structs scope: &mut S, @@ -707,6 +708,7 @@ fn render_relation<'a, S>( } // TODO: Regions for SCCs +#[allow(clippy::clippy::too_many_arguments)] fn render_scc<'a>( rels: &[RecursiveRelation], node_id: RelId,