├── .gitignore ├── .travis.yml ├── Cargo.toml ├── LICENSE ├── README.md ├── examples ├── four-cliques.rs ├── four-cliques_shuffled.rs ├── motif-careful-splits-lean.rs ├── motif-careful-splits.rs ├── motif-careful.rs ├── motif.rs ├── motifs-single.rs ├── old_examples │ ├── five-cliques.rs │ └── five-cliques_shuffled.rs ├── q3-lean.rs ├── q6-lean-alt.rs ├── q6-lean.rs ├── q7-lean.rs ├── seed-q3-lean.rs ├── seed-q5-lean.rs ├── seed-q6-lean-alt.rs ├── seed-q6-lean.rs ├── seed-q7-lean.rs ├── seed.rs ├── triangles-lean.rs ├── triangles-single.rs ├── triangles.rs ├── triangles_updates.rs └── triangles_updates_edges.rs ├── naive ├── Cargo.toml ├── README.md └── src │ ├── bin │ ├── q0.rs │ ├── q1.rs │ ├── q2-star.rs │ ├── q2.rs │ ├── q3.rs │ ├── q5-star.rs │ ├── q6-star.rs │ ├── q6.rs │ └── q7.rs │ └── lib.rs ├── src ├── extender.rs ├── index.rs ├── lib.rs └── motif.rs └── static_graph ├── Cargo.toml ├── LICENSE ├── README.md ├── examples ├── diamonds.rs ├── layout.rs ├── triangles-single.rs └── triangles.rs └── src ├── flattener.rs ├── graph.rs ├── lib.rs └── typedrw.rs /.gitignore: -------------------------------------------------------------------------------- 1 | target 2 | Cargo.lock 3 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: rust 2 | sudo: required 3 | rust: 4 | - stable 5 | before_script: 6 | - pip install 'travis-cargo<0.2' --user && export PATH=$HOME/.local/bin:$PATH 7 | script: 8 | - | 9 | travis-cargo build && 10 | travis-cargo test && 11 | travis-cargo bench && 12 | travis-cargo doc 13 | after_success: 14 | - travis-cargo --only stable doc-upload 15 | env: 16 | global: 17 | secure: aExpVdQF9S4FJIx3kfHx2f7zZW1XhtJuh9TuYTUtcs5V1hkKUExucjb5pOU/fcVW+NDDCurSDgoPSkmu6spZzdu+DGIReOP9/Dm573TW3epCu1NglE1057CKc1e8dpvRvKNYlshau2csudFPl7/mkB0tBYauJntb/KknRTJv6QA= 18 | -------------------------------------------------------------------------------- /Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "alg3_dynamic" 3 | version = "0.1.0" 4 | authors = ["Frank McSherry "] 5 | 6 | [dependencies] 7 | timely="0.7.0" 8 | 9 | [dependencies.graph_map] 10 | git="http://github.com/frankmcsherry/graph-map" 11 | 12 | [profile.release] 13 | opt-level = 3 14 | debug = true 15 | rpath = false 16 | lto = false 17 | debug-assertions = false 18 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | The MIT License (MIT) 2 | 3 | Copyright (c) 2015 Frank McSherry 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | 23 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # dataflow-join 2 | 3 | A streaming implementation of Ngo et al's GenericJoin in timely dataflow. 4 | 5 | --- 6 | 7 | Ngo et al presented a very cool join algorithm, some details of which are described in [a blog post](http://www.frankmcsherry.org/dataflow/relational/join/2015/04/11/genericjoin.html). This project (a collaboration with Khaled Ammar and Semih Salihoglu) extends Ngo et al's algorithm to the case where the underlying relations change, allowing us to track the results of complex join queries as they change. 8 | 9 | A [series of posts](https://github.com/frankmcsherry/blog/blob/master/posts/2016-09-17.md) describe the ideas behind this implementation. 10 | 11 | ## An example: graph motifs 12 | 13 | For an example, the [`examples/motif.rs`](https://github.com/frankmcsherry/dataflow-join/blob/master/examples/motif.rs) program takes the description of a directed graph motif (to be explained) and a list of graph edges, and reports the change in the numbers of these motifs as we stream the edges in. To look for directed triangles of the form `(a,b), (a,c), (b,c)`, using the livejournal graph edges in a random order (any text file where each line has the form `src dst`), loading the first 68 million edges, and then swinging over the remaining entries in batches of 1,000, we would type: 14 | 15 | cargo run --release --example motif -- 3 0 1 0 2 1 2 ./soc-LiveJournal1.random.txt 68000000 1000 inspect 16 | 17 | The motif is expressed clumsily at the moment, but has the form `num_edges [src dst]^num_edges` where you indicate how many edges (3 for triangles) and then repeatedly indicate edges in the motif. There are bugs here, including symmetric edges ((a,b) and (b,a)) not being handled correctly, among others I probably don't know about yet. 18 | 19 | If you run the command above you should see something like: 20 | 21 | Echidnatron% cargo run --release --example motif -- 3 0 1 0 2 1 2 ./soc-LiveJournal1.random.txt 68000000 1000 inspect 22 | Finished release [optimized + debuginfo] target(s) in 0.0 secs 23 | Running `target/release/examples/motif 3 0 1 0 2 1 2 ./soc-LiveJournal1.random.txt 68000000 1000 inspect` 24 | motif: [(0, 1), (0, 2), (1, 2)] 25 | filename: "./soc-LiveJournal1.random.txt" 26 | index initialized: (4295196, 68000000) 27 | index initialized: (4474513, 68000000) 28 | Duration { secs: 36, nanos: 995859508 } [worker 0] data loaded 29 | Duration { secs: 36, nanos: 995924311 } [worker 0] indices merged 30 | (Root, 2): [37695] 31 | (Root, 3): [36745] 32 | (Root, 4): [42000] 33 | ... 34 | 35 | which reports the input reading, index building (forward and reverse), and then starts reporting the number of changes to the directed motif in each batch of 1,000 edges. In this example all changes are edge additions, so all the motif changes are all additions as well. 36 | 37 | As the program continues to run, it should result in something like 38 | 39 | ... 40 | (Root, 993): [37462] 41 | (Root, 994): [44505] 42 | (Root, 995): [30189] 43 | elapsed: 67.91147424298106 total motifs at this process: 39772836 44 | Echidnatron% 45 | 46 | This reports the total number of motif changes observed at this process and an elapsed running time. 47 | 48 | ## Parallelism 49 | 50 | Like other timely dataflow programs, we can increase the number of workers to parallelize computation (and printing of things to the screen): 51 | 52 | Echidnatron% cargo run --release --example motif -- 3 0 1 0 2 1 2 ./soc-LiveJournal1.random.txt 68000000 1000 inspect -w2 53 | Finished release [optimized + debuginfo] target(s) in 0.0 secs 54 | Running `target/release/examples/motif 3 0 1 0 2 1 2 ./soc-LiveJournal1.random.txt 68000000 1000 inspect -w2` 55 | motif: [(0, 1), (0, 2), (1, 2)] 56 | filename: "./soc-LiveJournal1.random.txt" 57 | motif: [(0, 1), (0, 2), (1, 2)] 58 | filename: "./soc-LiveJournal1.random.txt" 59 | index initialized: (2147644, 34060151) 60 | index initialized: (2147552, 33939849) 61 | index initialized: (2237328, 34057137) 62 | index initialized: (2237185, 33942863) 63 | Duration { secs: 22, nanos: 149237598 } [worker 1] data loaded 64 | Duration { secs: 22, nanos: 149245453 } [worker 0] data loaded 65 | Duration { secs: 22, nanos: 149295018 } [worker 0] indices merged 66 | Duration { secs: 22, nanos: 149296396 } [worker 1] indices merged 67 | (Root, 2): [15863] 68 | (Root, 2): [21832] 69 | (Root, 3): [16715] 70 | (Root, 3): [20030] 71 | ... 72 | (Root, 994): [22018] 73 | (Root, 994): [22487] 74 | (Root, 995): [16231] 75 | (Root, 995): [13958] 76 | elapsed: 40.51301470899489 total motifs at this process: 39772836 77 | Echidnatron% 78 | 79 | Each worker reads the input file, separates lines, but only parses lines whose index equals their worker id modulo the number of workers. For now this means that each machine needs access to a copy of the file, which should be improved. 80 | 81 | ## Reporting 82 | 83 | The `inspect` argument attaches a dataflow fragment to count each of the changes, and by omitting it performance improves somewhat (by shuttling around much less data). 84 | 85 | Echidnatron% cargo run --release --example motif -- 3 0 1 0 2 1 2 ./soc-LiveJournal1.random.txt 68000000 1000 -w2 86 | Finished release [optimized + debuginfo] target(s) in 0.0 secs 87 | Running `target/release/examples/motif 3 0 1 0 2 1 2 ./soc-LiveJournal1.random.txt 68000000 1000 -w2` 88 | motif: [(0, 1), (0, 2), (1, 2)] 89 | motif: [(0, 1), (0, 2), (1, 2)] 90 | filename: "./soc-LiveJournal1.random.txt" 91 | filename: "./soc-LiveJournal1.random.txt" 92 | index initialized: (2147552, 33939849) 93 | index initialized: (2147644, 34060151) 94 | index initialized: (2237185, 33942863) 95 | index initialized: (2237328, 34057137) 96 | Duration { secs: 21, nanos: 904351750 } [worker 0] data loaded 97 | Duration { secs: 21, nanos: 904355535 } [worker 1] data loaded 98 | Duration { secs: 21, nanos: 904398564 } [worker 1] indices merged 99 | Duration { secs: 21, nanos: 904412222 } [worker 0] indices merged 100 | elapsed: 35.086652864003554 total motifs at this process: 0 101 | Echidnatron% 102 | 103 | The total reported is zero, because we detached the counting infrastructure. 104 | 105 | ## Other example motifs 106 | 107 | The infrastructure decouples loading graph data from observing changes in motif counts, allowing us to observe the *change* in motif counts without paying the cost of determining the original count, allowing us to track relatively complex motifs whose computation would otherwise be rather painful. 108 | 109 | Here are some examples: 110 | 111 | ### Directed 3-cycles 112 | 113 | These are also triangles, but ones that form a cycle, rather than pointing from `a` to `b` to `c`. 114 | 115 | Echidnatron% cargo run --release --example motif -- 3 0 1 1 2 2 0 ./soc-LiveJournal1.random.txt 68000000 1000 inspect 116 | Finished release [optimized + debuginfo] target(s) in 0.0 secs 117 | Running `target/release/examples/motif 3 0 1 1 2 2 0 ./soc-LiveJournal1.random.txt 68000000 1000 inspect` 118 | motif: [(0, 1), (1, 2), (2, 0)] 119 | filename: "./soc-LiveJournal1.random.txt" 120 | ... 121 | (Root, 994): [35917] 122 | (Root, 995): [24601] 123 | elapsed: 62.80372881703079 total motifs at this process: 31171871 124 | Echidnatron% 125 | 126 | ### Directed 4-cliques 127 | 128 | Generalizing directed triangles to 4-cliques, where we have four nodes `a`, `b`, `c`, and `d` that each point to those nodes after them, and pre-loading a bit more data so we don't have to wait as long for the result, 129 | 130 | Echidnatron% cargo run --release --example motif -- 6 0 1 0 2 0 3 1 2 1 3 2 3 ./soc-LiveJournal1.random.txt 68900000 1000 inspect 131 | Finished release [optimized + debuginfo] target(s) in 0.0 secs 132 | Running `target/release/examples/motif 6 0 1 0 2 0 3 1 2 1 3 2 3 ./soc-LiveJournal1.random.txt 68900000 1000 inspect` 133 | motif: [(0, 1), (0, 2), (0, 3), (1, 2), (1, 3), (2, 3)] 134 | filename: "./soc-LiveJournal1.random.txt" 135 | index initialized: (4307203, 68900000) 136 | index initialized: (4487812, 68900000) 137 | Duration { secs: 38, nanos: 751447929 } [worker 0] data loaded 138 | Duration { secs: 38, nanos: 751585622 } [worker 0] indices merged 139 | (Root, 2): [6239159] 140 | (Root, 3): [5386443] 141 | ... 142 | (Root, 95): [3938957] 143 | elapsed: 233.18197546800366 total motifs at this process: 550759817 144 | Echidnatron% 145 | 146 | This took longer, but there were also a larger number of the motifs to process. 147 | 148 | ### Directed 6-cliques 149 | 150 | You can imagine where this is going, but I thought I'd do this example because it is something that seems is much too complicated to compute completely and then difference. Here we stream edge updates in one at a time (a batch size of one). 151 | 152 | Echidnatron% cargo run --release --example motif -- 15 0 1 0 2 0 3 0 4 0 5 1 2 1 3 1 4 1 5 2 3 2 4 2 5 3 4 3 5 4 5 soc-LiveJournal1.random.txt 68900000 1 inspect 153 | Finished release [optimized + debuginfo] target(s) in 0.0 secs 154 | Running `target/release/examples/motif 15 0 1 0 2 0 3 0 4 0 5 1 2 1 3 1 4 1 5 2 3 2 4 2 5 3 4 3 5 4 5 ./soc-LiveJournal1.random.txt 68900000 1 inspect` 155 | motif: [(0, 1), (0, 2), (0, 3), (0, 4), (0, 5), (1, 2), (1, 3), (1, 4), (1, 5), (2, 3), (2, 4), (2, 5), (3, 4), (3, 5), (4, 5)] 156 | filename: "./soc-LiveJournal1.random.txt" 157 | index initialized: (4307203, 68900000) 158 | index initialized: (4487812, 68900000) 159 | Duration { secs: 38, nanos: 759079572 } [worker 0] data loaded 160 | Duration { secs: 38, nanos: 760087081 } [worker 0] indices merged 161 | (Root, 2): [473] 162 | (Root, 3): [94] 163 | (Root, 5): [8975] 164 | (Root, 6): [101569] 165 | (Root, 7): [5] 166 | (Root, 8): [1554] 167 | (Root, 9): [128] 168 | (Root, 10): [33446] 169 | (Root, 11): [5339] 170 | Echidnatron% 171 | 172 | This program goes to 11. 173 | 174 | But just 11. What happened? As it turns out, the 12th update causes so many changed 6-cliques (or candidates along the way) that the process wanders up to 60GB on my laptop and then shuts itself down. 175 | 176 | For comparison, here are the output numbers for 5-cliques, where the fatal 12th update produces 48,807,868 updates. Each of these updates are produced along the way in the 6-clique dataflow (which just extends the 5-clique dataflow), and each apparently leads to some large number of further candidates. 177 | 178 | Echidnatron% cargo run --release --example motif -- 10 0 1 0 2 0 3 0 4 1 2 1 3 1 4 2 3 2 4 3 4 ./soc-LiveJournal1.random.txt 68900000 1 inspect 179 | Finished release [optimized + debuginfo] target(s) in 0.0 secs 180 | Running `target/release/examples/motif 10 0 1 0 2 0 3 0 4 1 2 1 3 1 4 2 3 2 4 3 4 ./soc-LiveJournal1.random.txt 68900000 1 inspect` 181 | motif: [(0, 1), (0, 2), (0, 3), (0, 4), (1, 2), (1, 3), (1, 4), (2, 3), (2, 4), (3, 4)] 182 | filename: "./soc-LiveJournal1.random.txt" 183 | index initialized: (4307203, 68900000) 184 | index initialized: (4487812, 68900000) 185 | Duration { secs: 36, nanos: 387479783 } [worker 0] data loaded 186 | Duration { secs: 36, nanos: 387815357 } [worker 0] indices merged 187 | (Root, 2): [131] 188 | (Root, 3): [53] 189 | (Root, 5): [1570] 190 | (Root, 6): [11081] 191 | (Root, 7): [4] 192 | (Root, 8): [574] 193 | (Root, 9): [67] 194 | (Root, 10): [6197] 195 | (Root, 11): [952] 196 | (Root, 12): [48807868] 197 | (Root, 13): [3612] 198 | (Root, 14): [470] 199 | ... 200 | 201 | The problem is that, even though we are streaming in single updates, we try and do all of the count, propose, and intersect work for these 48 million tuples at the same time. What we *should* be doing, in a better world, is stream through the 48 million bits of intermediate work as well. We should stage them so that we don't try and do all of the work at once, but rather retire chunks of updates at a time, keeping our resource use in check. 202 | 203 | This is an exciting open area for us, where the answer lies in [recent scheduling work](https://people.inf.ethz.ch/zchothia/papers/faucet-beyondmr16.pdf) with Andrea Lattuada and Zaheer Chothia that prioritizes operators further down the dataflow graph, aggressively draining the dataflow rather than producing more work. In principle we should be able to try this out and see what happens! -------------------------------------------------------------------------------- /examples/four-cliques.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | use timely::dataflow::operators::capture::Extract; 11 | 12 | use graph_map::GraphMMap; 13 | 14 | #[allow(non_snake_case)] 15 | fn main () { 16 | 17 | let start = ::std::time::Instant::now(); 18 | 19 | let (send, recv) = ::std::sync::mpsc::channel(); 20 | let send = Arc::new(Mutex::new(send)); 21 | 22 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 23 | 24 | timely::execute_from_args(std::env::args(), move |root| { 25 | 26 | let send = send.lock().unwrap().clone(); 27 | 28 | // used to partition graph loading 29 | let index = root.index(); 30 | let peers = root.peers(); 31 | 32 | // handles to input and probe, but also both indices so we can compact them. 33 | let (mut input, probe, forward, reverse) = root.dataflow::(|builder| { 34 | 35 | // Please see triangles for more information on "graph" and dG. 36 | let (graph, dG) = builder.new_input::<((u32, u32), i32)>(); 37 | 38 | // Our query is K4 = Q(a1,a2,a3,a4) = A(a1,a2) B(a1, a3) C(a1,a4) D(a2,a3) E(a2,a4) F(a3,a4) 39 | // 40 | // The dataflow has 6 derivatives, with respect to each relation: 41 | // dQdA := dA x B x C x D x E x F 42 | // dQdB := dB x A x C x D x E x F 43 | // dQdC := dC x A x B x D x E x F 44 | // dQdD := dD x A x B x C x E x F 45 | // dQdE := dE x A x B x C x D x F 46 | // dQdF := dF x A x B x C x D x E 47 | 48 | let forward = IndexStream::from(|k| k as u64, &Vec::new().to_stream(builder), &dG); 49 | let reverse = IndexStream::from(|k| k as u64, &Vec::new().to_stream(builder), &dG.map(|((src,dst),wgt)| ((dst,src),wgt))); 50 | 51 | // We then pick an ordering of attributes for each derivative: 52 | // dQdA: we start with dA(a1, a2) and extend to a3 and then to a4. So there will be 2 extensions: 53 | // (1) Extending (a1, a2) prefixes to (a1, a2, a3) prefixes: a3 is included in B(a1, a3), D(a2, a3), and F(a3, a4). 54 | // However we ignore F for now becase a3 is not bounded by a1 or a2 yet (the two variables bounded by 55 | // dA(a1, a2)). We will use F when we extend to a4 in the 2nd extensions below. 56 | // 1st extension involves two ``extenders'', who will offer proposals and do intersections: 57 | // (i) B(a1, a3) using the forward index because a1 is bounded and is the 1st variable in B(a1, a3). 58 | // We'd pick the reverse index if a1 was the 2nd variable. We will also use a1 as exchange; and 59 | // (ii) D(a2, a3) also using the forward index and now using a2 as the exchange by the same reasons. 60 | // (2) Extending (a1, a2, a3) to (a1, a2, a3, a4): a4 appears in C, E, and F. We will use 3 extenders. 61 | // (i) C(a1, a4): Use forward index and use a1 as exchange; 62 | // (ii) E(a2, a4): Use forward index and use a2 as exchange; 63 | // (ii) F(a3, a4): Use forward index and use a3 as exchange; 64 | let dK4dA1 = dG.extend(vec![Box::new(forward.extend_using(|&(a1,_a2)| a1, <_ as PartialOrd>::lt)), 65 | Box::new(forward.extend_using(|&(_a1,a2)| a2, <_ as PartialOrd>::lt))]) 66 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((p.0,p.1,e), wght))); 67 | let dK4dA = dK4dA1.extend(vec![Box::new(forward.extend_using(|&(a1,_a2,_a3)| a1, <_ as PartialOrd>::lt)), 68 | Box::new(forward.extend_using(|&(_a1,a2,_a3)| a2, <_ as PartialOrd>::lt)), 69 | Box::new(forward.extend_using(|&(_a1,_a2,a3)| a3, <_ as PartialOrd>::lt))]) 70 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((p.0,p.1,p.2,e), wght))); 71 | 72 | // dQdB(a1,a3): Similar to above first extend (a1, a3) to (a1, a2, a3). Then to (a1, a2, a3, a4). 73 | let dK4dB1 = dG.extend(vec![Box::new(forward.extend_using(|&(a1,_a3)| a1, <_ as PartialOrd>::le)), 74 | Box::new(reverse.extend_using(|&(_a1,a3)| a3, <_ as PartialOrd>::lt))]) 75 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((p.0, e, p.1), wght))); 76 | let dK4dB = dK4dB1.extend(vec![Box::new(forward.extend_using(|&(a1,_a2,_a3)| a1, <_ as PartialOrd>::lt)), 77 | Box::new(forward.extend_using(|&(_a1,a2,_a3)| a2, <_ as PartialOrd>::lt)), 78 | Box::new(forward.extend_using(|&(_a1,_a2,a3)| a3, <_ as PartialOrd>::lt))]) 79 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((p.0,p.1,p.2,e), wght))); 80 | 81 | // dQdC(a1,a4): Similar to above first extend (a1, a4) to (a1, a2, a4). Then to (a1, a2, a3, a4). 82 | let dK4dC1 = dG.extend(vec![Box::new(forward.extend_using(|&(a1,_a4)| a1, <_ as PartialOrd>::le)), 83 | Box::new(reverse.extend_using(|&(_a1,a4)| a4, <_ as PartialOrd>::lt))]) 84 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((p.0, e, p.1), wght))); 85 | let dK4dC = dK4dC1.extend(vec![Box::new(forward.extend_using(|&(a1,_a2,_a4)| a1, <_ as PartialOrd>::le)), 86 | Box::new(forward.extend_using(|&(_a1,a2,_a4)| a2, <_ as PartialOrd>::lt)), 87 | Box::new(reverse.extend_using(|&(_a1,_a2,a4)| a4, <_ as PartialOrd>::lt))]) 88 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((p.0,p.1,e, p.2), wght))); 89 | 90 | // dQdD(a2,a3): Similar to above first extend (a2, a3) to (a1, a2, a3). Then to (a1, a2, a3, a4). 91 | let dK4dD1 = dG.extend(vec![Box::new(reverse.extend_using(|&(a2,_a3)| a2, <_ as PartialOrd>::le)), 92 | Box::new(reverse.extend_using(|&(_a2,a3)| a3, <_ as PartialOrd>::le))]) 93 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((e, p.0, p.1), wght))); 94 | let dK4dD = dK4dD1.extend(vec![Box::new(forward.extend_using(|&(a1,_a2,_a3)| a1, <_ as PartialOrd>::le)), 95 | Box::new(forward.extend_using(|&(_a1,a2,_a3)| a2, <_ as PartialOrd>::lt)), 96 | Box::new(forward.extend_using(|&(_a1,_a2,a3)| a3, <_ as PartialOrd>::lt))]) 97 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((p.0,p.1,p.2,e), wght))); 98 | 99 | // dQdE(a2,a4): Similar to above first extend (a2, a4) to (a1, a2, a4). Then to (a1, a2, a3, a4). 100 | let dK4dE1 = dG.extend(vec![Box::new(reverse.extend_using(|&(a2,_a4)| a2, <_ as PartialOrd>::le)), 101 | Box::new(reverse.extend_using(|&(_a2,a4)| a4, <_ as PartialOrd>::le))]) 102 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((e, p.0, p.1), wght))); 103 | let dK4dE = dK4dE1.extend(vec![Box::new(forward.extend_using(|&(a1,_a2,_a4)| a1, <_ as PartialOrd>::le)), 104 | Box::new(forward.extend_using(|&(_a1,a2,_a4)| a2, <_ as PartialOrd>::le)), 105 | Box::new(reverse.extend_using(|&(_a1,_a2,a4)| a4, <_ as PartialOrd>::lt))]) 106 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((p.0,p.1, e, p.2), wght))); 107 | 108 | // dQdF(a3,a4): Similar to above first extend (a3, a4) to (a1, a3, a4). Then to (a1, a2, a3, a4). 109 | let dK4dF1 = dG.extend(vec![Box::new(reverse.extend_using(|&(a3,_a4)| a3, <_ as PartialOrd>::le)), 110 | Box::new(reverse.extend_using(|&(_a3,a4)| a4, <_ as PartialOrd>::le))]) 111 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((e, p.0, p.1), wght))); 112 | let dK4dF = dK4dF1.extend(vec![Box::new(forward.extend_using(|&(a1,_a3,_a4)| a1, <_ as PartialOrd>::le)), 113 | Box::new(reverse.extend_using(|&(_a1,a3,_a4)| a3, <_ as PartialOrd>::le)), 114 | Box::new(reverse.extend_using(|&(_a1,_a3,a4)| a4, <_ as PartialOrd>::le))]) 115 | .flat_map(|(p,es,wght)| es.into_iter().map(move |e| ((p.0, e, p.1, p.2), wght))); 116 | 117 | 118 | // accumulate all changes together into a single dataflow. 119 | let cliques = dK4dF.concat(&dK4dE).concat(&dK4dD).concat(&dK4dC).concat(&dK4dB).concat(&dK4dA); 120 | 121 | // if the third argument is "inspect", report 4-clique counts. 122 | if inspect { 123 | cliques.exchange(|x| (x.0).0 as u64) 124 | // .inspect_batch(|t,x| println!("{:?}: {:?}", t, x)) 125 | .count() 126 | .inspect_batch(|t,x| println!("{:?}: {:?}", t, x)) 127 | .capture_into(send); 128 | } 129 | (graph, cliques.probe(), forward, reverse) 130 | }); 131 | 132 | // load fragment of input graph into memory to avoid io while running. 133 | let filename = std::env::args().nth(1).unwrap(); 134 | let graph = GraphMMap::new(&filename); 135 | 136 | let nodes = graph.nodes(); 137 | let mut edges = Vec::new(); 138 | 139 | for node in 0 .. graph.nodes() { 140 | if node % peers == index { 141 | edges.push(graph.edges(node).to_vec()); 142 | } 143 | } 144 | 145 | drop(graph); 146 | 147 | // synchronize with other workers. 148 | let prev = input.time().clone(); 149 | input.advance_to(prev.inner + 1); 150 | root.step_while(|| probe.less_than(input.time())); 151 | 152 | // number of nodes introduced at a time 153 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 154 | 155 | // start the experiment! 156 | let start = ::std::time::Instant::now(); 157 | for node in 0 .. nodes { 158 | 159 | // introduce the node if it is this worker's responsibility 160 | if node % peers == index { 161 | for &edge in &edges[node / peers] { 162 | input.send(((node as u32, edge), 1)); 163 | } 164 | } 165 | 166 | // if at a batch boundary, advance time and do work. 167 | if node % batch == (batch - 1) { 168 | let prev = input.time().clone(); 169 | input.advance_to(prev.inner + 1); 170 | root.step_while(|| probe.less_than(input.time())); 171 | 172 | // merge all of the indices we maintain. 173 | forward.index.borrow_mut().merge_to(&prev); 174 | reverse.index.borrow_mut().merge_to(&prev); 175 | } 176 | } 177 | 178 | input.close(); 179 | while root.step() { } 180 | 181 | if inspect { 182 | println!("worker {} elapsed: {:?}", index, start.elapsed()); 183 | } 184 | 185 | }).unwrap(); 186 | 187 | let result = recv.extract(); 188 | 189 | let mut total = 0; 190 | for &(_, ref counts) in &result { 191 | for &count in counts { 192 | total += count; 193 | } 194 | } 195 | 196 | if inspect { 197 | println!("elapsed: {:?}\ttotal 4-cliques at this process: {:?}", start.elapsed(), total); 198 | } 199 | } 200 | -------------------------------------------------------------------------------- /examples/motif-careful.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate alg3_dynamic; 3 | 4 | use std::sync::{Arc, Mutex}; 5 | use std::io::BufReader; 6 | use std::error::Error; 7 | use std::fs::File; 8 | use std::io::prelude::*; 9 | use std::path::Path; 10 | 11 | use timely::dataflow::operators::*; 12 | 13 | use alg3_dynamic::*; 14 | 15 | type Node = u32; 16 | 17 | fn main () { 18 | 19 | let start = ::std::time::Instant::now(); 20 | 21 | let send = Arc::new(Mutex::new(0)); 22 | let send2 = send.clone(); 23 | 24 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 25 | 26 | timely::execute_from_args(std::env::args(), move |root| { 27 | 28 | let send = send.clone(); 29 | 30 | // used to partition graph loading 31 | let index = root.index(); 32 | let peers = root.peers(); 33 | 34 | let mut motif = vec![]; 35 | let query_size: usize = std::env::args().nth(1).unwrap().parse().unwrap(); 36 | for query in 0 .. query_size { 37 | let attr1: usize = std::env::args().nth(2 * (query + 1) + 0).unwrap().parse().unwrap(); 38 | let attr2: usize = std::env::args().nth(2 * (query + 1) + 1).unwrap().parse().unwrap(); 39 | motif.push((attr1, attr2)); 40 | } 41 | 42 | // load fragment of input graph into memory to avoid io while running. 43 | let filename = std::env::args().nth(2 * (query_size) + 2).unwrap(); 44 | let pre_load = std::env::args().nth(2 * (query_size) + 3).unwrap().parse().unwrap(); 45 | let query_batch: usize = std::env::args().nth(2 * (query_size) + 4).unwrap().parse().unwrap(); 46 | 47 | println!("motif:\t{:?}", motif); 48 | println!("filename:\t{:?}", filename); 49 | 50 | // handles to input and probe, but also both indices so we can compact them. 51 | let (mut input_graph1, mut input_graph2, mut input_delta, probe, load_probe1, load_probe2, handles) = root.dataflow::(move |builder| { 52 | 53 | // inputs for initial edges and changes to the edge set, respectively. 54 | let (graph_input1, graph1) = builder.new_input::<(Node, Node)>(); 55 | let (graph_input2, graph2) = builder.new_input::<(Node, Node)>(); 56 | let (delta_input, delta) = builder.new_input::<((Node, Node), i32)>(); 57 | 58 | // create indices and handles from the initial edges plus updates. 59 | let (graph_index, handles) = motif::GraphStreamIndex::from_separately(graph1, graph2, delta, |k| k as u64, |k| k as u64); 60 | 61 | // construct the motif dataflow subgraph. 62 | let motifs = graph_index.track_motif(&motif); 63 | 64 | // if "inspect", report motif counts. 65 | if inspect { 66 | motifs 67 | .count() 68 | .inspect_batch(|t,x| println!("{:?}: {:?}", t, x)) 69 | .inspect_batch(move |_,x| { 70 | if let Ok(mut bound) = send.lock() { 71 | *bound += x[0]; 72 | } 73 | }); 74 | } 75 | 76 | let load_probe1 = graph_index.forward.handle.clone(); 77 | let load_probe2 = graph_index.reverse.handle.clone(); 78 | 79 | (graph_input1, graph_input2, delta_input, motifs.probe(), load_probe1, load_probe2, handles) 80 | }); 81 | 82 | // start the experiment! 83 | let start = ::std::time::Instant::now(); 84 | 85 | 86 | let prev_time = input_delta.time().clone(); 87 | input_delta.advance_to(prev_time.inner + 1); 88 | 89 | 90 | // Open the path in read-only mode, returns `io::Result` 91 | let mut lines = match File::open(&Path::new(&filename)) { 92 | Ok(file) => BufReader::new(file).lines(), 93 | Err(why) => { 94 | panic!("EXCEPTION: couldn't open {}: {}", 95 | Path::new(&filename).display(), 96 | Error::description(&why)) 97 | }, 98 | }; 99 | 100 | // load up the graph, using the first `limit` lines in the file. 101 | for (counter, line) in lines.by_ref().take(pre_load).enumerate() { 102 | // each worker is responsible for a fraction of the queries 103 | if counter % peers == index { 104 | let good_line = line.ok().expect("EXCEPTION: read error"); 105 | if !good_line.starts_with('#') && good_line.len() > 0 { 106 | let mut elements = good_line[..].split_whitespace(); 107 | let src: Node = elements.next().unwrap().parse().ok().expect("malformed src"); 108 | let dst: Node = elements.next().unwrap().parse().ok().expect("malformed dst"); 109 | input_graph1.send((src, dst)); 110 | } 111 | } 112 | } 113 | 114 | // synchronize with other workers before reporting data loaded. 115 | input_graph1.close(); 116 | root.step_while(|| load_probe1.less_than(input_delta.time())); 117 | println!("{:?}\t[worker {}]\tforward index loaded", start.elapsed(), index); 118 | 119 | 120 | 121 | // REPEAT ABOVE 122 | 123 | 124 | // Open the path in read-only mode, returns `io::Result` 125 | let mut lines = match File::open(&Path::new(&filename)) { 126 | Ok(file) => BufReader::new(file).lines(), 127 | Err(why) => { 128 | panic!("EXCEPTION: couldn't open {}: {}", 129 | Path::new(&filename).display(), 130 | Error::description(&why)) 131 | }, 132 | }; 133 | 134 | // load up the graph, using the first `limit` lines in the file. 135 | for (counter, line) in lines.by_ref().take(pre_load).enumerate() { 136 | // each worker is responsible for a fraction of the queries 137 | if counter % peers == index { 138 | let good_line = line.ok().expect("EXCEPTION: read error"); 139 | if !good_line.starts_with('#') && good_line.len() > 0 { 140 | let mut elements = good_line[..].split_whitespace(); 141 | let src: Node = elements.next().unwrap().parse().ok().expect("malformed src"); 142 | let dst: Node = elements.next().unwrap().parse().ok().expect("malformed dst"); 143 | input_graph2.send((src, dst)); 144 | } 145 | } 146 | } 147 | 148 | // synchronize with other workers before reporting data loaded. 149 | input_graph2.close(); 150 | root.step_while(|| load_probe2.less_than(input_delta.time())); 151 | println!("{:?}\t[worker {}]\treverse index loaded", start.elapsed(), index); 152 | 153 | 154 | // END REPEAT 155 | 156 | 157 | 158 | 159 | 160 | 161 | // loop { } 162 | 163 | // merge all of the indices the worker maintains. 164 | let prev_time = input_delta.time().clone(); 165 | handles.merge_to(&prev_time); 166 | 167 | // synchronize with other workers before reporting indices merged. 168 | let prev_time = input_delta.time().clone(); 169 | // input_graph.advance_to(prev_time.inner + 1); 170 | input_delta.advance_to(prev_time.inner + 1); 171 | root.step_while(|| probe.less_than(input_delta.time())); 172 | println!("{:?}\t[worker {}]\tindices merged", start.elapsed(), index); 173 | 174 | // issue queries and updates, using the remaining lines in the file. 175 | for (query_counter, line) in lines.enumerate() { 176 | 177 | // each worker is responsible for a fraction of the queries 178 | if query_counter % peers == index { 179 | let good_line = line.ok().expect("EXCEPTION: read error"); 180 | if !good_line.starts_with('#') && good_line.len() > 0 { 181 | let mut elements = good_line[..].split_whitespace(); 182 | let src: Node = elements.next().unwrap().parse().ok().expect("malformed src"); 183 | let dst: Node = elements.next().unwrap().parse().ok().expect("malformed dst"); 184 | input_delta.send(((src, dst), 1)); 185 | } 186 | } 187 | 188 | // synchronize and merge indices. 189 | if query_counter % query_batch == (query_batch - 1) { 190 | let prev_time = input_delta.time().clone(); 191 | // input_graph.advance_to(prev_time.inner + 1); 192 | input_delta.advance_to(prev_time.inner + 1); 193 | root.step_while(|| probe.less_than(input_delta.time())); 194 | handles.merge_to(&prev_time); 195 | } 196 | } 197 | }).unwrap(); 198 | 199 | let total = send2.lock().map(|x| *x).unwrap_or(0); 200 | println!("elapsed: {:?}\ttotal motifs at this process: {:?}", start.elapsed(), total); 201 | } -------------------------------------------------------------------------------- /examples/motif.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate alg3_dynamic; 3 | 4 | use std::sync::{Arc, Mutex}; 5 | use std::io::BufReader; 6 | use std::error::Error; 7 | use std::fs::File; 8 | use std::io::prelude::*; 9 | use std::path::Path; 10 | 11 | use timely::dataflow::operators::*; 12 | 13 | use alg3_dynamic::*; 14 | 15 | type Node = u32; 16 | 17 | fn main () { 18 | 19 | let start = ::std::time::Instant::now(); 20 | 21 | let send = Arc::new(Mutex::new(0)); 22 | let send2 = send.clone(); 23 | 24 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 25 | 26 | timely::execute_from_args(std::env::args(), move |root| { 27 | 28 | let send = send.clone(); 29 | 30 | // used to partition graph loading 31 | let index = root.index(); 32 | let peers = root.peers(); 33 | 34 | let mut motif = vec![]; 35 | let query_size: usize = std::env::args().nth(1).unwrap().parse().unwrap(); 36 | for query in 0 .. query_size { 37 | let attr1: usize = std::env::args().nth(2 * (query + 1) + 0).unwrap().parse().unwrap(); 38 | let attr2: usize = std::env::args().nth(2 * (query + 1) + 1).unwrap().parse().unwrap(); 39 | motif.push((attr1, attr2)); 40 | } 41 | 42 | // load fragment of input graph into memory to avoid io while running. 43 | let filename = std::env::args().nth(2 * (query_size) + 2).unwrap(); 44 | let pre_load = std::env::args().nth(2 * (query_size) + 3).unwrap().parse().unwrap(); 45 | let query_batch: usize = std::env::args().nth(2 * (query_size) + 4).unwrap().parse().unwrap(); 46 | 47 | println!("motif:\t{:?}", motif); 48 | println!("filename:\t{:?}", filename); 49 | 50 | // handles to input and probe, but also both indices so we can compact them. 51 | let (mut input_graph, mut input_delta, probe, handles) = root.dataflow::(move |builder| { 52 | 53 | // inputs for initial edges and changes to the edge set, respectively. 54 | let (graph_input, graph) = builder.new_input::<(Node, Node)>(); 55 | let (delta_input, delta) = builder.new_input::<((Node, Node), i32)>(); 56 | 57 | // create indices and handles from the initial edges plus updates. 58 | let (graph_index, handles) = motif::GraphStreamIndex::from(graph, delta, |k| k as u64, |k| k as u64); 59 | 60 | // construct the motif dataflow subgraph. 61 | let motifs = graph_index.track_motif(&motif); 62 | 63 | // if "inspect", report motif counts. 64 | if inspect { 65 | motifs 66 | .count() 67 | .inspect_batch(|t,x| println!("{:?}: {:?}", t, x)) 68 | .inspect_batch(move |_,x| { 69 | if let Ok(mut bound) = send.lock() { 70 | *bound += x[0]; 71 | } 72 | }); 73 | } 74 | 75 | (graph_input, delta_input, motifs.probe(), handles) 76 | }); 77 | 78 | // start the experiment! 79 | let start = ::std::time::Instant::now(); 80 | 81 | // Open the path in read-only mode, returns `io::Result` 82 | let mut lines = match File::open(&Path::new(&filename)) { 83 | Ok(file) => BufReader::new(file).lines(), 84 | Err(why) => { 85 | panic!("EXCEPTION: couldn't open {}: {}", 86 | Path::new(&filename).display(), 87 | Error::description(&why)) 88 | }, 89 | }; 90 | 91 | // load up the graph, using the first `limit` lines in the file. 92 | for (counter, line) in lines.by_ref().take(pre_load).enumerate() { 93 | // each worker is responsible for a fraction of the queries 94 | if counter % peers == index { 95 | let good_line = line.ok().expect("EXCEPTION: read error"); 96 | if !good_line.starts_with('#') && good_line.len() > 0 { 97 | let mut elements = good_line[..].split_whitespace(); 98 | let src: Node = elements.next().unwrap().parse().ok().expect("malformed src"); 99 | let dst: Node = elements.next().unwrap().parse().ok().expect("malformed dst"); 100 | input_graph.send((src, dst)); 101 | } 102 | } 103 | } 104 | 105 | // synchronize with other workers before reporting data loaded. 106 | let prev_time = input_graph.time().clone(); 107 | input_graph.advance_to(prev_time.inner + 1); 108 | input_delta.advance_to(prev_time.inner + 1); 109 | root.step_while(|| probe.less_than(input_graph.time())); 110 | println!("{:?}\t[worker {}]\tdata loaded", start.elapsed(), index); 111 | 112 | // loop { } 113 | 114 | // merge all of the indices the worker maintains. 115 | let prev_time = input_graph.time().clone(); 116 | handles.merge_to(&prev_time); 117 | 118 | // synchronize with other workers before reporting indices merged. 119 | let prev_time = input_graph.time().clone(); 120 | input_graph.advance_to(prev_time.inner + 1); 121 | input_delta.advance_to(prev_time.inner + 1); 122 | root.step_while(|| probe.less_than(input_graph.time())); 123 | println!("{:?}\t[worker {}]\tindices merged", start.elapsed(), index); 124 | 125 | // issue queries and updates, using the remaining lines in the file. 126 | for (query_counter, line) in lines.enumerate() { 127 | 128 | // each worker is responsible for a fraction of the queries 129 | if query_counter % peers == index { 130 | let good_line = line.ok().expect("EXCEPTION: read error"); 131 | if !good_line.starts_with('#') && good_line.len() > 0 { 132 | let mut elements = good_line[..].split_whitespace(); 133 | let src: Node = elements.next().unwrap().parse().ok().expect("malformed src"); 134 | let dst: Node = elements.next().unwrap().parse().ok().expect("malformed dst"); 135 | input_delta.send(((src, dst), 1)); 136 | } 137 | } 138 | 139 | // synchronize and merge indices. 140 | if query_counter % query_batch == (query_batch - 1) { 141 | let prev_time = input_graph.time().clone(); 142 | input_graph.advance_to(prev_time.inner + 1); 143 | input_delta.advance_to(prev_time.inner + 1); 144 | root.step_while(|| probe.less_than(input_delta.time())); 145 | handles.merge_to(&prev_time); 146 | } 147 | } 148 | }).unwrap(); 149 | 150 | let total = send2.lock().map(|x| *x).unwrap_or(0); 151 | println!("elapsed: {:?}\ttotal motifs at this process: {:?}", start.elapsed(), total); 152 | } -------------------------------------------------------------------------------- /examples/q3-lean.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0usize)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // used to partition graph loading 28 | let index = root.index(); 29 | let peers = root.peers(); 30 | 31 | // handles to input and probe, but also both indices so we can compact them. 32 | let (mut input, mut query, probe, /*forward,*/ reverse) = root.dataflow::(|builder| { 33 | 34 | // Please see triangles for more information on "graph" and dG. 35 | let (graph, dG) = builder.new_input::<((u32, u32), i32)>(); 36 | // Please see triangles for more information on "graph" and dG. 37 | let (query, dQ) = builder.new_input::<((u32, u32), ())>(); 38 | 39 | // Our query is K3 = A(x,y) B(x,z) C(y,z): triangles. 40 | // 41 | // The dataflow determines how to update this query with respect to changes in each 42 | // of the input relations: A, B, and C. Each partial derivative will use the other 43 | // relations, but the order in which attributes are added may (will) be different. 44 | // 45 | // The updates also use the other relations with slightly stale data: updates to each 46 | // relation must not see updates for "later" relations (under some order on relations). 47 | 48 | // we will index the data both by src and dst. 49 | // let (forward, f_handle) = dG.index_from(&dG.filter(|_| false).map(|_| (0,0))); 50 | let forward = IndexStream::from(|k| k as u64, 51 | &dG.map(|((x,y),_)| (x,y)), 52 | &Vec::new().to_stream(builder)); 53 | 54 | // let r_handle = reverse.index.clone() 55 | 56 | // let (reverse, r_handle) = dG.filter(|_|false) 57 | // .map(|((src,dst),wgt)| ((dst,src),wgt)) 58 | // .index_from(&dG.map(|((x,y),_)| (y,x)), |&k| k as u64); 59 | 60 | // dC(y,z) extends to x first through A(x,y) then B(x,z), both using reverse indices. 61 | let cliques = dQ.extend(vec![Box::new(forward.extend_using(|&(v1,_)| v1, |t1, t2| t1.le(t2))), 62 | Box::new(forward.extend_using(|&(_,v2)| v2, |t1, t2| t1.le(t2)))]) 63 | .flat_map(|((v1,v2),v3s,w)| v3s.into_iter().map(move |v3| ((v1,v2,v3),w))) 64 | .extend(vec![Box::new(forward.extend_using(|&(v1,_,_)| v1, |t1, t2| t1.le(t2))), 65 | Box::new(forward.extend_using(|&(_,v2,_)| v2, |t1, t2| t1.le(t2))), 66 | Box::new(forward.extend_using(|&(_,_,v3)| v3, |t1, t2| t1.le(t2)))]) 67 | ; 68 | 69 | // if the third argument is "inspect", report triangle counts. 70 | if inspect { 71 | cliques 72 | .inspect_batch(move |_,x| { 73 | let sum = x.iter().map(|xx| xx.1.len()).sum::(); 74 | if let Ok(mut bound) = send.lock() { 75 | *bound += sum; 76 | } 77 | }); 78 | } 79 | 80 | (graph, query, cliques.probe(), forward) 81 | }); 82 | 83 | // load fragment of input graph into memory to avoid io while running. 84 | let filename = std::env::args().nth(1).unwrap(); 85 | let graph = GraphMMap::new(&filename); 86 | 87 | let nodes = graph.nodes(); 88 | let mut edges = Vec::new(); 89 | 90 | for node in 0 .. graph.nodes() { 91 | if node % peers == index { 92 | edges.push(graph.edges(node).to_vec()); 93 | } 94 | } 95 | 96 | drop(graph); 97 | 98 | // synchronize with other workers. 99 | let prev = input.time().clone(); 100 | input.advance_to(prev.inner + 1); 101 | query.advance_to(prev.inner + 1); 102 | root.step_while(|| probe.less_than(input.time())); 103 | 104 | // number of nodes introduced at a time 105 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 106 | 107 | // start the experiment! 108 | let start = ::std::time::Instant::now(); 109 | 110 | for node in 0 .. nodes { 111 | 112 | // introduce the node if it is this worker's responsibility 113 | if node % peers == index { 114 | for &edge in &edges[node / peers] { 115 | input.send(((node as u32, edge), 1)); 116 | } 117 | } 118 | } 119 | 120 | let prev = input.time().clone(); 121 | input.advance_to(prev.inner + 1); 122 | query.advance_to(prev.inner + 1); 123 | root.step_while(|| probe.less_than(query.time())); 124 | reverse.index.borrow_mut().merge_to(&prev); 125 | input.close(); 126 | 127 | println!("{:?}: index built", start.elapsed()); 128 | 129 | for node in 0 .. nodes { 130 | 131 | // introduce the node if it is this worker's responsibility 132 | if node % peers == index { 133 | for &edge in &edges[node / peers] { 134 | query.send(((node as u32, edge), ())); 135 | } 136 | } 137 | 138 | // if at a batch boundary, advance time and do work. 139 | if node % batch == (batch - 1) { 140 | let prev = query.time().clone(); 141 | query.advance_to(prev.inner + 1); 142 | root.step_while(|| probe.less_than(query.time())); 143 | 144 | // merge all of the indices we maintain. 145 | // forward.borrow_mut().merge_to(&prev); 146 | reverse.index.borrow_mut().merge_to(&prev); 147 | } 148 | } 149 | 150 | query.close(); 151 | while root.step() { } 152 | 153 | if inspect { 154 | println!("worker {} elapsed: {:?}", index, start.elapsed()); 155 | } 156 | 157 | }).unwrap(); 158 | 159 | let total = if let Ok(lock) = send2.lock() { 160 | *lock 161 | } 162 | else { 0 }; 163 | 164 | if inspect { 165 | println!("elapsed: {:?}\ttotal triangles at this process: {:?}", start.elapsed(), total); 166 | } 167 | } 168 | -------------------------------------------------------------------------------- /examples/q6-lean-alt.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0usize)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // used to partition graph loading 28 | let index = root.index(); 29 | let peers = root.peers(); 30 | 31 | // handles to input and probe, but also both indices so we can compact them. 32 | let (mut input, mut query, probe, forward) = root.dataflow::(|builder| { 33 | 34 | // Please see triangles for more information on "graph" and dG. 35 | let (graph, dG) = builder.new_input::<((u32, u32), i32)>(); 36 | // Please see triangles for more information on "graph" and dG. 37 | let (query, dQ) = builder.new_input::<((u32, u32), ())>(); 38 | 39 | // Our query is K3 = A(x,y) B(x,z) C(y,z): triangles. 40 | // 41 | // The dataflow determines how to update this query with respect to changes in each 42 | // of the input relations: A, B, and C. Each partial derivative will use the other 43 | // relations, but the order in which attributes are added may (will) be different. 44 | // 45 | // The updates also use the other relations with slightly stale data: updates to each 46 | // relation must not see updates for "later" relations (under some order on relations). 47 | 48 | // we will index the data both by src and dst. 49 | // let (forward, f_handle) = dG.index_from(&dG.filter(|_| false).map(|_| (0,0))); 50 | let forward = IndexStream::from(|k| k as u64, 51 | &dG.map(|((x,y),_)| (x,y)), 52 | &Vec::new().to_stream(builder)); 53 | 54 | // dC(y,z) extends to x first through A(x,y) then B(x,z), both using reverse indices. 55 | let cliques = dQ.extend(vec![Box::new(forward.extend_using(|&(v2,_)| v2, |t1, t2| t1.le(t2))), 56 | Box::new(forward.extend_using(|&(_,v5)| v5, |t1, t2| t1.le(t2)))]) 57 | .flat_map(|((v2,v5),v3s,w)| { 58 | let v1s = v3s.clone(); 59 | v3s.into_iter().map(move |v3| ((v1s.clone(),v2,v3,v5),w)) 60 | }) 61 | .extend(vec![Box::new(forward.extend_using(|&(_,v2,_,_)| v2, |t1, t2| t1.le(t2))), 62 | Box::new(forward.extend_using(|&(_,_,v3,_)| v3, |t1, t2| t1.le(t2))), 63 | Box::new(forward.extend_using(|&(_,_,_,v5)| v5, |t1, t2| t1.le(t2)))]) 64 | .map(|((v1s,v2,v3,v5), mut v4s, w)| { 65 | v4s.retain(|&v4| v2 != v4 && v3 < v4); 66 | ((v1s,v2,v3,v4s,v5),w) 67 | }) ; 68 | 69 | // if the third argument is "inspect", report triangle counts. 70 | if inspect { 71 | cliques 72 | .inspect_batch(move |_,x| { 73 | let mut sum = 0; 74 | for &((ref v1s, _v2, v3, ref v4s, _v5),_) in x.iter() { 75 | for &v1 in v1s.iter() { 76 | if v1 != v3 { 77 | for &v4 in v4s.iter() { 78 | if v1 != v4 { 79 | sum += 1; 80 | } 81 | } 82 | } 83 | } 84 | } 85 | 86 | if let Ok(mut bound) = send.lock() { 87 | *bound += sum; 88 | } 89 | }); 90 | } 91 | 92 | (graph, query, cliques.probe(), forward) 93 | }); 94 | 95 | // load fragment of input graph into memory to avoid io while running. 96 | let filename = std::env::args().nth(1).unwrap(); 97 | let graph = GraphMMap::new(&filename); 98 | 99 | let nodes = graph.nodes(); 100 | let mut edges = Vec::new(); 101 | 102 | for node in 0 .. graph.nodes() { 103 | if node % peers == index { 104 | edges.push(graph.edges(node).to_vec()); 105 | } 106 | } 107 | 108 | drop(graph); 109 | 110 | // synchronize with other workers. 111 | let prev = input.time().clone(); 112 | input.advance_to(prev.inner + 1); 113 | query.advance_to(prev.inner + 1); 114 | root.step_while(|| probe.less_than(input.time())); 115 | 116 | // number of nodes introduced at a time 117 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 118 | 119 | // start the experiment! 120 | let start = ::std::time::Instant::now(); 121 | 122 | for node in 0 .. nodes { 123 | 124 | // introduce the node if it is this worker's responsibility 125 | if node % peers == index { 126 | for &edge in &edges[node / peers] { 127 | input.send(((node as u32, edge), 1)); 128 | input.send(((edge, node as u32), 1)); 129 | } 130 | } 131 | } 132 | 133 | let prev = input.time().clone(); 134 | input.advance_to(prev.inner + 1); 135 | query.advance_to(prev.inner + 1); 136 | root.step_while(|| probe.less_than(query.time())); 137 | forward.index.borrow_mut().merge_to(&prev); 138 | input.close(); 139 | 140 | println!("{:?}: index built", start.elapsed()); 141 | 142 | for node in 0 .. nodes { 143 | 144 | // introduce the node if it is this worker's responsibility 145 | if node % peers == index { 146 | for &edge in &edges[node / peers] { 147 | query.send(((node as u32, edge), ())); 148 | } 149 | } 150 | 151 | // if at a batch boundary, advance time and do work. 152 | if node % batch == (batch - 1) { 153 | let prev = query.time().clone(); 154 | query.advance_to(prev.inner + 1); 155 | root.step_while(|| probe.less_than(query.time())); 156 | 157 | // merge all of the indices we maintain. 158 | forward.index.borrow_mut().merge_to(&prev); 159 | } 160 | } 161 | 162 | query.close(); 163 | while root.step() { } 164 | 165 | if inspect { 166 | println!("worker {} elapsed: {:?}", index, start.elapsed()); 167 | } 168 | 169 | }).unwrap(); 170 | 171 | let total = if let Ok(lock) = send2.lock() { 172 | *lock 173 | } 174 | else { 0 }; 175 | 176 | if inspect { 177 | println!("elapsed: {:?}\ttotal triangles at this process: {:?}", start.elapsed(), total); 178 | } 179 | } 180 | -------------------------------------------------------------------------------- /examples/q6-lean.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0usize)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // used to partition graph loading 28 | let index = root.index(); 29 | let peers = root.peers(); 30 | 31 | // handles to input and probe, but also both indices so we can compact them. 32 | let (mut input, mut query, probe, forward) = root.dataflow::(|builder| { 33 | 34 | // Please see triangles for more information on "graph" and dG. 35 | let (graph, dG) = builder.new_input::<((u32, u32), i32)>(); 36 | // Please see triangles for more information on "graph" and dG. 37 | let (query, dQ) = builder.new_input::<((u32, u32), ())>(); 38 | 39 | // Our query is K3 = A(x,y) B(x,z) C(y,z): triangles. 40 | // 41 | // The dataflow determines how to update this query with respect to changes in each 42 | // of the input relations: A, B, and C. Each partial derivative will use the other 43 | // relations, but the order in which attributes are added may (will) be different. 44 | // 45 | // The updates also use the other relations with slightly stale data: updates to each 46 | // relation must not see updates for "later" relations (under some order on relations). 47 | 48 | // we will index the data both by src and dst. 49 | // let (forward, f_handle) = dG.index_from(&dG.filter(|_| false).map(|_| (0,0))); 50 | let forward = IndexStream::from(|k| k as u64, 51 | &dG.map(|((x,y),_)| (x,y)), 52 | &Vec::new().to_stream(builder)); 53 | 54 | // dC(y,z) extends to x first through A(x,y) then B(x,z), both using reverse indices. 55 | let cliques = dQ.extend(vec![Box::new(forward.extend_using(|&(v2,_)| v2, |t1, t2| t1.le(t2))), 56 | Box::new(forward.extend_using(|&(_,v5)| v5, |t1, t2| t1.le(t2)))]) 57 | .flat_map(|((v2,v5),v3s,w)| v3s.into_iter().map(move |v3| ((v2,v3,v5),w))) 58 | .extend(vec![Box::new(forward.extend_using(|&(v2,_,_)| v2, |t1, t2| t1.le(t2))), 59 | Box::new(forward.extend_using(|&(_,v3,_)| v3, |t1, t2| t1.le(t2))), 60 | Box::new(forward.extend_using(|&(_,_,v5)| v5, |t1, t2| t1.le(t2)))]) 61 | .map(|((v2,v3,v5), mut v4s, w)| { 62 | v4s.retain(|&v4| v2 != v4 && v3 < v4); 63 | ((v2,v3,v4s,v5),w) 64 | }) 65 | .extend(vec![Box::new(forward.extend_using(|&(v1,_,_,_)| v1, |t1, t2| t1.le(t2))), 66 | Box::new(forward.extend_using(|&(_,_,_,v5)| v5, |t1, t2| t1.le(t2)))]) 67 | .map(|((v2,v3,v4s,v5), v1s, w)| ((v1s,v2,v3,v4s,v5),w)) 68 | ; 69 | 70 | // if the third argument is "inspect", report triangle counts. 71 | if inspect { 72 | cliques 73 | .inspect_batch(move |_,x| { 74 | let mut sum = 0; 75 | for &((ref v1s, _v2, v3, ref v4s, _v5),_) in x.iter() { 76 | for &v1 in v1s.iter() { 77 | if v1 != v3 { 78 | for &v4 in v4s.iter() { 79 | if v1 != v4 { 80 | sum += 1; 81 | } 82 | } 83 | } 84 | } 85 | } 86 | 87 | if let Ok(mut bound) = send.lock() { 88 | *bound += sum; 89 | } 90 | }); 91 | } 92 | 93 | (graph, query, cliques.probe(), forward) 94 | }); 95 | 96 | // load fragment of input graph into memory to avoid io while running. 97 | let filename = std::env::args().nth(1).unwrap(); 98 | let graph = GraphMMap::new(&filename); 99 | 100 | let nodes = graph.nodes(); 101 | let mut edges = Vec::new(); 102 | 103 | for node in 0 .. graph.nodes() { 104 | if node % peers == index { 105 | edges.push(graph.edges(node).to_vec()); 106 | } 107 | } 108 | 109 | drop(graph); 110 | 111 | // synchronize with other workers. 112 | let prev = input.time().clone(); 113 | input.advance_to(prev.inner + 1); 114 | query.advance_to(prev.inner + 1); 115 | root.step_while(|| probe.less_than(input.time())); 116 | 117 | // number of nodes introduced at a time 118 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 119 | 120 | // start the experiment! 121 | let start = ::std::time::Instant::now(); 122 | 123 | for node in 0 .. nodes { 124 | 125 | // introduce the node if it is this worker's responsibility 126 | if node % peers == index { 127 | for &edge in &edges[node / peers] { 128 | input.send(((node as u32, edge), 1)); 129 | input.send(((edge, node as u32), 1)); 130 | } 131 | } 132 | } 133 | 134 | let prev = input.time().clone(); 135 | input.advance_to(prev.inner + 1); 136 | query.advance_to(prev.inner + 1); 137 | root.step_while(|| probe.less_than(query.time())); 138 | forward.index.borrow_mut().merge_to(&prev); 139 | input.close(); 140 | 141 | println!("{:?}: index built", start.elapsed()); 142 | 143 | for node in 0 .. nodes { 144 | 145 | // introduce the node if it is this worker's responsibility 146 | if node % peers == index { 147 | for &edge in &edges[node / peers] { 148 | query.send(((node as u32, edge), ())); 149 | } 150 | } 151 | 152 | // if at a batch boundary, advance time and do work. 153 | if node % batch == (batch - 1) { 154 | let prev = query.time().clone(); 155 | query.advance_to(prev.inner + 1); 156 | root.step_while(|| probe.less_than(query.time())); 157 | 158 | // merge all of the indices we maintain. 159 | forward.index.borrow_mut().merge_to(&prev); 160 | } 161 | } 162 | 163 | query.close(); 164 | while root.step() { } 165 | 166 | if inspect { 167 | println!("worker {} elapsed: {:?}", index, start.elapsed()); 168 | } 169 | 170 | }).unwrap(); 171 | 172 | let total = if let Ok(lock) = send2.lock() { 173 | *lock 174 | } 175 | else { 0 }; 176 | 177 | if inspect { 178 | println!("elapsed: {:?}\ttotal triangles at this process: {:?}", start.elapsed(), total); 179 | } 180 | } 181 | -------------------------------------------------------------------------------- /examples/q7-lean.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0usize)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // used to partition graph loading 28 | let index = root.index(); 29 | let peers = root.peers(); 30 | 31 | // handles to input and probe, but also both indices so we can compact them. 32 | let (mut input, mut query, probe, /*forward,*/ reverse) = root.dataflow::(|builder| { 33 | 34 | // Please see triangles for more information on "graph" and dG. 35 | let (graph, dG) = builder.new_input::<((u32, u32), i32)>(); 36 | // Please see triangles for more information on "graph" and dG. 37 | let (query, dQ) = builder.new_input::<((u32, u32), ())>(); 38 | 39 | // Our query is K3 = A(x,y) B(x,z) C(y,z): triangles. 40 | // 41 | // The dataflow determines how to update this query with respect to changes in each 42 | // of the input relations: A, B, and C. Each partial derivative will use the other 43 | // relations, but the order in which attributes are added may (will) be different. 44 | // 45 | // The updates also use the other relations with slightly stale data: updates to each 46 | // relation must not see updates for "later" relations (under some order on relations). 47 | 48 | // we will index the data both by src and dst. 49 | // let (forward, f_handle) = dG.index_from(&dG.filter(|_| false).map(|_| (0,0))); 50 | let forward = IndexStream::from(|k| k as u64, 51 | &dG.map(|((x,y),_)| (x,y)), 52 | &Vec::new().to_stream(builder)); 53 | 54 | // let r_handle = reverse.index.clone() 55 | 56 | // let (reverse, r_handle) = dG.filter(|_|false) 57 | // .map(|((src,dst),wgt)| ((dst,src),wgt)) 58 | // .index_from(&dG.map(|((x,y),_)| (y,x)), |&k| k as u64); 59 | 60 | // dC(y,z) extends to x first through A(x,y) then B(x,z), both using reverse indices. 61 | let cliques = dQ.extend(vec![Box::new(forward.extend_using(|&(v1,_)| v1, |t1, t2| t1.le(t2))), 62 | Box::new(forward.extend_using(|&(_,v2)| v2, |t1, t2| t1.le(t2)))]) 63 | .flat_map(|((v1,v2),v3s,w)| v3s.into_iter().map(move |v3| ((v1,v2,v3),w))) 64 | .extend(vec![Box::new(forward.extend_using(|&(v1,_,_)| v1, |t1, t2| t1.le(t2))), 65 | Box::new(forward.extend_using(|&(_,v2,_)| v2, |t1, t2| t1.le(t2))), 66 | Box::new(forward.extend_using(|&(_,_,v3)| v3, |t1, t2| t1.le(t2)))]) 67 | .flat_map(|((v1,v2,v3),v4s,w)| v4s.into_iter().map(move |v4| ((v1,v2,v3,v4),w))) 68 | .extend(vec![Box::new(forward.extend_using(|&(v1,_,_,_)| v1, |t1, t2| t1.le(t2))), 69 | Box::new(forward.extend_using(|&(_,v2,_,_)| v2, |t1, t2| t1.le(t2))), 70 | Box::new(forward.extend_using(|&(_,_,v3,_)| v3, |t1, t2| t1.le(t2))), 71 | Box::new(forward.extend_using(|&(_,_,_,v4)| v4, |t1, t2| t1.le(t2)))]) 72 | ; 73 | 74 | // if the third argument is "inspect", report triangle counts. 75 | if inspect { 76 | cliques 77 | .inspect_batch(move |_,x| { 78 | let sum = x.iter().map(|xx| xx.1.len()).sum::(); 79 | if let Ok(mut bound) = send.lock() { 80 | *bound += sum; 81 | } 82 | }); 83 | } 84 | 85 | (graph, query, cliques.probe(), forward) 86 | }); 87 | 88 | // load fragment of input graph into memory to avoid io while running. 89 | let filename = std::env::args().nth(1).unwrap(); 90 | let graph = GraphMMap::new(&filename); 91 | 92 | let nodes = graph.nodes(); 93 | let mut edges = Vec::new(); 94 | 95 | for node in 0 .. graph.nodes() { 96 | if node % peers == index { 97 | edges.push(graph.edges(node).to_vec()); 98 | } 99 | } 100 | 101 | drop(graph); 102 | 103 | // synchronize with other workers. 104 | let prev = input.time().clone(); 105 | input.advance_to(prev.inner + 1); 106 | query.advance_to(prev.inner + 1); 107 | root.step_while(|| probe.less_than(input.time())); 108 | 109 | // number of nodes introduced at a time 110 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 111 | 112 | // start the experiment! 113 | let start = ::std::time::Instant::now(); 114 | 115 | for node in 0 .. nodes { 116 | 117 | // introduce the node if it is this worker's responsibility 118 | if node % peers == index { 119 | for &edge in &edges[node / peers] { 120 | input.send(((node as u32, edge), 1)); 121 | } 122 | } 123 | } 124 | 125 | let prev = input.time().clone(); 126 | input.advance_to(prev.inner + 1); 127 | query.advance_to(prev.inner + 1); 128 | root.step_while(|| probe.less_than(query.time())); 129 | reverse.index.borrow_mut().merge_to(&prev); 130 | input.close(); 131 | 132 | println!("{:?}: index built", start.elapsed()); 133 | 134 | for node in 0 .. nodes { 135 | 136 | // introduce the node if it is this worker's responsibility 137 | if node % peers == index { 138 | for &edge in &edges[node / peers] { 139 | query.send(((node as u32, edge), ())); 140 | } 141 | } 142 | 143 | // if at a batch boundary, advance time and do work. 144 | if node % batch == (batch - 1) { 145 | let prev = query.time().clone(); 146 | query.advance_to(prev.inner + 1); 147 | root.step_while(|| probe.less_than(query.time())); 148 | 149 | // merge all of the indices we maintain. 150 | // forward.borrow_mut().merge_to(&prev); 151 | reverse.index.borrow_mut().merge_to(&prev); 152 | } 153 | } 154 | 155 | query.close(); 156 | while root.step() { } 157 | 158 | if inspect { 159 | println!("worker {} elapsed: {:?}", index, start.elapsed()); 160 | } 161 | 162 | }).unwrap(); 163 | 164 | let total = if let Ok(lock) = send2.lock() { 165 | *lock 166 | } 167 | else { 0 }; 168 | 169 | if inspect { 170 | println!("elapsed: {:?}\ttotal triangles at this process: {:?}", start.elapsed(), total); 171 | } 172 | } 173 | -------------------------------------------------------------------------------- /examples/seed-q3-lean.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0usize)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // handles to input and probe, but also both indices so we can compact them. 28 | let (mut input, mut query, probe, forward) = root.dataflow::(|builder| { 29 | 30 | // A stream of changes to the set of *triangles*, where a < b < c. 31 | let (graph, dT) = builder.new_input::<((u32, u32, u32), i32)>(); 32 | 33 | // A stream of changes to the set of *triangles*, where a < b < c. 34 | let (query, dQ) = builder.new_input::<((u32, u32, u32), ())>(); 35 | 36 | // Our query is K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z), T(x,y,z) 37 | // 38 | // This query is technically redundant, because the middle two constraints imply the fourth, 39 | // so let's slim it down to 40 | // 41 | // K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z) 42 | // 43 | // This seems like it could be a bit more complicated than triangles, in determining the rules 44 | // for incremental updates. I'm going to write them down first, and we'll see which indices we 45 | // actually need. I'll use A, B, and C for the instances of T above. 46 | // 47 | // dK4dA(w,x,y,z) := dA(w,x,y), B(w,x,z), C(w,y,z) 48 | // dK4dB(w,x,y,z) := dB(w,x,z), A(w,x,y), C(w,y,z) 49 | // dK4dC(w,x,y,z) := dC(w,y,z), A(w,x,y), B(w,x,z) 50 | // 51 | // Looking at this, it seems like we will need 52 | // 53 | // dK4dA : indices on (w,x,_) and (w,_,y) 54 | // dK4dB : indices on (w,x,_) and (w,_,z) 55 | // dK4dC : indices on (w,_,y) and (w,_,z) 56 | // 57 | // All of this seems to boil down to a "forward" and a "reverse" index, just as for triangles, 58 | // but where `w` is always present as part of the key. We just might want the first or second 59 | // field that follows it. 60 | 61 | let forward = IndexStream::from( 62 | |(a,b)| (a + b) as u64, // distribute triangles by a + b. 63 | &dT.map(|((a,b,c),_)| ((a,b),c)), // initialize with (a,b) keys and c values. 64 | &Vec::new().to_stream(builder) // empty update stream. 65 | ); 66 | 67 | // extend (w,x,y) with z values such that both (w,x,z) and (w,y,z) exist. 68 | let dK4dA = dQ.extend(vec![Box::new(forward.extend_using(|&(w,x,_y)| (w,x), <_ as PartialOrd>::le)), 69 | Box::new(forward.extend_using(|&(w,_x,y)| (w,y), <_ as PartialOrd>::le))]); 70 | 71 | // if the third argument is "inspect", report triangle counts. 72 | if inspect { 73 | dK4dA.inspect_batch(move |_,x| { 74 | let sum = x.iter().map(|xx| xx.1.len()).sum::(); 75 | if let Ok(mut bound) = send.lock() { 76 | *bound += sum; 77 | } 78 | }); 79 | } 80 | 81 | (graph, query, dK4dA.probe(), forward) 82 | }); 83 | 84 | // load fragment of input graph into memory to avoid io while running. 85 | let filename = std::env::args().nth(1).unwrap(); 86 | let graph = GraphMMap::new(&filename); 87 | 88 | let mut triangles = Vec::new(); 89 | 90 | let mut v1 = root.index(); 91 | while v1 < graph.nodes() { 92 | let v1f = graph.edges(v1); 93 | for &v2 in v1f.iter() { 94 | intersect_and(v1f, graph.edges(v2 as usize), |v3| triangles.push((v1 as u32, v2, v3))); 95 | } 96 | v1 += root.peers(); 97 | } 98 | 99 | drop(graph); 100 | 101 | println!("{:?}\tworker {} computed {} triangles", start.elapsed(), root.index(), triangles.len()); 102 | 103 | for &(a,b,c) in triangles.iter() { 104 | input.send(((a,b,c), 1)); 105 | } 106 | 107 | // synchronize with other workers. 108 | let prev = query.time().clone(); 109 | input.close(); 110 | query.advance_to(prev.inner + 1); 111 | while probe.less_than(query.time()) { root.step(); } 112 | forward.index.borrow_mut().merge_to(&prev); 113 | 114 | println!("{:?}\tworker {} loaded index", start.elapsed(), root.index()); 115 | 116 | // number of nodes introduced at a time 117 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 118 | 119 | let mut node = 0; 120 | let mut sent = 0; 121 | 122 | while sent < triangles.len() { 123 | node += batch as u32; 124 | while sent < triangles.len() && triangles[sent].0 < node { 125 | query.send((triangles[sent], ())); 126 | sent += 1; 127 | } 128 | 129 | // advance input and synchronize. 130 | let prev = query.time().clone(); 131 | query.advance_to(prev.inner + 1); 132 | while probe.less_than(query.time()) { root.step(); } 133 | 134 | // merge all of the indices we maintain. 135 | forward.index.borrow_mut().merge_to(&prev); 136 | } 137 | 138 | query.close(); 139 | while root.step() { } 140 | 141 | if inspect { 142 | println!("{:?}\tworker {} complete", start.elapsed(), root.index()); 143 | } 144 | 145 | }).unwrap(); 146 | 147 | let total = if let Ok(lock) = send2.lock() { 148 | *lock 149 | } 150 | else { 0 }; 151 | 152 | if inspect { 153 | println!("elapsed: {:?}\ttotal instances at this process: {:?}", start.elapsed(), total); 154 | } 155 | } 156 | 157 | 158 | fn intersect_and(aaa: &[u32], mut bbb: &[u32], mut func: F) { 159 | 160 | if aaa.len() > bbb.len() { 161 | intersect_and(bbb, aaa, func); 162 | } 163 | else { 164 | if aaa.len() < bbb.len() / 16 { 165 | for &a in aaa.iter() { 166 | bbb = gallop_ge(bbb, &a); 167 | if bbb.len() > 0 && bbb[0] == a { 168 | func(a) 169 | } 170 | } 171 | } 172 | else { 173 | for &a in aaa.iter() { 174 | while bbb.len() > 0 && bbb[0] < a { 175 | bbb = &bbb[1..]; 176 | } 177 | if bbb.len() > 0 && a == bbb[0] { 178 | func(a); 179 | } 180 | } 181 | } 182 | } 183 | } 184 | 185 | #[inline(always)] 186 | pub fn gallop_ge<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 187 | // if empty slice, or already >= element, return 188 | if slice.len() > 0 && &slice[0] < value { 189 | let mut step = 1; 190 | while step < slice.len() && &slice[step] < value { 191 | slice = &slice[step..]; 192 | step = step << 1; 193 | } 194 | 195 | step = step >> 1; 196 | while step > 0 { 197 | if step < slice.len() && &slice[step] < value { 198 | slice = &slice[step..]; 199 | } 200 | step = step >> 1; 201 | } 202 | 203 | slice = &slice[1..]; // advance one, as we always stayed < value 204 | } 205 | 206 | return slice; 207 | } 208 | 209 | // #[inline(always)] 210 | // pub fn gallop_gt<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 211 | // // if empty slice, or already > element, return 212 | // if slice.len() > 0 && &slice[0] <= value { 213 | // let mut step = 1; 214 | // while step < slice.len() && &slice[step] <= value { 215 | // slice = &slice[step..]; 216 | // step = step << 1; 217 | // } 218 | 219 | // step = step >> 1; 220 | // while step > 0 { 221 | // if step < slice.len() && &slice[step] <= value { 222 | // slice = &slice[step..]; 223 | // } 224 | // step = step >> 1; 225 | // } 226 | 227 | // slice = &slice[1..]; // advance one, as we always stayed <= value 228 | // } 229 | 230 | // return slice; 231 | // } -------------------------------------------------------------------------------- /examples/seed-q5-lean.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0usize)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // handles to input and probe, but also both indices so we can compact them. 28 | let (mut input, mut query, probe, forward) = root.dataflow::(|builder| { 29 | 30 | // A stream of changes to the set of *triangles*, where a < b < c. 31 | let (graph, dT) = builder.new_input::<((u32, u32, u32), i32)>(); 32 | 33 | // A stream of changes to the set of *triangles*, where a < b < c. 34 | let (query, dQ) = builder.new_input::<((u32, u32, u32), ())>(); 35 | 36 | // Our query is K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z), T(x,y,z) 37 | // 38 | // This query is technically redundant, because the middle two constraints imply the fourth, 39 | // so let's slim it down to 40 | // 41 | // K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z) 42 | // 43 | // This seems like it could be a bit more complicated than triangles, in determining the rules 44 | // for incremental updates. I'm going to write them down first, and we'll see which indices we 45 | // actually need. I'll use A, B, and C for the instances of T above. 46 | // 47 | // dK4dA(w,x,y,z) := dA(w,x,y), B(w,x,z), C(w,y,z) 48 | // dK4dB(w,x,y,z) := dB(w,x,z), A(w,x,y), C(w,y,z) 49 | // dK4dC(w,x,y,z) := dC(w,y,z), A(w,x,y), B(w,x,z) 50 | // 51 | // Looking at this, it seems like we will need 52 | // 53 | // dK4dA : indices on (w,x,_) and (w,_,y) 54 | // dK4dB : indices on (w,x,_) and (w,_,z) 55 | // dK4dC : indices on (w,_,y) and (w,_,z) 56 | // 57 | // All of this seems to boil down to a "forward" and a "reverse" index, just as for triangles, 58 | // but where `w` is always present as part of the key. We just might want the first or second 59 | // field that follows it. 60 | 61 | let index = IndexStream::from( 62 | |(a,b)| (a + b) as u64, // distribute triangles by a + b. 63 | &dT.map(|((a,b,c),_)| ((a,b),c)), // initialize with (a,b) keys and c values. 64 | &Vec::new().to_stream(builder) // empty update stream. 65 | ); 66 | 67 | // we start with dQ = (v1, v3, v4) 68 | let dQ5dA = dQ 69 | .extend(vec![Box::new(index.extend_using(|&(v1,v3,_v4)| min_max(v1,v3), <_ as PartialOrd>::le))]) 70 | .map(|((v1,v3,v4),v2s,w)| ((v1,v2s,v3,v4),w)) 71 | .extend(vec![Box::new(index.extend_using(|&(v1,_,_v3,v4)| min_max(v1,v4), <_ as PartialOrd>::le))]) 72 | .flat_map(|((v1,v2s,v3,v4),v5s,w)| 73 | v5s.into_iter() 74 | .filter(move |&v5| v3 < v5) 75 | .map(move|v5| ((v1,v2s.clone(),v3,v4,v5),w)) 76 | ) 77 | .extend(vec![Box::new(index.extend_using(|&(v1,_,_,_,v5)| min_max(v1,v5), <_ as PartialOrd>::le))]) 78 | .map(|((v1, mut v2s, v3, v4, v5), mut v6s, w)| { 79 | v2s.retain(|&v2| v2 != v4 && v2 != v5); 80 | v6s.retain(|&v6| v6 != v3 && v6 != v4); 81 | ((v1,v2s,v3,v4,v5,v6s),w) 82 | }); 83 | 84 | // if the third argument is "inspect", report triangle counts. 85 | if inspect { 86 | dQ5dA.inspect_batch(move |_,x| { 87 | 88 | let mut sum = 0; 89 | for &(ref tuple,_) in x.iter() { 90 | let v2s = &tuple.1; 91 | let v6s = &tuple.5; 92 | for &v2 in v2s.iter() { 93 | for &v6 in v6s.iter() { 94 | if v2 != v6 { 95 | sum += 1; 96 | } 97 | } 98 | } 99 | } 100 | 101 | if let Ok(mut bound) = send.lock() { 102 | *bound += sum; 103 | } 104 | }); 105 | } 106 | 107 | (graph, query, dQ5dA.probe(), index) 108 | }); 109 | 110 | // load fragment of input graph into memory to avoid io while running. 111 | let filename = std::env::args().nth(1).unwrap(); 112 | let graph = GraphMMap::new(&filename); 113 | 114 | let mut triangles = Vec::new(); 115 | 116 | let mut v1 = root.index(); 117 | while v1 < graph.nodes() { 118 | let v1f = graph.edges(v1); 119 | for &v2 in v1f.iter() { 120 | intersect_and(v1f, graph.edges(v2 as usize), |v3| triangles.push((v1 as u32, v2, v3))); 121 | } 122 | v1 += root.peers(); 123 | } 124 | 125 | drop(graph); 126 | 127 | println!("{:?}\tworker {} computed {} triangles", start.elapsed(), root.index(), triangles.len()); 128 | 129 | for &(a,b,c) in triangles.iter() { 130 | // send each extension 131 | input.send(((a,b,c), 1)); 132 | input.send(((a,c,b), 1)); 133 | input.send(((b,c,a), 1)); 134 | } 135 | 136 | // synchronize with other workers. 137 | let prev = query.time().clone(); 138 | input.close(); 139 | query.advance_to(prev.inner + 1); 140 | while probe.less_than(query.time()) { root.step(); } 141 | forward.index.borrow_mut().merge_to(&prev); 142 | 143 | println!("{:?}\tworker {} loaded index", start.elapsed(), root.index()); 144 | 145 | // number of nodes introduced at a time 146 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 147 | 148 | let mut node = 0; 149 | let mut sent = 0; 150 | 151 | while sent < triangles.len() { 152 | node += batch as u32; 153 | while sent < triangles.len() && triangles[sent].0 < node { 154 | let (a,b,c) = triangles[sent]; 155 | query.send(((a,b,c), ())); 156 | query.send(((a,c,b), ())); 157 | query.send(((b,a,c), ())); 158 | query.send(((b,c,a), ())); 159 | query.send(((c,a,b), ())); 160 | query.send(((c,b,a), ())); 161 | sent += 1; 162 | } 163 | 164 | // advance input and synchronize. 165 | let prev = query.time().clone(); 166 | query.advance_to(prev.inner + 1); 167 | while probe.less_than(query.time()) { root.step(); } 168 | 169 | // merge all of the indices we maintain. 170 | forward.index.borrow_mut().merge_to(&prev); 171 | } 172 | 173 | query.close(); 174 | while root.step() { } 175 | 176 | if inspect { 177 | println!("{:?}\tworker {} complete", start.elapsed(), root.index()); 178 | } 179 | 180 | }).unwrap(); 181 | 182 | let total = if let Ok(lock) = send2.lock() { 183 | *lock 184 | } 185 | else { 0 }; 186 | 187 | if inspect { 188 | println!("elapsed: {:?}\ttotal instances at this process: {:?}", start.elapsed(), total); 189 | } 190 | } 191 | 192 | fn min_max(a: T, b: T) -> (T, T) { 193 | if a < b { (a, b) } else { (b, a) } 194 | } 195 | 196 | fn intersect_and(aaa: &[u32], mut bbb: &[u32], mut func: F) { 197 | 198 | if aaa.len() > bbb.len() { 199 | intersect_and(bbb, aaa, func); 200 | } 201 | else { 202 | if aaa.len() < bbb.len() / 16 { 203 | for &a in aaa.iter() { 204 | bbb = gallop_ge(bbb, &a); 205 | if bbb.len() > 0 && bbb[0] == a { 206 | func(a) 207 | } 208 | } 209 | } 210 | else { 211 | for &a in aaa.iter() { 212 | while bbb.len() > 0 && bbb[0] < a { 213 | bbb = &bbb[1..]; 214 | } 215 | if bbb.len() > 0 && a == bbb[0] { 216 | func(a); 217 | } 218 | } 219 | } 220 | } 221 | } 222 | 223 | #[inline(always)] 224 | pub fn gallop_ge<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 225 | // if empty slice, or already >= element, return 226 | if slice.len() > 0 && &slice[0] < value { 227 | let mut step = 1; 228 | while step < slice.len() && &slice[step] < value { 229 | slice = &slice[step..]; 230 | step = step << 1; 231 | } 232 | 233 | step = step >> 1; 234 | while step > 0 { 235 | if step < slice.len() && &slice[step] < value { 236 | slice = &slice[step..]; 237 | } 238 | step = step >> 1; 239 | } 240 | 241 | slice = &slice[1..]; // advance one, as we always stayed < value 242 | } 243 | 244 | return slice; 245 | } 246 | 247 | // #[inline(always)] 248 | // pub fn gallop_gt<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 249 | // // if empty slice, or already > element, return 250 | // if slice.len() > 0 && &slice[0] <= value { 251 | // let mut step = 1; 252 | // while step < slice.len() && &slice[step] <= value { 253 | // slice = &slice[step..]; 254 | // step = step << 1; 255 | // } 256 | 257 | // step = step >> 1; 258 | // while step > 0 { 259 | // if step < slice.len() && &slice[step] <= value { 260 | // slice = &slice[step..]; 261 | // } 262 | // step = step >> 1; 263 | // } 264 | 265 | // slice = &slice[1..]; // advance one, as we always stayed <= value 266 | // } 267 | 268 | // return slice; 269 | // } -------------------------------------------------------------------------------- /examples/seed-q6-lean-alt.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0usize)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // handles to input and probe, but also both indices so we can compact them. 28 | let (mut input, mut query, probe, forward) = root.dataflow::(|builder| { 29 | 30 | // A stream of changes to the set of *triangles*, where a < b < c. 31 | let (graph, dT) = builder.new_input::<((u32, u32, u32), i32)>(); 32 | 33 | // A stream of changes to the set of *triangles*, where a < b < c. 34 | let (query, dQ) = builder.new_input::<((u32, u32), ())>(); 35 | 36 | // Our query is K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z), T(x,y,z) 37 | // 38 | // This query is technically redundant, because the middle two constraints imply the fourth, 39 | // so let's slim it down to 40 | // 41 | // K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z) 42 | // 43 | // This seems like it could be a bit more complicated than triangles, in determining the rules 44 | // for incremental updates. I'm going to write them down first, and we'll see which indices we 45 | // actually need. I'll use A, B, and C for the instances of T above. 46 | // 47 | // dK4dA(w,x,y,z) := dA(w,x,y), B(w,x,z), C(w,y,z) 48 | // dK4dB(w,x,y,z) := dB(w,x,z), A(w,x,y), C(w,y,z) 49 | // dK4dC(w,x,y,z) := dC(w,y,z), A(w,x,y), B(w,x,z) 50 | // 51 | // Looking at this, it seems like we will need 52 | // 53 | // dK4dA : indices on (w,x,_) and (w,_,y) 54 | // dK4dB : indices on (w,x,_) and (w,_,z) 55 | // dK4dC : indices on (w,_,y) and (w,_,z) 56 | // 57 | // All of this seems to boil down to a "forward" and a "reverse" index, just as for triangles, 58 | // but where `w` is always present as part of the key. We just might want the first or second 59 | // field that follows it. 60 | 61 | let forward = IndexStream::from( 62 | |(a,b)| (a + b) as u64, // distribute triangles by a + b. 63 | &dT.map(|((a,b,c),_)| ((a,b),c)), // initialize with (a,b) keys and c values. 64 | &Vec::new().to_stream(builder) // empty update stream. 65 | ); 66 | 67 | // we bind dQ as (v2, v5, v3), exploiting the fact that we have bound each 68 | let dK4dA = dQ.extend(vec![Box::new(forward.extend_using(|&(v2,v5)| min_max(v2,v5), <_ as PartialOrd>::le))]) 69 | .flat_map(|((v2,v5),v3s,w)| { 70 | let v1s = v3s.clone(); 71 | v3s.into_iter().map(move |v3| ((v1s.clone(),v2,v3,v5),w)) 72 | }) 73 | .extend(vec![Box::new(forward.extend_using(|&(ref _v1s,v2,_v3,v5)| min_max(v2,v5), <_ as PartialOrd>::le)), 74 | Box::new(forward.extend_using(|&(ref _v1s,v2,v3,_v5)| min_max(v2,v3), <_ as PartialOrd>::le))]) 75 | .map(|((v1s,v2,v3,v5), mut v4s, w)| { 76 | v4s.retain(|&v4| v2 != v4 && v3 < v4); 77 | ((v1s,v2,v3,v4s,v5),w) 78 | }) 79 | ; 80 | // if the third argument is "inspect", report triangle counts. 81 | if inspect { 82 | dK4dA.inspect_batch(move |_,x| { 83 | let mut sum = 0; 84 | for &((ref v1s, _v2, v3, ref v4s, _v5),_) in x.iter() { 85 | for &v1 in v1s.iter() { 86 | if v1 != v3 { 87 | for &v4 in v4s.iter() { 88 | if v1 != v4 { 89 | sum += 1; 90 | } 91 | } 92 | } 93 | } 94 | } 95 | 96 | if let Ok(mut bound) = send.lock() { 97 | *bound += sum; 98 | } 99 | }); 100 | } 101 | 102 | (graph, query, dK4dA.probe(), forward) 103 | }); 104 | 105 | // load fragment of input graph into memory to avoid io while running. 106 | let filename = std::env::args().nth(1).unwrap(); 107 | let graph = GraphMMap::new(&filename); 108 | 109 | let mut triangles = Vec::new(); 110 | 111 | let mut v1 = root.index(); 112 | while v1 < graph.nodes() { 113 | let v1f = graph.edges(v1); 114 | for &v2 in v1f.iter() { 115 | intersect_and(v1f, graph.edges(v2 as usize), |v3| triangles.push((v1 as u32, v2, v3))); 116 | } 117 | v1 += root.peers(); 118 | } 119 | 120 | // drop(graph); 121 | 122 | println!("{:?}\tworker {} computed {} triangles", start.elapsed(), root.index(), triangles.len()); 123 | 124 | for &(a,b,c) in triangles.iter() { 125 | // send each extension 126 | input.send(((a,b,c), 1)); 127 | input.send(((a,c,b), 1)); 128 | input.send(((b,c,a), 1)); 129 | } 130 | 131 | // synchronize with other workers. 132 | let prev = query.time().clone(); 133 | input.close(); 134 | query.advance_to(prev.inner + 1); 135 | while probe.less_than(query.time()) { root.step(); } 136 | forward.index.borrow_mut().merge_to(&prev); 137 | 138 | println!("{:?}\tworker {} loaded index", start.elapsed(), root.index()); 139 | 140 | // number of nodes introduced at a time 141 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 142 | 143 | let mut node = 0; 144 | let mut sent = 0; 145 | 146 | while sent < graph.nodes() { 147 | node += batch; 148 | while sent < graph.nodes() && sent < node { 149 | for &edge in graph.edges(sent) { 150 | query.send(((sent as u32, edge), ())); 151 | } 152 | // let (a,b,c) = triangles[sent]; 153 | // query.send(((a,b,c), ())); 154 | // query.send(((a,c,b), ())); 155 | // query.send(((b,c,a), ())); 156 | sent += 1; 157 | } 158 | 159 | // advance input and synchronize. 160 | let prev = query.time().clone(); 161 | query.advance_to(prev.inner + 1); 162 | while probe.less_than(query.time()) { root.step(); } 163 | 164 | // merge all of the indices we maintain. 165 | forward.index.borrow_mut().merge_to(&prev); 166 | } 167 | 168 | query.close(); 169 | while root.step() { } 170 | 171 | if inspect { 172 | println!("{:?}\tworker {} complete", start.elapsed(), root.index()); 173 | } 174 | 175 | }).unwrap(); 176 | 177 | let total = if let Ok(lock) = send2.lock() { 178 | *lock 179 | } 180 | else { 0 }; 181 | 182 | if inspect { 183 | println!("elapsed: {:?}\ttotal instances at this process: {:?}", start.elapsed(), total); 184 | } 185 | } 186 | 187 | fn min_max(a: T, b: T) -> (T, T) { 188 | if a < b { (a, b) } else { (b, a) } 189 | } 190 | 191 | fn intersect_and(aaa: &[u32], mut bbb: &[u32], mut func: F) { 192 | 193 | if aaa.len() > bbb.len() { 194 | intersect_and(bbb, aaa, func); 195 | } 196 | else { 197 | if aaa.len() < bbb.len() / 16 { 198 | for &a in aaa.iter() { 199 | bbb = gallop_ge(bbb, &a); 200 | if bbb.len() > 0 && bbb[0] == a { 201 | func(a) 202 | } 203 | } 204 | } 205 | else { 206 | for &a in aaa.iter() { 207 | while bbb.len() > 0 && bbb[0] < a { 208 | bbb = &bbb[1..]; 209 | } 210 | if bbb.len() > 0 && a == bbb[0] { 211 | func(a); 212 | } 213 | } 214 | } 215 | } 216 | } 217 | 218 | #[inline(always)] 219 | pub fn gallop_ge<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 220 | // if empty slice, or already >= element, return 221 | if slice.len() > 0 && &slice[0] < value { 222 | let mut step = 1; 223 | while step < slice.len() && &slice[step] < value { 224 | slice = &slice[step..]; 225 | step = step << 1; 226 | } 227 | 228 | step = step >> 1; 229 | while step > 0 { 230 | if step < slice.len() && &slice[step] < value { 231 | slice = &slice[step..]; 232 | } 233 | step = step >> 1; 234 | } 235 | 236 | slice = &slice[1..]; // advance one, as we always stayed < value 237 | } 238 | 239 | return slice; 240 | } 241 | 242 | // #[inline(always)] 243 | // pub fn gallop_gt<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 244 | // // if empty slice, or already > element, return 245 | // if slice.len() > 0 && &slice[0] <= value { 246 | // let mut step = 1; 247 | // while step < slice.len() && &slice[step] <= value { 248 | // slice = &slice[step..]; 249 | // step = step << 1; 250 | // } 251 | 252 | // step = step >> 1; 253 | // while step > 0 { 254 | // if step < slice.len() && &slice[step] <= value { 255 | // slice = &slice[step..]; 256 | // } 257 | // step = step >> 1; 258 | // } 259 | 260 | // slice = &slice[1..]; // advance one, as we always stayed <= value 261 | // } 262 | 263 | // return slice; 264 | // } -------------------------------------------------------------------------------- /examples/seed-q6-lean.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0usize)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // handles to input and probe, but also both indices so we can compact them. 28 | let (mut input, mut query, probe, forward) = root.dataflow::(|builder| { 29 | 30 | // A stream of changes to the set of *triangles*, where a < b < c. 31 | let (graph, dT) = builder.new_input::<((u32, u32, u32), i32)>(); 32 | 33 | // A stream of changes to the set of *triangles*, where a < b < c. 34 | let (query, dQ) = builder.new_input::<((u32, u32, u32), ())>(); 35 | 36 | // Our query is K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z), T(x,y,z) 37 | // 38 | // This query is technically redundant, because the middle two constraints imply the fourth, 39 | // so let's slim it down to 40 | // 41 | // K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z) 42 | // 43 | // This seems like it could be a bit more complicated than triangles, in determining the rules 44 | // for incremental updates. I'm going to write them down first, and we'll see which indices we 45 | // actually need. I'll use A, B, and C for the instances of T above. 46 | // 47 | // dK4dA(w,x,y,z) := dA(w,x,y), B(w,x,z), C(w,y,z) 48 | // dK4dB(w,x,y,z) := dB(w,x,z), A(w,x,y), C(w,y,z) 49 | // dK4dC(w,x,y,z) := dC(w,y,z), A(w,x,y), B(w,x,z) 50 | // 51 | // Looking at this, it seems like we will need 52 | // 53 | // dK4dA : indices on (w,x,_) and (w,_,y) 54 | // dK4dB : indices on (w,x,_) and (w,_,z) 55 | // dK4dC : indices on (w,_,y) and (w,_,z) 56 | // 57 | // All of this seems to boil down to a "forward" and a "reverse" index, just as for triangles, 58 | // but where `w` is always present as part of the key. We just might want the first or second 59 | // field that follows it. 60 | 61 | let forward = IndexStream::from( 62 | |(a,b)| (a + b) as u64, // distribute triangles by a + b. 63 | &dT.map(|((a,b,c),_)| ((a,b),c)), // initialize with (a,b) keys and c values. 64 | &Vec::new().to_stream(builder) // empty update stream. 65 | ); 66 | 67 | // we bind dQ as (v2, v5, v3), exploiting the fact that we have bound each 68 | let dK4dA = dQ.extend(vec![Box::new(forward.extend_using(|&(v2,v5,_v3)| min_max(v2,v5), <_ as PartialOrd>::le)), 69 | Box::new(forward.extend_using(|&(v2,_v5,v3)| min_max(v2,v3), <_ as PartialOrd>::le))]) 70 | .map(|((v2,v5,v3), mut v4s, w)| { 71 | v4s.retain(|&v4| v2 != v4 && v3 < v4); 72 | ((v2,v3,v4s,v5),w) 73 | }) 74 | .extend(vec![Box::new(forward.extend_using(|&(v2,_,_,v5)| min_max(v2,v5), <_ as PartialOrd>::le))]) 75 | .map(|((v2,v3,v4s,v5), v1s, w)| ((v1s,v2,v3,v4s,v5),w)); 76 | 77 | // if the third argument is "inspect", report triangle counts. 78 | if inspect { 79 | dK4dA.inspect_batch(move |_,x| { 80 | let mut sum = 0; 81 | for &((ref v1s, _v2, v3, ref v4s, _v5),_) in x.iter() { 82 | for &v1 in v1s.iter() { 83 | if v1 != v3 { 84 | for &v4 in v4s.iter() { 85 | if v1 != v4 { 86 | sum += 1; 87 | } 88 | } 89 | } 90 | } 91 | } 92 | 93 | if let Ok(mut bound) = send.lock() { 94 | *bound += sum; 95 | } 96 | }); 97 | } 98 | 99 | (graph, query, dK4dA.probe(), forward) 100 | }); 101 | 102 | // load fragment of input graph into memory to avoid io while running. 103 | let filename = std::env::args().nth(1).unwrap(); 104 | let graph = GraphMMap::new(&filename); 105 | 106 | let mut triangles = Vec::new(); 107 | 108 | let mut v1 = root.index(); 109 | while v1 < graph.nodes() { 110 | let v1f = graph.edges(v1); 111 | for &v2 in v1f.iter() { 112 | intersect_and(v1f, graph.edges(v2 as usize), |v3| triangles.push((v1 as u32, v2, v3))); 113 | } 114 | v1 += root.peers(); 115 | } 116 | 117 | drop(graph); 118 | 119 | println!("{:?}\tworker {} computed {} triangles", start.elapsed(), root.index(), triangles.len()); 120 | 121 | for &(a,b,c) in triangles.iter() { 122 | // send each extension 123 | input.send(((a,b,c), 1)); 124 | input.send(((a,c,b), 1)); 125 | input.send(((b,c,a), 1)); 126 | } 127 | 128 | // synchronize with other workers. 129 | let prev = query.time().clone(); 130 | input.close(); 131 | query.advance_to(prev.inner + 1); 132 | while probe.less_than(query.time()) { root.step(); } 133 | forward.index.borrow_mut().merge_to(&prev); 134 | 135 | println!("{:?}\tworker {} loaded index", start.elapsed(), root.index()); 136 | 137 | // number of nodes introduced at a time 138 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 139 | 140 | let mut node = 0; 141 | let mut sent = 0; 142 | 143 | while sent < triangles.len() { 144 | node += batch as u32; 145 | while sent < triangles.len() && triangles[sent].0 < node { 146 | let (a,b,c) = triangles[sent]; 147 | query.send(((a,b,c), ())); 148 | query.send(((a,c,b), ())); 149 | query.send(((b,c,a), ())); 150 | sent += 1; 151 | } 152 | 153 | // advance input and synchronize. 154 | let prev = query.time().clone(); 155 | query.advance_to(prev.inner + 1); 156 | while probe.less_than(query.time()) { root.step(); } 157 | 158 | // merge all of the indices we maintain. 159 | forward.index.borrow_mut().merge_to(&prev); 160 | } 161 | 162 | query.close(); 163 | while root.step() { } 164 | 165 | if inspect { 166 | println!("{:?}\tworker {} complete", start.elapsed(), root.index()); 167 | } 168 | 169 | }).unwrap(); 170 | 171 | let total = if let Ok(lock) = send2.lock() { 172 | *lock 173 | } 174 | else { 0 }; 175 | 176 | if inspect { 177 | println!("elapsed: {:?}\ttotal instances at this process: {:?}", start.elapsed(), total); 178 | } 179 | } 180 | 181 | fn min_max(a: T, b: T) -> (T, T) { 182 | if a < b { (a, b) } else { (b, a) } 183 | } 184 | 185 | fn intersect_and(aaa: &[u32], mut bbb: &[u32], mut func: F) { 186 | 187 | if aaa.len() > bbb.len() { 188 | intersect_and(bbb, aaa, func); 189 | } 190 | else { 191 | if aaa.len() < bbb.len() / 16 { 192 | for &a in aaa.iter() { 193 | bbb = gallop_ge(bbb, &a); 194 | if bbb.len() > 0 && bbb[0] == a { 195 | func(a) 196 | } 197 | } 198 | } 199 | else { 200 | for &a in aaa.iter() { 201 | while bbb.len() > 0 && bbb[0] < a { 202 | bbb = &bbb[1..]; 203 | } 204 | if bbb.len() > 0 && a == bbb[0] { 205 | func(a); 206 | } 207 | } 208 | } 209 | } 210 | } 211 | 212 | #[inline(always)] 213 | pub fn gallop_ge<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 214 | // if empty slice, or already >= element, return 215 | if slice.len() > 0 && &slice[0] < value { 216 | let mut step = 1; 217 | while step < slice.len() && &slice[step] < value { 218 | slice = &slice[step..]; 219 | step = step << 1; 220 | } 221 | 222 | step = step >> 1; 223 | while step > 0 { 224 | if step < slice.len() && &slice[step] < value { 225 | slice = &slice[step..]; 226 | } 227 | step = step >> 1; 228 | } 229 | 230 | slice = &slice[1..]; // advance one, as we always stayed < value 231 | } 232 | 233 | return slice; 234 | } 235 | 236 | // #[inline(always)] 237 | // pub fn gallop_gt<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 238 | // // if empty slice, or already > element, return 239 | // if slice.len() > 0 && &slice[0] <= value { 240 | // let mut step = 1; 241 | // while step < slice.len() && &slice[step] <= value { 242 | // slice = &slice[step..]; 243 | // step = step << 1; 244 | // } 245 | 246 | // step = step >> 1; 247 | // while step > 0 { 248 | // if step < slice.len() && &slice[step] <= value { 249 | // slice = &slice[step..]; 250 | // } 251 | // step = step >> 1; 252 | // } 253 | 254 | // slice = &slice[1..]; // advance one, as we always stayed <= value 255 | // } 256 | 257 | // return slice; 258 | // } -------------------------------------------------------------------------------- /examples/seed-q7-lean.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0usize)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // handles to input and probe, but also both indices so we can compact them. 28 | let (mut input, mut query, probe, forward) = root.dataflow::(|builder| { 29 | 30 | // A stream of changes to the set of *triangles*, where a < b < c. 31 | let (graph, dT) = builder.new_input::<((u32, u32, u32), i32)>(); 32 | 33 | // A stream of changes to the set of *triangles*, where a < b < c. 34 | let (query, dQ) = builder.new_input::<((u32, u32, u32), ())>(); 35 | 36 | // Our query is K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z), T(x,y,z) 37 | // 38 | // This query is technically redundant, because the middle two constraints imply the fourth, 39 | // so let's slim it down to 40 | // 41 | // K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z) 42 | // 43 | // This seems like it could be a bit more complicated than triangles, in determining the rules 44 | // for incremental updates. I'm going to write them down first, and we'll see which indices we 45 | // actually need. I'll use A, B, and C for the instances of T above. 46 | // 47 | // dK4dA(w,x,y,z) := dA(w,x,y), B(w,x,z), C(w,y,z) 48 | // dK4dB(w,x,y,z) := dB(w,x,z), A(w,x,y), C(w,y,z) 49 | // dK4dC(w,x,y,z) := dC(w,y,z), A(w,x,y), B(w,x,z) 50 | // 51 | // Looking at this, it seems like we will need 52 | // 53 | // dK4dA : indices on (w,x,_) and (w,_,y) 54 | // dK4dB : indices on (w,x,_) and (w,_,z) 55 | // dK4dC : indices on (w,_,y) and (w,_,z) 56 | // 57 | // All of this seems to boil down to a "forward" and a "reverse" index, just as for triangles, 58 | // but where `w` is always present as part of the key. We just might want the first or second 59 | // field that follows it. 60 | 61 | let forward = IndexStream::from( 62 | |(a,b)| (a + b) as u64, // distribute triangles by a + b. 63 | &dT.map(|((a,b,c),_)| ((a,b),c)), // initialize with (a,b) keys and c values. 64 | &Vec::new().to_stream(builder) // empty update stream. 65 | ); 66 | 67 | // extend (w,x,y) with z values such that both (w,x,z) and (w,y,z) exist. 68 | let dQ7dA = dQ.extend(vec![Box::new(forward.extend_using(|&(v1,v2,_v3)| (v1,v2), <_ as PartialOrd>::le)), 69 | Box::new(forward.extend_using(|&(v1,_v2,v3)| (v1,v3), <_ as PartialOrd>::le))]) 70 | .flat_map(|((v1,v2,v3),v4s,w)| v4s.into_iter().map(move |v4| ((v1,v2,v3,v4),w))) 71 | .extend(vec![Box::new(forward.extend_using(|&(v1,v2,_v3,_v4)| (v1,v2), <_ as PartialOrd>::le)), 72 | Box::new(forward.extend_using(|&(_v1,_v2,v3,v4)| (v3,v4), <_ as PartialOrd>::le))]); 73 | 74 | // if the third argument is "inspect", report triangle counts. 75 | if inspect { 76 | dQ7dA.inspect_batch(move |_,x| { 77 | let sum: usize = x.iter().map(|xx| xx.1.len()).sum(); 78 | if let Ok(mut bound) = send.lock() { 79 | *bound += sum; 80 | } 81 | }); 82 | } 83 | 84 | (graph, query, dQ7dA.probe(), forward) 85 | }); 86 | 87 | // load fragment of input graph into memory to avoid io while running. 88 | let filename = std::env::args().nth(1).unwrap(); 89 | let graph = GraphMMap::new(&filename); 90 | 91 | let mut triangles = Vec::new(); 92 | 93 | let mut v1 = root.index(); 94 | while v1 < graph.nodes() { 95 | let v1f = graph.edges(v1); 96 | for &v2 in v1f.iter() { 97 | intersect_and(v1f, graph.edges(v2 as usize), |v3| triangles.push((v1 as u32, v2, v3))); 98 | } 99 | v1 += root.peers(); 100 | } 101 | 102 | drop(graph); 103 | 104 | println!("{:?}\tworker {} computed {} triangles", start.elapsed(), root.index(), triangles.len()); 105 | 106 | for &(a,b,c) in triangles.iter() { 107 | input.send(((a,b,c), 1)); 108 | } 109 | 110 | // synchronize with other workers. 111 | let prev = query.time().clone(); 112 | input.close(); 113 | query.advance_to(prev.inner + 1); 114 | while probe.less_than(query.time()) { root.step(); } 115 | forward.index.borrow_mut().merge_to(&prev); 116 | 117 | println!("{:?}\tworker {} loaded index", start.elapsed(), root.index()); 118 | 119 | // number of nodes introduced at a time 120 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 121 | 122 | let mut node = 0; 123 | let mut sent = 0; 124 | 125 | while sent < triangles.len() { 126 | node += batch as u32; 127 | while sent < triangles.len() && triangles[sent].0 < node { 128 | query.send((triangles[sent], ())); 129 | sent += 1; 130 | } 131 | 132 | // advance input and synchronize. 133 | let prev = query.time().clone(); 134 | query.advance_to(prev.inner + 1); 135 | while probe.less_than(query.time()) { root.step(); } 136 | 137 | // merge all of the indices we maintain. 138 | forward.index.borrow_mut().merge_to(&prev); 139 | } 140 | 141 | query.close(); 142 | while root.step() { } 143 | 144 | if inspect { 145 | println!("{:?}\tworker {} complete", start.elapsed(), root.index()); 146 | } 147 | 148 | }).unwrap(); 149 | 150 | let total = if let Ok(lock) = send2.lock() { 151 | *lock 152 | } 153 | else { 0 }; 154 | 155 | if inspect { 156 | println!("elapsed: {:?}\ttotal instances at this process: {:?}", start.elapsed(), total); 157 | } 158 | } 159 | 160 | 161 | fn intersect_and(aaa: &[u32], mut bbb: &[u32], mut func: F) { 162 | 163 | if aaa.len() > bbb.len() { 164 | intersect_and(bbb, aaa, func); 165 | } 166 | else { 167 | if aaa.len() < bbb.len() / 16 { 168 | for &a in aaa.iter() { 169 | bbb = gallop_ge(bbb, &a); 170 | if bbb.len() > 0 && bbb[0] == a { 171 | func(a) 172 | } 173 | } 174 | } 175 | else { 176 | for &a in aaa.iter() { 177 | while bbb.len() > 0 && bbb[0] < a { 178 | bbb = &bbb[1..]; 179 | } 180 | if bbb.len() > 0 && a == bbb[0] { 181 | func(a); 182 | } 183 | } 184 | } 185 | } 186 | } 187 | 188 | #[inline(always)] 189 | pub fn gallop_ge<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 190 | // if empty slice, or already >= element, return 191 | if slice.len() > 0 && &slice[0] < value { 192 | let mut step = 1; 193 | while step < slice.len() && &slice[step] < value { 194 | slice = &slice[step..]; 195 | step = step << 1; 196 | } 197 | 198 | step = step >> 1; 199 | while step > 0 { 200 | if step < slice.len() && &slice[step] < value { 201 | slice = &slice[step..]; 202 | } 203 | step = step >> 1; 204 | } 205 | 206 | slice = &slice[1..]; // advance one, as we always stayed < value 207 | } 208 | 209 | return slice; 210 | } 211 | 212 | // #[inline(always)] 213 | // pub fn gallop_gt<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 214 | // // if empty slice, or already > element, return 215 | // if slice.len() > 0 && &slice[0] <= value { 216 | // let mut step = 1; 217 | // while step < slice.len() && &slice[step] <= value { 218 | // slice = &slice[step..]; 219 | // step = step << 1; 220 | // } 221 | 222 | // step = step >> 1; 223 | // while step > 0 { 224 | // if step < slice.len() && &slice[step] <= value { 225 | // slice = &slice[step..]; 226 | // } 227 | // step = step >> 1; 228 | // } 229 | 230 | // slice = &slice[1..]; // advance one, as we always stayed <= value 231 | // } 232 | 233 | // return slice; 234 | // } -------------------------------------------------------------------------------- /examples/seed.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // used to partition graph loading 28 | let index = root.index(); 29 | let peers = root.peers(); 30 | 31 | // handles to input and probe, but also both indices so we can compact them. 32 | let (mut input, probe, forward, reverse) = root.dataflow::(|builder| { 33 | 34 | // A stream of changes to the set of *triangles*, where a < b < c. 35 | let (graph, dT) = builder.new_input::<((u32, u32, u32), i32)>(); 36 | 37 | // Our query is K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z), T(x,y,z) 38 | // 39 | // This query is technically redundant, because the middle two constraints imply the fourth, 40 | // so let's slim it down to 41 | // 42 | // K4(w,x,y,z) := T(w,x,y), T(w,x,z), T(w,y,z) 43 | // 44 | // This seems like it could be a bit more complicated than triangles, in determining the rules 45 | // for incremental updates. I'm going to write them down first, and we'll see which indices we 46 | // actually need. I'll use A, B, and C for the instances of T above. 47 | // 48 | // dK4dA(w,x,y,z) := dA(w,x,y), B(w,x,z), C(w,y,z) 49 | // dK4dB(w,x,y,z) := dB(w,x,z), A(w,x,y), C(w,y,z) 50 | // dK4dC(w,x,y,z) := dC(w,y,z), A(w,x,y), B(w,x,z) 51 | // 52 | // Looking at this, it seems like we will need 53 | // 54 | // dK4dA : indices on (w,x,_) and (w,_,y) 55 | // dK4dB : indices on (w,x,_) and (w,_,z) 56 | // dK4dC : indices on (w,_,y) and (w,_,z) 57 | // 58 | // All of this seems to boil down to a "forward" and a "reverse" index, just as for triangles, 59 | // but where `w` is always present as part of the key. We just might want the first or second 60 | // field that follows it. 61 | 62 | // create two indices, one "forward" from (a,b) to c, and one "reverse" from (a,c) to b. 63 | let forward = IndexStream::from(|(a,b)| (a + b) as u64, &Vec::new().to_stream(builder), &dT.map(|((a,b,c),wgt)| (((a,b),c),wgt))); 64 | let reverse = IndexStream::from(|(a,b)| (a + b) as u64, &Vec::new().to_stream(builder), &dT.map(|((a,b,c),wgt)| (((a,c),b),wgt))); 65 | 66 | // dK4dA(w,x,y,z) := dA(w,x,y), B(w,x,z), C(w,y,z) 67 | let dK4dA = dT.extend(vec![Box::new(forward.extend_using(|&(w,x,_y)| (w,x), <_ as PartialOrd>::le)), 68 | Box::new(forward.extend_using(|&(w,_x,y)| (w,y), <_ as PartialOrd>::le))]) 69 | .flat_map(|((w,x,y), zs, wgt)| zs.into_iter().map(move |z| ((w,x,y,z),wgt))); 70 | 71 | // dK4dB(w,x,y,z) := dB(w,x,z), A(w,x,y), C(w,y,z) 72 | let dK4dB = dT.extend(vec![Box::new(forward.extend_using(|&(w,x,_z)| (w,x), <_ as PartialOrd>::lt)), 73 | Box::new(reverse.extend_using(|&(w,_x,z)| (w,z), <_ as PartialOrd>::le))]) 74 | .flat_map(|((w,x,z), ys, wgt)| ys.into_iter().map(move |y| ((w,x,y,z),wgt))); 75 | 76 | // dK4dC(w,x,y,z) := dC(w,y,z), A(w,x,y), B(w,x,z) 77 | let dK4dC = dT.extend(vec![Box::new(reverse.extend_using(|&(w,y,_z)| (w,y), <_ as PartialOrd>::lt)), 78 | Box::new(reverse.extend_using(|&(w,_y,z)| (w,z), <_ as PartialOrd>::lt))]) 79 | .flat_map(|((w,y,z), xs, wgt)| xs.into_iter().map(move |x| ((w,x,y,z),wgt))); 80 | 81 | let dK4 = dK4dA.concat(&dK4dB).concat(&dK4dC); 82 | 83 | // if the third argument is "inspect", report triangle counts. 84 | if inspect { 85 | dK4.exchange(|x| (x.0).0 as u64) 86 | // .inspect_batch(|t,x| println!("{:?}: {:?}", t, x)) 87 | .count() 88 | .inspect_batch(move |t,x| println!("{:?}: {:?}", t, x)) 89 | .inspect_batch(move |_,x| { 90 | if let Ok(mut bound) = send.lock() { 91 | *bound += x[0]; 92 | } 93 | }); 94 | } 95 | 96 | (graph, dK4.probe(), forward, reverse) 97 | }); 98 | 99 | // load fragment of input graph into memory to avoid io while running. 100 | let filename = std::env::args().nth(1).unwrap(); 101 | let graph = GraphMap::new(&filename); 102 | 103 | let nodes = graph.nodes(); 104 | let mut triangles = Vec::new(); 105 | 106 | let mut v1 = root.index() as u32; 107 | while v1 < graph.nodes() { 108 | let v1f = graph.forward(v1); 109 | for (index_v2, &v2) in v1f.iter().enumerate() { 110 | intersect_and(&v1f[(index_v2+1)..], graph.forward(v2), |v3| triangles.push((v1 as u32, v2, v3))); 111 | } 112 | v1 += root.peers() as u32; 113 | } 114 | 115 | drop(graph); 116 | 117 | println!("{:?}\tworker {} computed {} triangles", start.elapsed(), root.index(), triangles.len()); 118 | 119 | // synchronize with other workers. 120 | let prev = input.time().clone(); 121 | input.advance_to(prev.inner + 1); 122 | while probe.less_than(input.time()) { root.step(); } 123 | 124 | // number of nodes introduced at a time 125 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 126 | 127 | // start the experiment! 128 | let start = ::std::time::Instant::now(); 129 | 130 | let mut sent = 0; 131 | while sent < triangles.len() { 132 | let to_send = std::cmp::min(batch / root.peers(), triangles.len() - sent); 133 | for off in 0 .. to_send { 134 | input.send((triangles[sent + off], 1)); 135 | } 136 | sent += to_send; 137 | 138 | let prev = input.time().clone(); 139 | input.advance_to(prev.inner + 1); 140 | while probe.less_than(input.time()) { root.step(); } 141 | 142 | // merge all of the indices we maintain. 143 | forward.index.borrow_mut().merge_to(&prev); 144 | reverse.index.borrow_mut().merge_to(&prev); 145 | } 146 | 147 | input.close(); 148 | while root.step() { } 149 | 150 | if inspect { 151 | println!("worker {} elapsed: {:?}", index, start.elapsed()); 152 | } 153 | 154 | }).unwrap(); 155 | 156 | let total = if let Ok(lock) = send2.lock() { 157 | *lock 158 | } 159 | else { 0 }; 160 | 161 | if inspect { 162 | println!("elapsed: {:?}\ttotal triangles at this process: {:?}", start.elapsed(), total); 163 | } 164 | } 165 | 166 | 167 | struct GraphMap { 168 | map: GraphMMap, 169 | reverse: Vec, 170 | } 171 | 172 | 173 | impl GraphMap { 174 | pub fn new(filename: &str) -> Self { 175 | 176 | let map = GraphMMap::new(filename); 177 | 178 | let mut reverse = vec![0; map.nodes()]; 179 | for node in 0 .. map.nodes() { 180 | for &neighbor in map.edges(node) { 181 | if (neighbor as usize) < node { 182 | reverse[node] += 1; 183 | } 184 | if (neighbor as usize) == node { 185 | panic!("self-loop"); 186 | } 187 | } 188 | } 189 | 190 | GraphMap { 191 | map: map, 192 | reverse: reverse, 193 | } 194 | } 195 | 196 | #[inline(always)] 197 | pub fn nodes(&self) -> u32 { self.map.nodes() as u32 } 198 | #[inline(always)] 199 | pub fn edges(&self, node: u32) -> &[u32] { self.map.edges(node as usize) } 200 | #[inline(always)] 201 | pub fn forward(&self, node: u32) -> &[u32] { 202 | &self.edges(node)[(self.reverse[node as usize] as usize)..] 203 | } 204 | } 205 | 206 | fn intersect_and(aaa: &[u32], mut bbb: &[u32], mut func: F) { 207 | 208 | if aaa.len() > bbb.len() { 209 | intersect_and(bbb, aaa, func); 210 | } 211 | else { 212 | if aaa.len() < bbb.len() / 16 { 213 | for &a in aaa.iter() { 214 | bbb = gallop_ge(bbb, &a); 215 | if bbb.len() > 0 && bbb[0] == a { 216 | func(a) 217 | } 218 | } 219 | } 220 | else { 221 | for &a in aaa.iter() { 222 | while bbb.len() > 0 && bbb[0] < a { 223 | bbb = &bbb[1..]; 224 | } 225 | if bbb.len() > 0 && a == bbb[0] { 226 | func(a); 227 | } 228 | } 229 | } 230 | } 231 | } 232 | 233 | #[inline(always)] 234 | pub fn gallop_ge<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 235 | // if empty slice, or already >= element, return 236 | if slice.len() > 0 && &slice[0] < value { 237 | let mut step = 1; 238 | while step < slice.len() && &slice[step] < value { 239 | slice = &slice[step..]; 240 | step = step << 1; 241 | } 242 | 243 | step = step >> 1; 244 | while step > 0 { 245 | if step < slice.len() && &slice[step] < value { 246 | slice = &slice[step..]; 247 | } 248 | step = step >> 1; 249 | } 250 | 251 | slice = &slice[1..]; // advance one, as we always stayed < value 252 | } 253 | 254 | return slice; 255 | } 256 | 257 | #[inline(always)] 258 | pub fn gallop_gt<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 259 | // if empty slice, or already > element, return 260 | if slice.len() > 0 && &slice[0] <= value { 261 | let mut step = 1; 262 | while step < slice.len() && &slice[step] <= value { 263 | slice = &slice[step..]; 264 | step = step << 1; 265 | } 266 | 267 | step = step >> 1; 268 | while step > 0 { 269 | if step < slice.len() && &slice[step] <= value { 270 | slice = &slice[step..]; 271 | } 272 | step = step >> 1; 273 | } 274 | 275 | slice = &slice[1..]; // advance one, as we always stayed <= value 276 | } 277 | 278 | return slice; 279 | } -------------------------------------------------------------------------------- /examples/triangles-lean.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // used to partition graph loading 28 | let index = root.index(); 29 | let peers = root.peers(); 30 | 31 | // handles to input and probe, but also both indices so we can compact them. 32 | let (mut input, mut query, probe, /*forward,*/ reverse) = root.dataflow::(|builder| { 33 | 34 | // Please see triangles for more information on "graph" and dG. 35 | let (graph, dG) = builder.new_input::<((u32, u32), i32)>(); 36 | // Please see triangles for more information on "graph" and dG. 37 | let (query, dQ) = builder.new_input::<((u32, u32), i32)>(); 38 | 39 | // Our query is K3 = A(x,y) B(x,z) C(y,z): triangles. 40 | // 41 | // The dataflow determines how to update this query with respect to changes in each 42 | // of the input relations: A, B, and C. Each partial derivative will use the other 43 | // relations, but the order in which attributes are added may (will) be different. 44 | // 45 | // The updates also use the other relations with slightly stale data: updates to each 46 | // relation must not see updates for "later" relations (under some order on relations). 47 | 48 | // we will index the data both by src and dst. 49 | // let (forward, f_handle) = dG.index_from(&dG.filter(|_| false).map(|_| (0,0))); 50 | let reverse = IndexStream::from(|k| k as u64, 51 | &dG.map(|((x,y),_)| (y,x)), 52 | &Vec::new().to_stream(builder)); 53 | 54 | // let r_handle = reverse.index.clone() 55 | 56 | // let (reverse, r_handle) = dG.filter(|_|false) 57 | // .map(|((src,dst),wgt)| ((dst,src),wgt)) 58 | // .index_from(&dG.map(|((x,y),_)| (y,x)), |&k| k as u64); 59 | 60 | // dC(y,z) extends to x first through A(x,y) then B(x,z), both using reverse indices. 61 | let cliques = dQ.extend(vec![Box::new(reverse.extend_using(|&(y,_)| y, |t1, t2| t1.le(t2))), 62 | Box::new(reverse.extend_using(|&(_,z)| z, |t1, t2| t1.le(t2)))]); 63 | 64 | // if the third argument is "inspect", report triangle counts. 65 | if inspect { 66 | cliques 67 | .inspect_batch(move |_,x| { 68 | if let Ok(mut bound) = send.lock() { 69 | for xx in x.iter() { 70 | *bound += xx.1.len(); 71 | } 72 | } 73 | }); 74 | } 75 | 76 | (graph, query, cliques.probe(), reverse) 77 | }); 78 | 79 | // load fragment of input graph into memory to avoid io while running. 80 | let filename = std::env::args().nth(1).unwrap(); 81 | let graph = GraphMMap::new(&filename); 82 | 83 | let nodes = graph.nodes(); 84 | let mut edges = Vec::new(); 85 | 86 | for node in 0 .. graph.nodes() { 87 | if node % peers == index { 88 | edges.push(graph.edges(node).to_vec()); 89 | } 90 | } 91 | 92 | drop(graph); 93 | 94 | // synchronize with other workers. 95 | let prev = input.time().clone(); 96 | input.advance_to(prev.inner + 1); 97 | query.advance_to(prev.inner + 1); 98 | root.step_while(|| probe.less_than(input.time())); 99 | 100 | // number of nodes introduced at a time 101 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 102 | 103 | // start the experiment! 104 | let start = ::std::time::Instant::now(); 105 | 106 | for node in 0 .. nodes { 107 | 108 | // introduce the node if it is this worker's responsibility 109 | if node % peers == index { 110 | for &edge in &edges[node / peers] { 111 | input.send(((node as u32, edge), 1)); 112 | } 113 | } 114 | } 115 | 116 | let prev = input.time().clone(); 117 | input.advance_to(prev.inner + 1); 118 | query.advance_to(prev.inner + 1); 119 | root.step_while(|| probe.less_than(query.time())); 120 | reverse.index.borrow_mut().merge_to(&prev); 121 | input.close(); 122 | 123 | println!("{:?}: index built", start.elapsed()); 124 | 125 | for node in 0 .. nodes { 126 | 127 | // introduce the node if it is this worker's responsibility 128 | if node % peers == index { 129 | for &edge in &edges[node / peers] { 130 | query.send(((node as u32, edge), 1)); 131 | } 132 | } 133 | 134 | // if at a batch boundary, advance time and do work. 135 | if node % batch == (batch - 1) { 136 | let prev = query.time().clone(); 137 | query.advance_to(prev.inner + 1); 138 | root.step_while(|| probe.less_than(query.time())); 139 | 140 | // merge all of the indices we maintain. 141 | // forward.borrow_mut().merge_to(&prev); 142 | reverse.index.borrow_mut().merge_to(&prev); 143 | } 144 | } 145 | 146 | query.close(); 147 | while root.step() { } 148 | 149 | if inspect { 150 | println!("worker {} elapsed: {:?}", index, start.elapsed()); 151 | } 152 | 153 | }).unwrap(); 154 | 155 | let total = if let Ok(lock) = send2.lock() { 156 | *lock 157 | } 158 | else { 0 }; 159 | 160 | if inspect { 161 | println!("elapsed: {:?}\ttotal triangles at this process: {:?}", start.elapsed(), total); 162 | } 163 | } 164 | -------------------------------------------------------------------------------- /examples/triangles-single.rs: -------------------------------------------------------------------------------- 1 | // extern crate mmap; 2 | // extern crate time; 3 | // extern crate dataflow_join; 4 | 5 | extern crate graph_map; 6 | 7 | use graph_map::GraphMMap; 8 | 9 | fn main () { 10 | if let Some(source) = std::env::args().skip(1).next() { 11 | let timer = ::std::time::Instant::now(); 12 | let count = raw_triangles(&GraphMMap::new(&source)); 13 | println!("{:?}\ttriangles: {:?}", timer.elapsed(), count); 14 | } 15 | else { 16 | println!("usage: "); 17 | } 18 | } 19 | 20 | fn raw_triangles(graph: &GraphMMap) -> u64 { 21 | 22 | let mut count = 0; 23 | for a in 0..graph.nodes() { 24 | if graph.edges(a).len() > 0 { 25 | count += 1; 26 | } 27 | } 28 | 29 | println!("count: {}", count); 30 | 31 | let mut count = 0; 32 | for a in 0..graph.nodes() { 33 | let aaa = graph.edges(a); 34 | for &b in aaa { 35 | let bbb = graph.edges(b as usize); 36 | count += if aaa.len() < bbb.len() { intersect(aaa, bbb) } 37 | else { intersect(bbb, aaa) }; 38 | } 39 | } 40 | count 41 | } 42 | 43 | fn intersect(aaa: &[u32], mut bbb: &[u32]) -> u64 { 44 | let mut count = 0; 45 | // magic gallop overhead # is 4 46 | if aaa.len() < bbb.len() / 4 { 47 | for a in aaa { 48 | bbb = gallop(bbb, a); 49 | if bbb.len() > 0 && &bbb[0] == a { count += 1; } 50 | } 51 | } 52 | else { 53 | for &a in aaa { 54 | while bbb.len() > 0 && bbb[0] < a { 55 | bbb = &bbb[1..]; 56 | } 57 | if bbb.len() > 0 && a == bbb[0] { 58 | count += 1; 59 | } 60 | } 61 | } 62 | count 63 | } 64 | 65 | 66 | #[inline(always)] 67 | pub fn gallop<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 68 | // if empty slice, or already >= element, return 69 | if slice.len() > 0 && &slice[0] < value { 70 | let mut step = 1; 71 | while step < slice.len() && &slice[step] < value { 72 | slice = &slice[step..]; 73 | step = step << 1; 74 | } 75 | 76 | step = step >> 1; 77 | while step > 0 { 78 | if step < slice.len() && &slice[step] < value { 79 | slice = &slice[step..]; 80 | } 81 | step = step >> 1; 82 | } 83 | 84 | slice = &slice[1..]; // advance one, as we always stayed < value 85 | } 86 | 87 | return slice; 88 | } -------------------------------------------------------------------------------- /examples/triangles.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // used to partition graph loading 28 | let index = root.index(); 29 | let peers = root.peers(); 30 | 31 | // handles to input and probe, but also both indices so we can compact them. 32 | let (mut input, probe, forward, reverse) = root.dataflow::(|builder| { 33 | 34 | // Please see triangles for more information on "graph" and dG. 35 | let (graph, dG) = builder.new_input::<((u32, u32), i32)>(); 36 | 37 | // Our query is K3 = A(x,y) B(x,z) C(y,z): triangles. 38 | // 39 | // The dataflow determines how to update this query with respect to changes in each 40 | // of the input relations: A, B, and C. Each partial derivative will use the other 41 | // relations, but the order in which attributes are added may (will) be different. 42 | // 43 | // The updates also use the other relations with slightly stale data: updates to each 44 | // relation must not see updates for "later" relations (under some order on relations). 45 | 46 | let forward = IndexStream::from(|k| k as u64, &Vec::new().to_stream(builder), &dG); 47 | let reverse = IndexStream::from(|k| k as u64, &Vec::new().to_stream(builder), &dG.map(|((src,dst),wgt)| ((dst,src),wgt))); 48 | 49 | // dA(x,y) extends to z first through C(x,z) then B(y,z), both using forward indices. 50 | let dK3dA = dG.extend(vec![Box::new(forward.extend_using(|&(x,_)| x, <_ as PartialOrd>::lt)), 51 | Box::new(forward.extend_using(|&(_,y)| y, <_ as PartialOrd>::lt))]) 52 | .flat_map(|(p,es,w)| es.into_iter().map(move |e| ((p.0,p.1,e), w))); 53 | 54 | // dB(x,z) extends to y first through A(x,y) then C(y,z), using forward and reverse indices, respectively. 55 | let dK3dB = dG.extend(vec![Box::new(forward.extend_using(|&(x,_)| x, <_ as PartialOrd>::le)), 56 | Box::new(reverse.extend_using(|&(_,z)| z, <_ as PartialOrd>::lt))]) 57 | .flat_map(|(p,es,w)| es.into_iter().map(move |e| ((p.0,e,p.1), w))); 58 | 59 | // dC(y,z) extends to x first through A(x,y) then B(x,z), both using reverse indices. 60 | let dK3dC = dG.extend(vec![Box::new(reverse.extend_using(|&(y,_)| y, <_ as PartialOrd>::le)), 61 | Box::new(reverse.extend_using(|&(_,z)| z, <_ as PartialOrd>::le))]) 62 | .flat_map(|(p,es,w)| es.into_iter().map(move |e| ((e,p.0,p.1), w))); 63 | 64 | // accumulate all changes together 65 | let cliques = dK3dC.concat(&dK3dB).concat(&dK3dA); 66 | 67 | // if the third argument is "inspect", report triangle counts. 68 | if inspect { 69 | cliques.exchange(|x| (x.0).0 as u64) 70 | // .inspect_batch(|t,x| println!("{:?}: {:?}", t, x)) 71 | .count() 72 | .inspect_batch(move |t,x| println!("{:?}: {:?}", t, x)) 73 | .inspect_batch(move |_,x| { 74 | if let Ok(mut bound) = send.lock() { 75 | *bound += x[0]; 76 | } 77 | }); 78 | } 79 | 80 | (graph, cliques.probe(), forward, reverse) 81 | }); 82 | 83 | // load fragment of input graph into memory to avoid io while running. 84 | let filename = std::env::args().nth(1).unwrap(); 85 | let graph = GraphMMap::new(&filename); 86 | 87 | let nodes = graph.nodes(); 88 | let mut edges = Vec::new(); 89 | 90 | for node in 0 .. graph.nodes() { 91 | if node % peers == index { 92 | edges.push(graph.edges(node).to_vec()); 93 | } 94 | } 95 | 96 | drop(graph); 97 | 98 | // synchronize with other workers. 99 | let prev = input.time().clone(); 100 | input.advance_to(prev.inner + 1); 101 | root.step_while(|| probe.less_than(input.time())); 102 | 103 | // number of nodes introduced at a time 104 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 105 | 106 | // start the experiment! 107 | let start = ::std::time::Instant::now(); 108 | for node in 0 .. nodes { 109 | 110 | // introduce the node if it is this worker's responsibility 111 | if node % peers == index { 112 | for &edge in &edges[node / peers] { 113 | input.send(((node as u32, edge), 1)); 114 | } 115 | } 116 | 117 | // if at a batch boundary, advance time and do work. 118 | if node % batch == (batch - 1) { 119 | let prev = input.time().clone(); 120 | input.advance_to(prev.inner + 1); 121 | root.step_while(|| probe.less_than(input.time())); 122 | 123 | // merge all of the indices we maintain. 124 | forward.index.borrow_mut().merge_to(&prev); 125 | reverse.index.borrow_mut().merge_to(&prev); 126 | } 127 | } 128 | 129 | input.close(); 130 | while root.step() { } 131 | 132 | if inspect { 133 | println!("worker {} elapsed: {:?}", index, start.elapsed()); 134 | } 135 | 136 | }).unwrap(); 137 | 138 | let total = if let Ok(lock) = send2.lock() { 139 | *lock 140 | } 141 | else { 0 }; 142 | 143 | if inspect { 144 | println!("elapsed: {:?}\ttotal triangles at this process: {:?}", start.elapsed(), total); 145 | } 146 | } 147 | -------------------------------------------------------------------------------- /examples/triangles_updates.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use graph_map::GraphMMap; 12 | 13 | #[allow(non_snake_case)] 14 | fn main () { 15 | 16 | let start = ::std::time::Instant::now(); 17 | 18 | let send = Arc::new(Mutex::new(0)); 19 | let send2 = send.clone(); 20 | 21 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 22 | 23 | timely::execute_from_args(std::env::args(), move |root| { 24 | 25 | let send = send.clone(); 26 | 27 | // used to partition graph loading 28 | let index = root.index(); 29 | let peers = root.peers(); 30 | 31 | // handles to input and probe, but also both indices so we can compact them. 32 | let (mut inputG, mut inputQ, probe, forward, reverse) = root.dataflow::(|builder| { 33 | 34 | // A dynamic graph is a stream of updates: `((src, dst), wgt)`. 35 | // Each triple indicates a change to the count of the number of arcs from 36 | // `src` to `dst`. Typically this change would be +/-1, but whatever. 37 | let (graph, dG) = builder.new_input::<(u32, u32)>(); 38 | let (query, dQ) = builder.new_input::<((u32, u32), i32)>(); 39 | 40 | // Our query is K3 = A(x,y) B(x,z) C(y,z): triangles. 41 | // 42 | // The dataflow determines how to update this query with respect to changes in each 43 | // of the input relations: A, B, and C. Each partial derivative will use the other 44 | // relations, but the order in which attributes are added may (will) be different. 45 | // 46 | // The updates also use the other relations with slightly stale data: updates to each 47 | // relation must not see updates for "later" relations (under some order on relations). 48 | 49 | let forward = IndexStream::from(|k| k as u64, &dG, &dQ); 50 | let reverse = IndexStream::from(|k| k as u64, &dG.map(|(src,dst)| (dst,src)), &dQ.map(|((src,dst),wgt)| ((dst,src),wgt))); 51 | 52 | // dA(x,y) extends to z first through C(x,z) then B(y,z), both using forward indices. 53 | let dK3dA = dQ//.filter(|_| false) 54 | .extend(vec![Box::new(forward.extend_using(|&(x,_)| x, <_ as PartialOrd>::lt)), 55 | Box::new(forward.extend_using(|&(_,y)| y, <_ as PartialOrd>::lt))]) 56 | .flat_map(|(p,es,w)| es.into_iter().map(move |e| ((p.0,p.1,e), w))); 57 | 58 | // dB(x,z) extends to y first through A(x,y) then C(y,z), using forward and reverse indices, respectively. 59 | let dK3dB = dQ//.filter(|_| false) 60 | .extend(vec![Box::new(forward.extend_using(|&(x,_)| x, <_ as PartialOrd>::le)), 61 | Box::new(reverse.extend_using(|&(_,z)| z, <_ as PartialOrd>::lt))]) 62 | .flat_map(|(p,es,w)| es.into_iter().map(move |e| ((p.0,e,p.1), w))); 63 | 64 | // dC(y,z) extends to x first through A(x,y) then B(x,z), both using reverse indices. 65 | let dK3dC = dQ.extend(vec![Box::new(reverse.extend_using(|&(y,_)| y, <_ as PartialOrd>::le)), 66 | Box::new(reverse.extend_using(|&(_,z)| z, <_ as PartialOrd>::le))]) 67 | .flat_map(|(p,es,w)| es.into_iter().map(move |e| ((e,p.0,p.1), w))); 68 | 69 | // accumulate all changes together 70 | let cliques = dK3dC.concat(&dK3dB).concat(&dK3dA); 71 | 72 | // if the third argument is "inspect", report triangle counts. 73 | if inspect { 74 | cliques.exchange(|x| (x.0).0 as u64) 75 | // .inspect_batch(|t,x| println!("{:?}: {:?}", t, x)) 76 | .count() 77 | // .inspect_batch(move |t,x| println!("{:?}: {:?}", t, x)) 78 | .inspect_batch(move |_,x| { 79 | if let Ok(mut bound) = send.lock() { 80 | *bound += x[0]; 81 | } 82 | }); 83 | } 84 | (graph, query, cliques.probe(), forward, reverse) 85 | }); 86 | 87 | // load fragment of input graph into memory to avoid io while running. 88 | let filename = std::env::args().nth(1).unwrap(); 89 | let graph = GraphMMap::new(&filename); 90 | 91 | let nodes = graph.nodes(); 92 | let mut edges = Vec::new(); 93 | 94 | for node in 0 .. graph.nodes() { 95 | if node % peers == index { 96 | edges.push(graph.edges(node).to_vec()); 97 | } 98 | } 99 | 100 | drop(graph); 101 | 102 | // synchronize with other workers. 103 | let prevG = inputG.time().clone(); 104 | inputG.advance_to(prevG.inner + 1); 105 | inputQ.advance_to(prevG.inner + 1); 106 | root.step_while(|| probe.less_than(inputG.time())); 107 | 108 | // number of nodes introduced at a time 109 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 110 | 111 | // start the experiment! 112 | let start = ::std::time::Instant::now(); 113 | let limit = (95 * nodes /100) as usize ; 114 | 115 | for node in 0 .. limit { 116 | if node % peers == index { 117 | for &edge in &edges[node / peers] { 118 | inputG.send((node as u32, edge)); 119 | } 120 | } 121 | } 122 | 123 | let prevG = inputG.time().clone(); 124 | inputG.advance_to(prevG.inner + 1); 125 | inputQ.advance_to(prevG.inner + 1); 126 | root.step_while(|| probe.less_than(inputG.time())); 127 | 128 | if inspect { 129 | println!("{:?}\t[worker {}]\tdata loaded", start.elapsed(), index); 130 | } 131 | 132 | // merge all of the indices we maintain. 133 | let prevG = inputG.time().clone(); 134 | forward.index.borrow_mut().merge_to(&prevG); 135 | reverse.index.borrow_mut().merge_to(&prevG); 136 | 137 | if inspect { 138 | println!("{:?}\t[worker {}]\tindices merged", start.elapsed(), index); 139 | } 140 | 141 | let prevG = inputG.time().clone(); 142 | inputG.advance_to(prevG.inner + 1); 143 | inputQ.advance_to(prevG.inner + 1); 144 | root.step_while(|| probe.less_than(inputG.time())); 145 | 146 | for node in limit .. nodes { 147 | 148 | if node % peers == index { 149 | for &edge in &edges[node / peers] { 150 | inputQ.send(((node as u32, edge), 1)); 151 | } 152 | } 153 | 154 | // advance the graph stream (only useful in the first time) 155 | let prevG = inputG.time().clone(); 156 | inputG.advance_to(prevG.inner + 1); 157 | 158 | if node % batch == (batch - 1) { 159 | let prev = inputQ.time().clone(); 160 | inputQ.advance_to(prev.inner + 1); 161 | root.step_while(|| probe.less_than(inputQ.time())); 162 | 163 | // merge all of the indices we maintain. 164 | forward.index.borrow_mut().merge_to(&prev); 165 | reverse.index.borrow_mut().merge_to(&prev); 166 | } 167 | } 168 | 169 | inputG.close(); 170 | inputQ.close(); 171 | while root.step() { } 172 | 173 | if inspect { 174 | println!("{:?}\t[worker {}]\tcomplete", start.elapsed(), index); 175 | } 176 | 177 | }).unwrap(); 178 | 179 | let total = if let Ok(lock) = send2.lock() { 180 | *lock 181 | } 182 | else { 0 }; 183 | 184 | if inspect { 185 | println!("elapsed: {:?}\ttotal triangles at this process: {:?}", start.elapsed(), total); 186 | } 187 | } 188 | -------------------------------------------------------------------------------- /examples/triangles_updates_edges.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate graph_map; 3 | extern crate alg3_dynamic; 4 | 5 | use std::sync::{Arc, Mutex}; 6 | 7 | use alg3_dynamic::*; 8 | 9 | use timely::dataflow::operators::*; 10 | 11 | use std::io::BufReader; 12 | use std::error::Error; 13 | use std::fs::File; 14 | use std::io::prelude::*; 15 | use std::path::Path; 16 | 17 | #[allow(non_snake_case)] 18 | fn main () { 19 | 20 | let start = ::std::time::Instant::now(); 21 | 22 | let send = Arc::new(Mutex::new(0)); 23 | let send2 = send.clone(); 24 | 25 | let inspect = ::std::env::args().find(|x| x == "inspect").is_some(); 26 | 27 | timely::execute_from_args(std::env::args(), move |root| { 28 | 29 | let send = send.clone(); 30 | 31 | // used to partition graph loading 32 | let index = root.index() as u32; 33 | let peers = root.peers() as u32; 34 | 35 | // handles to input and probe, but also both indices so we can compact them. 36 | let (mut inputG, mut inputQ, probe, forward, reverse) = root.dataflow::(|builder| { 37 | 38 | // A dynamic graph is a stream of updates: `((src, dst), wgt)`. 39 | // Each triple indicates a change to the count of the number of arcs from 40 | // `src` to `dst`. Typically this change would be +/-1, but whatever. 41 | let (graph, dG) = builder.new_input::<(u32, u32)>(); 42 | let (query, dQ) = builder.new_input::<((u32, u32), i32)>(); 43 | 44 | // Our query is K3 = A(x,y) B(x,z) C(y,z): triangles. 45 | // 46 | // The dataflow determines how to update this query with respect to changes in each 47 | // of the input relations: A, B, and C. Each partial derivative will use the other 48 | // relations, but the order in which attributes are added may (will) be different. 49 | // 50 | // The updates also use the other relations with slightly stale data: updates to each 51 | // relation must not see updates for "later" relations (under some order on relations). 52 | 53 | // we will index the data both by src and dst. 54 | let forward = IndexStream::from(|k| k as u64, &dG, &dQ); 55 | let reverse = IndexStream::from(|k| k as u64, &dG.map(|(src,dst)| (dst,src)), 56 | &dQ.map(|((src,dst),wgt)| ((dst,src),wgt))); 57 | 58 | // dA(x,y) extends to z first through C(x,z) then B(y,z), both using forward indices. 59 | let dK3dA = dQ//.filter(|_| false) 60 | .extend(vec![Box::new(forward.extend_using(|&(x,_)| x, <_ as PartialOrd>::lt)), 61 | Box::new(forward.extend_using(|&(_,y)| y, <_ as PartialOrd>::lt))]) 62 | .flat_map(|(p,es,w)| es.into_iter().map(move |e| ((p.0,p.1,e), w))); 63 | 64 | // dB(x,z) extends to y first through A(x,y) then C(y,z), using forward and reverse indices, respectively. 65 | let dK3dB = dQ//.filter(|_| false) 66 | .extend(vec![Box::new(forward.extend_using(|&(x,_)| x, <_ as PartialOrd>::le)), 67 | Box::new(reverse.extend_using(|&(_,z)| z, <_ as PartialOrd>::lt))]) 68 | .flat_map(|(p,es,w)| es.into_iter().map(move |e| ((p.0,e,p.1), w))); 69 | 70 | // dC(y,z) extends to x first through A(x,y) then B(x,z), both using reverse indices. 71 | let dK3dC = dQ.extend(vec![Box::new(reverse.extend_using(|&(y,_)| y, <_ as PartialOrd>::le)), 72 | Box::new(reverse.extend_using(|&(_,z)| z, <_ as PartialOrd>::le))]) 73 | .flat_map(|(p,es,w)| es.into_iter().map(move |e| ((e,p.0,p.1), w))); 74 | 75 | // accumulate all changes together 76 | let cliques = dK3dC.concat(&dK3dB).concat(&dK3dA); 77 | 78 | // if the third argument is "inspect", report triangle counts. 79 | if inspect { 80 | cliques.exchange(|x| (x.0).0 as u64) 81 | // .inspect_batch(|t,x| println!("{:?}: {:?}", t, x)) 82 | .count() 83 | // .inspect_batch(move |t,x| println!("{:?}: {:?}", t, x)) 84 | .inspect_batch(move |_,x| { 85 | if let Ok(mut bound) = send.lock() { 86 | *bound += x[0]; 87 | } 88 | }); 89 | } 90 | (graph, query, cliques.probe(), forward, reverse) 91 | }); 92 | 93 | // load fragment of input graph into memory to avoid io while running. 94 | let filename = std::env::args().nth(1).unwrap(); 95 | // load percentage out of 100 96 | let percent: usize = std::env::args().nth(3).unwrap().parse().unwrap(); 97 | 98 | 99 | let input_graph = read_edges(&filename, peers, index); 100 | let graphSize: usize = std::env::args().nth(4).unwrap().parse().unwrap(); 101 | let limit = (percent * graphSize /100 / peers as usize) as usize ; 102 | /* 103 | let graph = GraphMMap::new(&filename); 104 | let nodes = graph.nodes(); 105 | let mut edges = Vec::new(); 106 | for node in 0 .. graph.nodes() { 107 | if node % peers == index { 108 | edges.push(graph.edges(node).to_vec()); 109 | } 110 | } 111 | 112 | drop(graph); 113 | */ 114 | 115 | let mut edges = Vec::new(); 116 | let mut edgesQ = Vec::new(); 117 | for e in 0 .. input_graph.len() { 118 | if e <= limit { 119 | edges.push(input_graph[e]); 120 | } 121 | else { 122 | edgesQ.push(input_graph[e]); 123 | } 124 | } 125 | 126 | drop(input_graph); 127 | 128 | // synchronize with other workers. 129 | let prevG = inputG.time().clone(); 130 | inputG.advance_to(prevG.inner + 1); 131 | inputQ.advance_to(prevG.inner + 1); 132 | root.step_while(|| probe.less_than(inputG.time())); 133 | 134 | // number of nodes introduced at a time 135 | let batch: usize = std::env::args().nth(2).unwrap().parse().unwrap(); 136 | 137 | // start the experiment! 138 | let start = ::std::time::Instant::now(); 139 | 140 | // load graph to data flow 141 | for e in 0 .. edges.len() { 142 | inputG.send(edges[e]); 143 | } 144 | 145 | let prevG = inputG.time().clone(); 146 | inputG.advance_to(prevG.inner + 1); 147 | inputQ.advance_to(prevG.inner + 1); 148 | root.step_while(|| probe.less_than(inputG.time())); 149 | 150 | if inspect { 151 | println!("{:?}\t[worker {}]\tdata loaded", start.elapsed(), index); 152 | } 153 | 154 | // merge all of the indices we maintain. 155 | let prevG = inputG.time().clone(); 156 | forward.index.borrow_mut().merge_to(&prevG); 157 | reverse.index.borrow_mut().merge_to(&prevG); 158 | 159 | if inspect { 160 | println!("{:?}\t[worker {}]\tindices merged", start.elapsed(), index); 161 | } 162 | 163 | let prevG = inputG.time().clone(); 164 | inputG.advance_to(prevG.inner + 1); 165 | inputQ.advance_to(prevG.inner + 1); 166 | root.step_while(|| probe.less_than(inputG.time())); 167 | 168 | let mut counter = 0 as usize; 169 | for e in 0 .. edgesQ.len() { 170 | 171 | inputQ.send((edgesQ[e], 1)); 172 | counter += 1; 173 | 174 | // advance the graph stream (only useful in the first time) 175 | // should I check if counter == 1 before we do this step ! 176 | let prevG = inputG.time().clone(); 177 | inputG.advance_to(prevG.inner + 1); 178 | 179 | if counter % batch == (batch - 1) { 180 | let prev = inputQ.time().clone(); 181 | inputQ.advance_to(prev.inner + 1); 182 | root.step_while(|| probe.less_than(inputQ.time())); 183 | 184 | // merge all of the indices we maintain. 185 | forward.index.borrow_mut().merge_to(&prev); 186 | reverse.index.borrow_mut().merge_to(&prev); 187 | } 188 | } 189 | 190 | inputG.close(); 191 | inputQ.close(); 192 | while root.step() { } 193 | 194 | if inspect { 195 | println!("{:?}\t[worker {}]\tcomplete", start.elapsed(), index); 196 | } 197 | 198 | }).unwrap(); 199 | 200 | let total = if let Ok(lock) = send2.lock() { 201 | *lock 202 | } 203 | else { 5 }; 204 | 205 | if inspect { 206 | println!("elapsed: {:?}\ttotal triangles at this process: {:?}", start.elapsed(), total); 207 | } 208 | } 209 | 210 | 211 | /// Reads a list of edges from a file and puts them into an array of (u32, u32). 212 | fn read_edges(filename: &str, peers: u32, index:u32) -> Vec<(u32, u32)> { 213 | // Create a path to the desired file 214 | let path = Path::new(filename); 215 | let display = path.display(); 216 | 217 | // Open the path in read-only mode, returns `io::Result` 218 | let file = match File::open(&path) { 219 | // The `description` method of `io::Error` returns a string that describes the error 220 | Err(why) => { 221 | panic!("EXCEPTION: couldn't open {}: {}", 222 | display, 223 | Error::description(&why)) 224 | } 225 | Ok(file) => file, 226 | }; 227 | 228 | // Collect all lines into a vector 229 | let reader = BufReader::new(file); 230 | // graph is a vector of tuples. 231 | let mut graph = Vec::new(); 232 | for line in reader.lines() { 233 | let good_line = line.ok().expect("EXCEPTION: read error"); 234 | if !good_line.starts_with('#') && good_line.len() > 0 { 235 | let elts: Vec<&str> = good_line[..].split_whitespace().collect(); 236 | let src: u32 = elts[0].parse().ok().expect("malformed src"); 237 | let dst: u32 = elts[1].parse().ok().expect("malformed dst"); 238 | if src % peers == index { 239 | graph.push((src, dst)); 240 | } 241 | } 242 | } 243 | return graph; 244 | } 245 | -------------------------------------------------------------------------------- /naive/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "naive" 3 | version = "0.1.0" 4 | authors = ["Frank McSherry "] 5 | 6 | [dependencies] 7 | graph_map = { git = "https://github.com/frankmcsherry/graph-map" } 8 | timely = "0.6" -------------------------------------------------------------------------------- /naive/README.md: -------------------------------------------------------------------------------- 1 | ## naive 2 | 3 | Naive parallel implementations of graph pattern mining 4 | 5 | --- 6 | 7 | These are naive [timely dataflow](https://github.com/frankmcsherry/timely-dataflow) implementations of several graph mining queries. To execute them, you will need graphs in the [graph-map](https://github.com/frankmcsherry/graph-map) format, and to get correct answers you will want to make sure that the graphs are symmetric (have the same edges in forward and reverse direction). 8 | 9 | For example, using the relatively small [LiveJournal graph](http://snap.stanford.edu/data/soc-LiveJournal1.html) to perform query 3 (which counts four-cliques) one might type: 10 | 11 | frankmc@fdr1:~/dataflow-join/naive$ cargo run --release --bin q3 -- ~/lj 12 | Finished release [optimized] target(s) in 0.0 secs 13 | Running `target/release/q3 /home/frankmc/lj` 14 | Duration { secs: 64, nanos: 839351740 } worker 0/1: count: 9933532019 15 | frankmc@fdr1:~/dataflow-join/naive$ 16 | 17 | You can use standard timely dataflow arguments here, for example increasing the number of worker threads like so: 18 | 19 | frankmc@fdr1:~/dataflow-join/naive$ cargo run --release --bin q3 -- ~/lj -w16 20 | Finished release [optimized] target(s) in 0.0 secs 21 | Running `target/release/q3 /home/frankmc/lj -w16` 22 | Duration { secs: 4, nanos: 202392818 } worker 1/16: count: 584181375 23 | Duration { secs: 4, nanos: 325044206 } worker 12/16: count: 564532287 24 | Duration { secs: 4, nanos: 334431840 } worker 10/16: count: 596933184 25 | Duration { secs: 4, nanos: 384298574 } worker 15/16: count: 583504168 26 | Duration { secs: 4, nanos: 396689252 } worker 0/16: count: 593037511 27 | Duration { secs: 4, nanos: 399859841 } worker 13/16: count: 620514577 28 | Duration { secs: 4, nanos: 427228653 } worker 11/16: count: 618154900 29 | Duration { secs: 4, nanos: 437146125 } worker 14/16: count: 597109329 30 | Duration { secs: 4, nanos: 456128426 } worker 3/16: count: 624909578 31 | Duration { secs: 4, nanos: 464649594 } worker 9/16: count: 654328036 32 | Duration { secs: 4, nanos: 509829896 } worker 2/16: count: 586789630 33 | Duration { secs: 4, nanos: 508653220 } worker 6/16: count: 636120221 34 | Duration { secs: 4, nanos: 560799953 } worker 5/16: count: 665648136 35 | Duration { secs: 4, nanos: 568182112 } worker 8/16: count: 646472177 36 | Duration { secs: 4, nanos: 662570020 } worker 7/16: count: 684003005 37 | Duration { secs: 4, nanos: 768076006 } worker 4/16: count: 677293905 38 | frankmc@fdr1:~/dataflow-join/naive$ 39 | 40 | You can also involve multiple processes, using standard timely dataflow machinery, like so: 41 | 42 | frankmc@fdr1:~/dataflow-join/naive$ cargo run --release --bin q3 -- ~/lj -w16 -h ~/fdr-hosts.txt -n4 -p0 43 | Finished release [optimized] target(s) in 0.0 secs 44 | Running `target/release/q3 /home/frankmc/lj -w16 -h /home/frankmc/fdr-hosts.txt -n4 -p0` 45 | Duration { secs: 1, nanos: 180489839 } worker 1/64: count: 143160851 46 | Duration { secs: 1, nanos: 194260985 } worker 15/64: count: 138804302 47 | Duration { secs: 1, nanos: 204388927 } worker 2/64: count: 131297752 48 | Duration { secs: 1, nanos: 208143406 } worker 14/64: count: 142389947 49 | Duration { secs: 1, nanos: 212073494 } worker 0/64: count: 150057695 50 | Duration { secs: 1, nanos: 241199365 } worker 9/64: count: 146856171 51 | Duration { secs: 1, nanos: 267122334 } worker 12/64: count: 155727439 52 | Duration { secs: 1, nanos: 281714899 } worker 5/64: count: 160704569 53 | Duration { secs: 1, nanos: 285222571 } worker 11/64: count: 164154660 54 | Duration { secs: 1, nanos: 287690930 } worker 6/64: count: 151420814 55 | Duration { secs: 1, nanos: 298658553 } worker 10/64: count: 157484735 56 | Duration { secs: 1, nanos: 307072383 } worker 8/64: count: 151645014 57 | Duration { secs: 1, nanos: 316065464 } worker 4/64: count: 173606961 58 | Duration { secs: 1, nanos: 320209615 } worker 13/64: count: 159364814 59 | Duration { secs: 1, nanos: 327837654 } worker 3/64: count: 164525353 60 | Duration { secs: 1, nanos: 387238398 } worker 7/64: count: 171146505 61 | frankmc@fdr1:~/dataflow-join/naive$ 62 | 63 | For that last one to work out, you would have also needed to run 64 | 65 | frankmc@fdr2:~/dataflow-join/naive$ cargo run --release --bin q3 -- ~/lj -w16 -h ~/fdr-hosts.txt -n4 -p1 66 | frankmc@fdr3:~/dataflow-join/naive$ cargo run --release --bin q3 -- ~/lj -w16 -h ~/fdr-hosts.txt -n4 -p2 67 | frankmc@fdr4:~/dataflow-join/naive$ cargo run --release --bin q3 -- ~/lj -w16 -h ~/fdr-hosts.txt -n4 -p3 68 | 69 | on some other machines. -------------------------------------------------------------------------------- /naive/src/bin/q0.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate naive; 3 | 4 | use naive::{GraphMap, intersect_and}; 5 | 6 | fn main () { 7 | 8 | let filename = std::env::args().nth(1).unwrap(); 9 | 10 | timely::execute_from_args(std::env::args(), move |root| { 11 | 12 | let timer = std::time::Instant::now(); 13 | let index = root.index() as u32; 14 | let peers = root.peers() as u32; 15 | let graph = GraphMap::new(&filename); 16 | 17 | let mut count: usize = 0; 18 | let mut v1 = index; 19 | while v1 < graph.nodes() { 20 | let v1f = graph.forward(v1); 21 | for (index_v2, &v2) in v1f.iter().enumerate() { 22 | intersect_and(&v1f[(index_v2+1)..], graph.forward(v2), |v3| if v3 != u32::max_value() { count += 1 }); 23 | } 24 | v1 += peers; 25 | } 26 | println!("{:?}\tworker {:?}/{:?}:\tcount: {:?}", timer.elapsed(), index, peers, count); 27 | 28 | }).unwrap(); 29 | } -------------------------------------------------------------------------------- /naive/src/bin/q1.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate naive; 3 | 4 | use naive::{GraphMap, gallop_gt, intersect_and}; 5 | 6 | fn main () { 7 | 8 | let filename = std::env::args().nth(1).unwrap(); 9 | 10 | timely::execute_from_args(std::env::args().skip(1), move |root| { 11 | 12 | let timer = std::time::Instant::now(); 13 | let index = root.index() as u32; 14 | let peers = root.peers() as u32; 15 | let graph = GraphMap::new(&filename); 16 | 17 | let mut count: usize = 0; 18 | let mut v1 = index; 19 | while v1 < graph.nodes() { 20 | let v1f = graph.forward(v1); 21 | for (index_v2, &v2) in v1f.iter().enumerate() { 22 | let v2e = gallop_gt(graph.edges(v2), &v1); 23 | for &v4 in v1f[(index_v2 + 1)..].iter() { 24 | let v4e = gallop_gt(graph.edges(v4), &v1); 25 | intersect_and(v2e, v4e, |v3| if v3 != u32::max_value() { count += 1 }); 26 | } 27 | } 28 | v1 += peers; 29 | } 30 | println!("{:?}\tworker {:?}/{:?}:\tcount: {:?}", timer.elapsed(), index, peers, count); 31 | 32 | }).unwrap(); 33 | } -------------------------------------------------------------------------------- /naive/src/bin/q2-star.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate naive; 3 | 4 | use naive::{GraphMap, intersect_and}; 5 | 6 | fn main () { 7 | 8 | let filename = std::env::args().nth(1).unwrap(); 9 | 10 | timely::execute_from_args(std::env::args(), move |root| { 11 | 12 | let timer = std::time::Instant::now(); 13 | let index = root.index() as u32; 14 | let peers = root.peers() as u32; 15 | let graph = GraphMap::new(&filename); 16 | 17 | let mut count: usize = 0; 18 | let mut prefix = Vec::new(); 19 | let mut v1 = index; 20 | while v1 < graph.nodes() { 21 | let v1f = graph.forward(v1); 22 | for &v3 in v1f.iter() { 23 | intersect_and(graph.edges(v1), graph.edges(v3), |x| prefix.push(x)); 24 | count += (prefix.len() * (prefix.len() - 1)) / 2; 25 | prefix.clear(); 26 | } 27 | v1 += peers; 28 | } 29 | println!("{:?}\tworker {:?}/{:?}:\tcount: {:?}", timer.elapsed(), index, peers, count); 30 | 31 | }).unwrap(); 32 | } -------------------------------------------------------------------------------- /naive/src/bin/q2.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate naive; 3 | 4 | use naive::{GraphMap, intersect_and}; 5 | 6 | fn main () { 7 | 8 | let filename = std::env::args().nth(1).unwrap(); 9 | 10 | timely::execute_from_args(std::env::args(), move |root| { 11 | 12 | let timer = std::time::Instant::now(); 13 | let index = root.index() as u32; 14 | let peers = root.peers() as u32; 15 | let graph = GraphMap::new(&filename); 16 | 17 | let mut count: usize = 0; 18 | let mut prefix = Vec::new(); 19 | let mut v1 = index; 20 | while v1 < graph.nodes() { 21 | let v1f = graph.forward(v1); 22 | for &v3 in v1f.iter() { 23 | intersect_and(graph.edges(v1), graph.edges(v3), |x| prefix.push(x)); 24 | for (index_v2, &_v2) in prefix.iter().enumerate() { 25 | for &v4 in prefix[(index_v2 + 1)..].iter() { 26 | if _v2 != u32::max_value() && v4 != u32::max_value() { count += 1 } 27 | } 28 | } 29 | prefix.clear(); 30 | } 31 | v1 += peers; 32 | } 33 | println!("{:?}\tworker {:?}/{:?}:\tcount: {:?}", timer.elapsed(), index, peers, count); 34 | 35 | }).unwrap(); 36 | } -------------------------------------------------------------------------------- /naive/src/bin/q3.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate naive; 3 | 4 | use naive::{GraphMap, intersect_and}; 5 | 6 | fn main () { 7 | 8 | let filename = std::env::args().nth(1).unwrap(); 9 | 10 | timely::execute_from_args(std::env::args(), move |root| { 11 | 12 | let timer = std::time::Instant::now(); 13 | let index = root.index() as u32; 14 | let peers = root.peers() as u32; 15 | let graph = GraphMap::new(&filename); 16 | 17 | let mut count: usize = 0; 18 | let mut prefix = Vec::new(); 19 | let mut v1 = index; 20 | while v1 < graph.nodes() { 21 | let v1f = graph.forward(v1); 22 | for (index_v2, &v2) in v1f.iter().enumerate() { 23 | intersect_and(&v1f[(index_v2 + 1)..], graph.forward(v2), |v3| prefix.push(v3)); 24 | for (index_v3, &v3) in prefix.iter().enumerate() { 25 | intersect_and(&prefix[(index_v3 + 1)..], graph.forward(v3), |_v4| count += 1 ); 26 | } 27 | prefix.clear(); 28 | } 29 | v1 += peers; 30 | } 31 | println!("{:?}\tworker {:?}/{:?}:\tcount: {:?}", timer.elapsed(), index, peers, count); 32 | 33 | }).unwrap(); 34 | } -------------------------------------------------------------------------------- /naive/src/bin/q5-star.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate naive; 3 | 4 | use naive::{GraphMap, intersect_and}; 5 | 6 | fn main () { 7 | 8 | let filename = std::env::args().nth(1).unwrap(); 9 | 10 | timely::execute_from_args(std::env::args(), move |root| { 11 | 12 | let timer = std::time::Instant::now(); 13 | let index = root.index() as u32; 14 | let peers = root.peers() as u32; 15 | let graph = GraphMap::new(&filename); 16 | 17 | let count: usize = 0; 18 | let mut tris = Vec::new(); 19 | let mut v1 = index; 20 | while v1 < graph.nodes() { 21 | let v1e = graph.edges(v1); 22 | for (index_b, &b) in v1e.iter().enumerate() { 23 | intersect_and(&v1e[(index_b + 1)..], graph.forward(b), |c| tris.push((b, c))); 24 | } 25 | // enumerate 4-paths in `tris`. 26 | tris.clear(); 27 | v1 += peers; 28 | } 29 | println!("{:?}\tworker {:?}/{:?}:\tcount: {:?}", timer.elapsed(), index, peers, count); 30 | 31 | }).unwrap(); 32 | } -------------------------------------------------------------------------------- /naive/src/bin/q6-star.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate naive; 3 | 4 | use naive::{GraphMap, intersect_and}; 5 | 6 | fn main () { 7 | 8 | let filename = std::env::args().nth(1).unwrap(); 9 | 10 | timely::execute_from_args(std::env::args(), move |root| { 11 | 12 | let timer = std::time::Instant::now(); 13 | let index = root.index() as u32; 14 | let peers = root.peers() as u32; 15 | let graph = GraphMap::new(&filename); 16 | 17 | let mut count: usize = 0; 18 | let mut prefix = Vec::new(); 19 | let mut v2 = index; 20 | while v2 < graph.nodes() { 21 | let v2f = graph.forward(v2); 22 | for &v5 in v2f.iter() { 23 | intersect_and(graph.edges(v2), graph.edges(v5), |x| prefix.push(x)); 24 | for (index_v3, &v3) in prefix.iter().enumerate() { 25 | intersect_and(&prefix[(index_v3 + 1)..], graph.forward(v3), |_v4| { 26 | count += prefix.len() - 2; 27 | }); 28 | } 29 | prefix.clear(); 30 | } 31 | v2 += peers; 32 | } 33 | println!("{:?}\tworker {:?}/{:?}:\tcount: {:?}", timer.elapsed(), index, peers, count); 34 | 35 | }).unwrap(); 36 | } -------------------------------------------------------------------------------- /naive/src/bin/q6.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate naive; 3 | 4 | use naive::{GraphMap, intersect_and}; 5 | 6 | fn main () { 7 | 8 | let filename = std::env::args().nth(1).unwrap(); 9 | 10 | timely::execute_from_args(std::env::args(), move |root| { 11 | 12 | let timer = std::time::Instant::now(); 13 | let index = root.index() as u32; 14 | let peers = root.peers() as u32; 15 | let graph = GraphMap::new(&filename); 16 | 17 | let mut count: usize = 0; 18 | let mut prefix = Vec::new(); 19 | let mut v2 = index; 20 | while v2 < graph.nodes() { 21 | let v2f = graph.forward(v2); 22 | for &v5 in v2f.iter() { 23 | intersect_and(graph.edges(v2), graph.edges(v5), |x| prefix.push(x)); 24 | for (index_v3, &v3) in prefix.iter().enumerate() { 25 | intersect_and(&prefix[(index_v3 + 1)..], graph.forward(v3), |v4| { 26 | for &v5 in prefix.iter() { 27 | if v5 != v3 && v5 != v4 { 28 | count += 1; 29 | } 30 | } 31 | }); 32 | } 33 | prefix.clear(); 34 | } 35 | v2 += peers; 36 | } 37 | println!("{:?}\tworker {:?}/{:?}:\tcount: {:?}", timer.elapsed(), index, peers, count); 38 | 39 | }).unwrap(); 40 | } -------------------------------------------------------------------------------- /naive/src/bin/q7.rs: -------------------------------------------------------------------------------- 1 | extern crate timely; 2 | extern crate naive; 3 | 4 | use naive::{GraphMap, intersect_and}; 5 | 6 | fn main () { 7 | 8 | let filename = std::env::args().nth(1).unwrap(); 9 | 10 | timely::execute_from_args(std::env::args(), move |root| { 11 | 12 | let timer = std::time::Instant::now(); 13 | let index = root.index() as u32; 14 | let peers = root.peers() as u32; 15 | let graph = GraphMap::new(&filename); 16 | 17 | let mut count: usize = 0; 18 | let mut prefix1 = Vec::new(); 19 | let mut prefix2 = Vec::new(); 20 | let mut v1 = index; 21 | while v1 < graph.nodes() { 22 | let v1f = graph.forward(v1); 23 | for (index_v2, &v2) in v1f.iter().enumerate() { 24 | intersect_and(&v1f[(index_v2 + 1)..], graph.forward(v2), |v3| prefix1.push(v3)); 25 | for (index_v3, &v3) in prefix1.iter().enumerate() { 26 | intersect_and(&prefix1[(index_v3 + 1)..], graph.forward(v3), |v4| prefix2.push(v4)); 27 | for (index_v4, &v4) in prefix2.iter().enumerate() { 28 | intersect_and(&prefix2[(index_v4 + 1)..], graph.forward(v4), |v5| if v5 != u32::max_value() { count += 1 }); 29 | } 30 | prefix2.clear(); 31 | } 32 | prefix1.clear(); 33 | } 34 | v1 += peers; 35 | } 36 | println!("{:?}\tworker {:?}/{:?}:\tcount: {:?}", timer.elapsed(), index, peers, count); 37 | 38 | }).unwrap(); 39 | } -------------------------------------------------------------------------------- /naive/src/lib.rs: -------------------------------------------------------------------------------- 1 | extern crate graph_map; 2 | 3 | pub struct GraphMap { 4 | map: graph_map::GraphMMap, 5 | reverse: Vec, 6 | } 7 | 8 | impl GraphMap { 9 | pub fn new(filename: &str) -> Self { 10 | 11 | let map = graph_map::GraphMMap::new(filename); 12 | 13 | let mut reverse = vec![0; map.nodes()]; 14 | for node in 0 .. map.nodes() { 15 | for &neighbor in map.edges(node) { 16 | if (neighbor as usize) < node { 17 | reverse[node] += 1; 18 | } 19 | if (neighbor as usize) == node { 20 | // panic!("self-loop"); 21 | } 22 | } 23 | } 24 | 25 | GraphMap { 26 | map: map, 27 | reverse: reverse, 28 | } 29 | } 30 | 31 | #[inline(always)] 32 | pub fn nodes(&self) -> u32 { self.map.nodes() as u32 } 33 | #[inline(always)] 34 | pub fn edges(&self, node: u32) -> &[u32] { self.map.edges(node as usize) } 35 | #[inline(always)] 36 | pub fn forward(&self, node: u32) -> &[u32] { 37 | &self.edges(node)[(self.reverse[node as usize] as usize)..] 38 | } 39 | } 40 | 41 | pub fn intersect_and(aaa: &[u32], mut bbb: &[u32], mut func: F) { 42 | 43 | if aaa.len() > bbb.len() { 44 | intersect_and(bbb, aaa, func); 45 | } 46 | else { 47 | if aaa.len() < bbb.len() / 16 { 48 | for &a in aaa.iter() { 49 | bbb = gallop_ge(bbb, &a); 50 | if bbb.len() > 0 && bbb[0] == a { 51 | func(a) 52 | } 53 | } 54 | } 55 | else { 56 | for &a in aaa.iter() { 57 | while bbb.len() > 0 && bbb[0] < a { 58 | bbb = &bbb[1..]; 59 | } 60 | if bbb.len() > 0 && a == bbb[0] { 61 | func(a); 62 | } 63 | } 64 | } 65 | } 66 | } 67 | 68 | 69 | #[inline(always)] 70 | pub fn gallop_ge<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 71 | // if empty slice, or already >= element, return 72 | if slice.len() > 0 && &slice[0] < value { 73 | let mut step = 1; 74 | while step < slice.len() && &slice[step] < value { 75 | slice = &slice[step..]; 76 | step = step << 1; 77 | } 78 | 79 | step = step >> 1; 80 | while step > 0 { 81 | if step < slice.len() && &slice[step] < value { 82 | slice = &slice[step..]; 83 | } 84 | step = step >> 1; 85 | } 86 | 87 | slice = &slice[1..]; // advance one, as we always stayed < value 88 | } 89 | 90 | return slice; 91 | } 92 | 93 | #[inline(always)] 94 | pub fn gallop_gt<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 95 | // if empty slice, or already > element, return 96 | if slice.len() > 0 && &slice[0] <= value { 97 | let mut step = 1; 98 | while step < slice.len() && &slice[step] <= value { 99 | slice = &slice[step..]; 100 | step = step << 1; 101 | } 102 | 103 | step = step >> 1; 104 | while step > 0 { 105 | if step < slice.len() && &slice[step] <= value { 106 | slice = &slice[step..]; 107 | } 108 | step = step >> 1; 109 | } 110 | 111 | slice = &slice[1..]; // advance one, as we always stayed <= value 112 | } 113 | 114 | return slice; 115 | } -------------------------------------------------------------------------------- /src/lib.rs: -------------------------------------------------------------------------------- 1 | //! An incremental implementation of worst-case optimal joins. 2 | //! 3 | //! This crate contains functionality to construct timely dataflow computations to compute and maintain 4 | //! the results of complex relational joins under changes to the relations, with worst-case optimality 5 | //! guarantees about the running time. 6 | //! 7 | //! As an example, consider a stream of directed graph edges `(src, dst)` where we would like to find all 8 | //! directed cycles of length three. That is, node indentifiers `x0`, `x1`, and `x2` where the graph contains 9 | //! edges `(x0, x1)`, `(x1, x2)`, and `(x2, x0)`. We can write this query as a relational join on the edge 10 | //! relation `edge(x,y)`, as 11 | //! 12 | //! cycle_3(x0, x1, x2) := edge(x0, x1), edge(x1, x2), edge(x2, x0) 13 | //! 14 | //! To determine the set of three-cycles, we could use standard techniques from the database literature to 15 | //! perform the join, typically first picking one attribute (`x0`, `x1`, or `x2`) and performing the join on 16 | //! the two relations containing that attribute, then joining (intersecting) with the remaining relation. 17 | //! 18 | //! This has the defect that it may perform an amount of work quadratic in the size of `edges`. Recent work 19 | //! on "worst-case optimal join processing" shows how to get around this problem, by considering multiple 20 | //! relations at the same time. 21 | //! 22 | //! This crate is a streaming implementation of incremental worst-case optimal join processing. You may 23 | //! indicate a relational query like above, and the crate with synthesize a timely dataflow computation which 24 | //! reports all changes to the occurrences of satisfying assignments to the values. The amount of work performed 25 | //! is no more than the worst-case optimal bound. 26 | 27 | extern crate timely; 28 | 29 | use timely::dataflow::*; 30 | use timely::dataflow::operators::*; 31 | use timely::Data; 32 | 33 | mod index; 34 | mod extender; 35 | pub mod motif; 36 | 37 | pub use index::Index; 38 | pub use extender::IndexStream; 39 | 40 | /// Functionality used by GenericJoin to extend prefixes with new attributes. 41 | /// 42 | /// These methods are used in `GenericJoin`'s `extend` method, and may not be broadly useful elsewhere. 43 | pub trait StreamPrefixExtender { 44 | /// The type of data to extend. 45 | type Prefix: Data; 46 | /// The type of the extentions. 47 | type Extension: Data; 48 | /// Updates each prefix with an upper bound on the number of extensions for this relation. 49 | fn count(&self, Stream, u64) -> Stream; 50 | /// Proposes each extension from this relation. 51 | fn propose(&self, Stream) -> Stream, W)>; 52 | /// Restricts proposals by those this relation would propose. 53 | fn intersect(&self, Stream, W)>) -> Stream, W)>; 54 | } 55 | 56 | /// Extension method for generic join functionality. 57 | pub trait GenericJoin { 58 | /// Extends a stream of prefixes using the supplied prefix extenders. 59 | fn extend<'a, E: Data>(&self, extenders: Vec+'a>>) 60 | -> Stream, W)>; 61 | } 62 | 63 | // A layer of GenericJoin, in which a collection of prefixes are extended by one attribute 64 | impl GenericJoin for Stream { 65 | fn extend<'a, E>(&self, extenders: Vec+'a>>) -> Stream, W)> 66 | where E: Data { 67 | 68 | if extenders.len() == 1 { 69 | extenders[0].propose(self.clone()) 70 | } 71 | else { 72 | let mut counts = self.map(|(p,s)| (p, 1 << 31, 0, s)); 73 | for (index,extender) in extenders.iter().enumerate() { 74 | counts = extender.count(counts, index as u64); 75 | } 76 | 77 | let parts = counts.partition(extenders.len() as u64, |(p, _, i, w)| (i, (p, w))); 78 | 79 | let mut results = Vec::new(); 80 | for (index, nominations) in parts.into_iter().enumerate() { 81 | let mut extensions = extenders[index].propose(nominations); 82 | for other in (0..extenders.len()).filter(|&x| x != index) { 83 | extensions = extenders[other].intersect(extensions); 84 | } 85 | 86 | results.push(extensions); // save extensions 87 | } 88 | 89 | self.scope().concatenate(results).map(|(p,es,w)| (p,es,w)) 90 | } 91 | } 92 | } 93 | 94 | /// Reports the number of elements satisfing the predicate. 95 | /// 96 | /// This methods *relies strongly* on the assumption that the predicate 97 | /// stays false once it becomes false, a joint property of the predicate 98 | /// and the slice. This allows `advance` to use exponential search to 99 | /// count the number of elements in time logarithmic in the result. 100 | // #[inline(never)] 101 | pub fn advancebool>(slice: &[T], function: F) -> usize { 102 | 103 | // start with no advance 104 | let mut index = 0; 105 | if index < slice.len() && function(&slice[index]) { 106 | 107 | // advance in exponentially growing steps. 108 | let mut step = 1; 109 | while index + step < slice.len() && function(&slice[index + step]) { 110 | index += step; 111 | step = step << 1; 112 | } 113 | 114 | // advance in exponentially shrinking steps. 115 | step = step >> 1; 116 | while step > 0 { 117 | if index + step < slice.len() && function(&slice[index + step]) { 118 | index += step; 119 | } 120 | step = step >> 1; 121 | } 122 | 123 | index += 1; 124 | } 125 | 126 | index 127 | } 128 | -------------------------------------------------------------------------------- /static_graph/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | 3 | name = "dataflow-join" 4 | version = "0.0.1" 5 | authors = ["Frank McSherry "] 6 | 7 | [dependencies] 8 | timely="0.7.0" 9 | core_affinity = "0.5.9" 10 | timely_sort="0.1.1" 11 | time = "*" 12 | rand = "0.5.5" 13 | mmap="*" 14 | -------------------------------------------------------------------------------- /static_graph/LICENSE: -------------------------------------------------------------------------------- 1 | The MIT License (MIT) 2 | 3 | Copyright (c) 2015 Frank McSherry 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | 23 | -------------------------------------------------------------------------------- /static_graph/README.md: -------------------------------------------------------------------------------- 1 | # dataflow-join 2 | An implementation of Ngo et al's GenericJoin in timely dataflow. 3 | 4 | Ngo et al presented a very cool join algorithm, some details of which are described in [a blog post](http://www.frankmcsherry.org/dataflow/relational/join/2015/04/11/genericjoin.html). This text will be replaced with a much better description of what is going on, in the fullness of time. 5 | -------------------------------------------------------------------------------- /static_graph/examples/diamonds.rs: -------------------------------------------------------------------------------- 1 | extern crate rand; 2 | extern crate core_affinity; 3 | extern crate timely; 4 | extern crate dataflow_join; 5 | 6 | use std::rc::Rc; 7 | 8 | use dataflow_join::*; 9 | use dataflow_join::graph::{GraphTrait, GraphMMap, GraphExtenderExt}; 10 | 11 | use timely::dataflow::operators::*; 12 | use timely::dataflow::operators::aggregation::Aggregate; 13 | 14 | fn main () { 15 | 16 | let filename = std::env::args().nth(1).unwrap(); 17 | let inspect = std::env::args().nth(2).unwrap() == "inspect"; 18 | let step_size = std::env::args().nth(3).unwrap().parse::().unwrap(); 19 | 20 | timely::execute_from_args(std::env::args().skip(4), move |root| { 21 | 22 | let index = root.index(); 23 | let peers = root.peers(); 24 | 25 | let core_ids = core_affinity::get_core_ids().unwrap(); 26 | core_affinity::set_for_current(core_ids[index % core_ids.len()]); 27 | 28 | let graph = Rc::new(GraphMMap::::new(&filename)); 29 | 30 | let (mut input, probe) = root.dataflow(|builder| { 31 | 32 | let (input, edges) = builder.new_input::<(u32, u32)>(); 33 | 34 | // // pairs 35 | // let cliques = cliques.extend(vec![&graph.extend_using(|&a| a as u64)]); 36 | 37 | // triangles 38 | let cliques = edges.extend(vec![&graph.extend_using(|&(a,_)| a as u64)]); 39 | 40 | // quadrangles? 41 | let cliques = cliques.flat_map(|(p,es)| es.into_iter().map(move |e| (p, e))) 42 | .filter(|&((_,b),c)| b < c) 43 | .extend(vec![&graph.extend_using(|&((_,b),_)| b as u64), 44 | &graph.extend_using(|&((_,_),c)| c as u64)]); 45 | 46 | // // 5 cliques? 47 | // let cliques = cliques.flat_map(|(p,es)| es.into_iter().map(move |e| (p, e))) 48 | // .extend(vec![&graph.extend_using(|&(((a,_),_),_)| a as u64), 49 | // &graph.extend_using(|&(((_,b),_),_)| b as u64), 50 | // &graph.extend_using(|&(((_,_),c),_)| c as u64), 51 | // &graph.extend_using(|&(((_,_),_),d)| d as u64)]); 52 | 53 | let mut count = 0; 54 | if inspect { 55 | cliques 56 | .map(|x| ((), x.1.len())) 57 | .aggregate(|_k, v: usize, a| *a += v, |_k, a: usize| a, |_k| 0) 58 | .inspect_batch(move |_t, b| { 59 | for x in b { count += *x; } 60 | println!("count: {}", count); 61 | }); 62 | } 63 | 64 | (input, cliques.probe()) 65 | }); 66 | 67 | let mut edges = Vec::new(); 68 | for node in 0 .. graph.nodes() { 69 | for &edge in graph.edges(node) { 70 | if (node + (edge as usize)) % peers == index { 71 | edges.push((node as u32, edge)); 72 | } 73 | } 74 | } 75 | use rand::prelude::*; 76 | let mut rng = thread_rng(); 77 | rng.shuffle(&mut edges); 78 | 79 | let mut round = 0; 80 | while !edges.is_empty() { 81 | let to_take = std::cmp::min(edges.len(), step_size / peers); 82 | for _index in 0 .. to_take { 83 | input.send(edges.pop().unwrap()); 84 | } 85 | input.advance_to(round as u64 + 1); 86 | round += 1; 87 | while probe.less_than(input.time()) { 88 | root.step(); 89 | } 90 | } 91 | 92 | }).unwrap(); 93 | } 94 | -------------------------------------------------------------------------------- /static_graph/examples/layout.rs: -------------------------------------------------------------------------------- 1 | extern crate dataflow_join; 2 | 3 | use std::io::{BufWriter, Write}; 4 | use std::fs::File; 5 | use std::slice; 6 | use std::mem; 7 | 8 | use dataflow_join::graph::{GraphTrait, GraphVector, GraphMMap}; 9 | 10 | fn main() { 11 | println!("Usage: layout "); 12 | let source = std::env::args().skip(1).next().unwrap(); 13 | let target = std::env::args().skip(2).next().unwrap(); 14 | let graph = layout(&source); 15 | 16 | let graph = _extract_fragment(&graph, 0, 1); 17 | _digest_graph_vector(&graph, &target); 18 | } 19 | 20 | fn layout(prefix: &str) -> Vec<(u32, u32)> { 21 | let graph = GraphMMap::::new(&prefix); 22 | 23 | let mut degree = Vec::new(); 24 | for node in 0..graph.nodes() { 25 | for &edge in graph.edges(node) { 26 | while degree.len() <= node { degree.push((0,0)); } 27 | while degree.len() <= edge as usize { degree.push((0,0)); } 28 | degree[node as usize].0 += 1u32; 29 | degree[edge as usize].0 += 1u32; 30 | } 31 | } 32 | 33 | for node in 0..degree.len() { 34 | degree[node].1 = node as u32; 35 | } 36 | 37 | degree.sort(); 38 | for node in 0..degree.len() { 39 | degree[node] = (degree[node].1, node as u32); 40 | } 41 | 42 | let mut result = Vec::new(); 43 | for node in 0..graph.nodes() { 44 | for &edge in graph.edges(node) { 45 | result.push((degree[node as usize].1, degree[edge as usize].1)); 46 | } 47 | } 48 | 49 | organize_graph(&mut result); 50 | result 51 | } 52 | 53 | 54 | fn organize_graph(graph: &mut Vec<(u32, u32)>) { 55 | 56 | for index in 0..graph.len() { 57 | if graph[index].1 < graph[index].0 { 58 | graph[index] = (graph[index].1, graph[index].0); 59 | } 60 | } 61 | 62 | graph.sort(); 63 | println!("graph data sorted; {:?} edges", graph.len()); 64 | 65 | graph.dedup(); 66 | println!("graph data uniqed; {:?} edges", graph.len()); 67 | } 68 | 69 | fn _extract_fragment(graph: &Vec<(u32, u32)>, index: u64, parts: u64) -> GraphVector { 70 | let mut nodes = Vec::new(); 71 | let mut edges = Vec::new(); 72 | 73 | for &(src,dst) in graph { 74 | if src as u64 % parts == index { 75 | while src + 1 >= nodes.len() as u32 { nodes.push(0); } 76 | while dst + 1 >= nodes.len() as u32 { nodes.push(0); } // allows unsafe access to nodes 77 | 78 | nodes[src as usize + 1] += 1; 79 | edges.push(dst); 80 | } 81 | } 82 | 83 | for index in 1..nodes.len() { 84 | nodes[index] += nodes[index - 1]; 85 | } 86 | 87 | return GraphVector { nodes: nodes, edges: edges }; 88 | } 89 | 90 | fn _digest_graph_vector(graph: &GraphVector, output_prefix: &str) { 91 | let mut edge_writer = BufWriter::new(File::create(format!("{}.targets", output_prefix)).unwrap()); 92 | let mut node_writer = BufWriter::new(File::create(format!("{}.offsets", output_prefix)).unwrap()); 93 | node_writer.write_all(unsafe { _typed_as_byte_slice(&graph.nodes[..]) }).unwrap(); 94 | 95 | let mut slice = unsafe { _typed_as_byte_slice(&graph.edges[..]) }; 96 | while slice.len() > 0 { 97 | let to_write = if slice.len() < 1000000 { slice.len() } else { 1000000 }; 98 | edge_writer.write_all(&slice[..to_write]).unwrap(); 99 | println!("wrote some; remaining: {}", slice.len()); 100 | slice = &slice[to_write..]; 101 | } 102 | } 103 | 104 | unsafe fn _typed_as_byte_slice(slice: &[T]) -> &[u8] { 105 | slice::from_raw_parts(slice.as_ptr() as *const u8, slice.len() * mem::size_of::()) 106 | } 107 | -------------------------------------------------------------------------------- /static_graph/examples/triangles-single.rs: -------------------------------------------------------------------------------- 1 | extern crate mmap; 2 | extern crate time; 3 | extern crate dataflow_join; 4 | 5 | use std::cmp::Ordering::*; 6 | 7 | use dataflow_join::graph::{GraphTrait, GraphMMap, gallop}; 8 | 9 | fn main () { 10 | if let Some(source) = std::env::args().skip(1).next() { 11 | println!("triangles: {:?}", raw_triangles(&GraphMMap::new(&source))); 12 | } 13 | else { 14 | println!("usage: "); 15 | } 16 | } 17 | 18 | fn raw_triangles>(graph: &G) -> u64 { 19 | 20 | let mut count = 0; 21 | for a in 0..graph.nodes() { 22 | if graph.edges(a).len() > 0 { 23 | count += 1; 24 | } 25 | } 26 | 27 | println!("count: {}", count); 28 | 29 | let mut count = 0; 30 | for a in (0..graph.nodes()) { 31 | let aaa = graph.edges(a); 32 | println!("degr[{}]: {}", a, aaa.len()); 33 | for &b in aaa { 34 | let bbb = graph.edges(b as usize); 35 | count += if aaa.len() < bbb.len() { intersect(aaa, bbb) } 36 | else { intersect(bbb, aaa) }; 37 | } 38 | } 39 | count 40 | } 41 | 42 | fn intersect(mut aaa: &[E], mut bbb: &[E]) -> u64 { 43 | let mut count = 0; 44 | // magic gallop overhead # is 4 45 | if aaa.len() < bbb.len() / 4 { 46 | for a in aaa { 47 | bbb = gallop(bbb, a); 48 | if bbb.len() > 0 && &bbb[0] == a { count += 1; } 49 | } 50 | } 51 | else { 52 | while aaa.len() > 0 && bbb.len() > 0 { 53 | match aaa[0].cmp(&bbb[0]) { 54 | Greater => { bbb = &bbb[1..]; }, 55 | Less => { aaa = &aaa[1..]; }, 56 | Equal => { aaa = &aaa[1..]; 57 | bbb = &bbb[1..]; 58 | count += 1; 59 | }, 60 | } 61 | } 62 | } 63 | count 64 | } 65 | -------------------------------------------------------------------------------- /static_graph/examples/triangles.rs: -------------------------------------------------------------------------------- 1 | 2 | extern crate mmap; 3 | extern crate time; 4 | extern crate timely; 5 | extern crate dataflow_join; 6 | 7 | use std::rc::Rc; 8 | 9 | use dataflow_join::*; 10 | use dataflow_join::graph::{GraphTrait, GraphMMap, GraphExtenderExt}; 11 | 12 | use timely::dataflow::*; 13 | use timely::dataflow::operators::*; 14 | use timely::progress::timestamp::RootTimestamp; 15 | 16 | fn main () { 17 | 18 | let filename = std::env::args().nth(1).unwrap(); 19 | let inspect = std::env::args().nth(2).unwrap() == "inspect"; 20 | let step_size = std::env::args().nth(3).unwrap().parse::().unwrap(); 21 | 22 | timely::execute_from_args(std::env::args().skip(4), move |root| { 23 | 24 | let index = root.index(); 25 | let peers = root.peers(); 26 | 27 | let graph = Rc::new(GraphMMap::::new(&filename)); 28 | 29 | let (mut input, probe) = root.dataflow(|builder| { 30 | 31 | let (input, cliques) = builder.new_input::(); 32 | 33 | // pairs 34 | let cliques = cliques.extend(vec![&graph.extend_using(|&a| a as u64)]); 35 | 36 | // triangles 37 | let cliques = cliques.flat_map(|(p, es)| es.into_iter().map(move |e| (p, e))) 38 | .extend(vec![&graph.extend_using(|&(a,_)| a as u64), 39 | &graph.extend_using(|&(_,b)| b as u64)]); 40 | 41 | // // quadrangles? 42 | // let cliques = cliques.flat_map(|(p,es)| es.into_iter().map(move |e| (p, e))) 43 | // .extend(vec![&graph.extend_using(|&((a,_),_)| a as u64), 44 | // &graph.extend_using(|&((_,b),_)| b as u64), 45 | // &graph.extend_using(|&((_,_),c)| c as u64)]); 46 | 47 | // // 5 cliques? 48 | // let cliques = cliques.flat_map(|(p,es)| es.into_iter().map(move |e| (p, e))) 49 | // .extend(vec![&graph.extend_using(|&(((a,_),_),_)| a as u64), 50 | // &graph.extend_using(|&(((_,b),_),_)| b as u64), 51 | // &graph.extend_using(|&(((_,_),c),_)| c as u64), 52 | // &graph.extend_using(|&(((_,_),_),d)| d as u64)]); 53 | 54 | let mut count = 0; 55 | if inspect { 56 | cliques 57 | .exchange(|x| 0) 58 | .inspect_batch(move |_t, b| { 59 | for x in b { count += x.1.len(); } 60 | println!("count: {}", count); 61 | }); 62 | } 63 | 64 | (input, cliques.probe()) 65 | }); 66 | 67 | let nodes = graph.nodes() - 1; 68 | let limit = (nodes / step_size) + 1; 69 | for round in 0..limit { 70 | for source in 0..step_size { 71 | let candidate = source + round * step_size; 72 | if candidate % peers == index && candidate < nodes { 73 | input.send(candidate as u32); 74 | } 75 | } 76 | 77 | input.advance_to(round as u64 + 1); 78 | root.step(); 79 | while probe.less_than(&RootTimestamp::new((round - ::std::cmp::min(1, round)) as u64)) { 80 | root.step(); 81 | } 82 | } 83 | }).unwrap(); 84 | } 85 | -------------------------------------------------------------------------------- /static_graph/src/flattener.rs: -------------------------------------------------------------------------------- 1 | use std::rc::Rc; 2 | use std::cell::RefCell; 3 | use std::collections::VecDeque; 4 | 5 | use timely::progress::nested::subgraph::Source::ScopeOutput; 6 | use timely::progress::nested::subgraph::Target::ScopeInput; 7 | 8 | use timely::communication::*; 9 | use timely::communication::pact::{Pipeline, PactPullable}; 10 | use timely::progress::count_map::CountMap; 11 | use timely::progress::{Timestamp, Scope, Antichain}; 12 | use timely::communication::channels::ObserverHelper; 13 | 14 | use timely::example_static::builder::*; 15 | use timely::example_static::unary::PullableHelper; 16 | use timely::example_static::stream::{ActiveStream, Stream}; 17 | 18 | use timely::drain::DrainExt; 19 | 20 | pub trait FlattenerExt { 21 | fn flatten(self) -> (Stream, ActiveStream>); 22 | } 23 | 24 | impl FlattenerExt for ActiveStream)> { 25 | fn flatten(mut self) -> (Stream, ActiveStream>) { 26 | let (sender, receiver) = Pipeline.connect(self.builder.communicator()); 27 | 28 | let (targets1, registrar1) = OutputPort::::new(); 29 | let (targets2, registrar2) = OutputPort::>::new(); 30 | 31 | let scope = FlattenerScope::new(receiver, targets1, targets2); 32 | let index = self.builder.add_scope(scope); 33 | self.connect_to(ScopeInput(index, 0), sender); 34 | 35 | (Stream { name: ScopeOutput(index, 0), ports: registrar1 }, 36 | self.transfer_borrow_to(ScopeOutput(index, 1), registrar2) ) 37 | } 38 | } 39 | 40 | pub struct FlattenerScope { 41 | input: PullableHelper), Rc)>)>>>>, 42 | output1: ObserverHelper>, 43 | output2: ObserverHelper>>, 44 | stash: Vec>, 45 | counter: u64, 46 | } 47 | 48 | impl FlattenerScope { 49 | pub fn new(receiver: PactPullable), Rc)>)>>>>, 50 | output1: OutputPort, 51 | output2: OutputPort>) -> FlattenerScope { 52 | FlattenerScope { 53 | input: PullableHelper::new(receiver), 54 | output1: ObserverHelper::new(output1, Rc::new(RefCell::new(CountMap::new()))), 55 | output2: ObserverHelper::new(output2, Rc::new(RefCell::new(CountMap::new()))), 56 | stash: Vec::new(), 57 | counter: 0, 58 | } 59 | } 60 | } 61 | 62 | impl Scope for FlattenerScope { 63 | fn inputs(&self) -> u64 { 1 } 64 | fn outputs(&self) -> u64 { 2 } 65 | 66 | fn set_external_summary(&mut self, _summaries: Vec>>, _frontier: &mut [CountMap]) -> () { } 67 | 68 | fn push_external_progress(&mut self,_external: &mut [CountMap]) -> () { } 69 | 70 | fn pull_internal_progress(&mut self,_internal: &mut [CountMap], 71 | consumed: &mut [CountMap], 72 | produced: &mut [CountMap]) -> bool { 73 | 74 | while let Some((time, pairs)) = self.input.pull() { 75 | let mut session = self.output1.session(&time); 76 | for (prefix, mut extensions) in pairs.drain_temp() { 77 | for extension in extensions.drain_temp() { 78 | session.give((prefix.clone(), extension)); 79 | } 80 | 81 | self.stash.push(extensions); 82 | } 83 | // self.counter += self.stash.len() as u64; 84 | self.output2.give_at(&time, self.stash.drain_temp()); 85 | } 86 | // println!("stashed: {}", self.counter); 87 | 88 | // extract what we know about progress from the input and output adapters. 89 | self.input.pull_progress(&mut consumed[0]); 90 | self.output1.pull_progress(&mut produced[0]); 91 | self.output2.pull_progress(&mut produced[1]); 92 | 93 | return false; // no unannounced internal work 94 | } 95 | 96 | fn name(&self) -> String { format!("Flattener") } 97 | fn notify_me(&self) -> bool { false } 98 | } 99 | -------------------------------------------------------------------------------- /static_graph/src/graph.rs: -------------------------------------------------------------------------------- 1 | use std::rc::Rc; 2 | use std::marker::PhantomData; 3 | 4 | use typedrw::TypedMemoryMap; 5 | use PrefixExtender; 6 | 7 | pub trait GraphExtenderExt { 8 | fn extend_usingu64+'static>(&self, route: L) -> Rc>; 9 | } 10 | 11 | impl GraphExtenderExt for Rc { 12 | fn extend_usingu64+'static>(&self, logic: L) -> Rc> { 13 | Rc::new(GraphExtender { 14 | graph: self.clone(), 15 | logic: Rc::new(logic), 16 | phant: PhantomData, 17 | }) 18 | } 19 | } 20 | 21 | pub trait GraphTrait : 'static { 22 | type Target: Ord; 23 | fn nodes(&self) -> usize; 24 | fn edges(&self, node: usize) -> &[Self::Target]; 25 | } 26 | 27 | pub struct GraphVector { 28 | pub nodes: Vec, 29 | pub edges: Vec, 30 | } 31 | 32 | impl GraphTrait for GraphVector { 33 | type Target = E; 34 | #[inline(always)] 35 | fn nodes(&self) -> usize { 36 | self.nodes.len() 37 | } 38 | #[inline(always)] 39 | fn edges(&self, node: usize) -> &[E] { 40 | if node + 1 < self.nodes.len() { 41 | let start = self.nodes[node] as usize; 42 | let limit = self.nodes[node+1] as usize; 43 | &self.edges[start..limit] 44 | } 45 | else { &[] } 46 | } 47 | } 48 | 49 | pub struct GraphMMap { 50 | nodes: TypedMemoryMap, 51 | edges: TypedMemoryMap, 52 | } 53 | 54 | impl GraphMMap { 55 | pub fn new(prefix: &str) -> GraphMMap { 56 | GraphMMap { 57 | nodes: TypedMemoryMap::new(format!("{}.offsets", prefix)), 58 | edges: TypedMemoryMap::new(format!("{}.targets", prefix)), 59 | } 60 | } 61 | } 62 | 63 | impl GraphTrait for GraphMMap { 64 | type Target = E; 65 | #[inline(always)] 66 | fn nodes(&self) -> usize { 67 | self.nodes[..].len() 68 | } 69 | #[inline(always)] 70 | fn edges(&self, node: usize) -> &[E] { 71 | let nodes = &self.nodes[..]; 72 | if node + 1 < nodes.len() { 73 | let start = nodes[node] as usize; 74 | let limit = nodes[node+1] as usize; 75 | &self.edges[..][start..limit] 76 | } 77 | else { &[] } 78 | } 79 | } 80 | 81 | pub struct GraphExtenderu64> { 82 | graph: Rc, 83 | logic: Rc, 84 | phant: PhantomData

, 85 | } 86 | 87 | implu64+'static> PrefixExtender for GraphExtender 88 | where ::Target : Clone { 89 | type Prefix = P; 90 | type Extension = G::Target; 91 | 92 | type RoutingFunction = L; 93 | fn logic(&self) -> Rc { self.logic.clone() } 94 | 95 | fn count(&self, prefix: &P) -> u64 { 96 | let node = (*self.logic)(prefix) as usize; 97 | self.graph.edges(node).len() as u64 98 | } 99 | 100 | fn propose(&self, prefix: &P, list: &mut Vec) { 101 | let node = (*self.logic)(prefix) as usize; 102 | *list = self.graph.edges(node).to_vec(); 103 | // list.extend(self.graph.edges(node).iter().cloned()); 104 | } 105 | 106 | fn intersect(&self, prefix: &P, list: &mut Vec) { 107 | let node = (*self.logic)(prefix) as usize; 108 | let mut slice = self.graph.edges(node); 109 | 110 | if list.len() < slice.len() / 4 { 111 | list.retain(|value| { 112 | slice = gallop(slice, value); 113 | slice.len() > 0 && &slice[0] == value 114 | }); 115 | } 116 | else { 117 | list.retain(move |value| { 118 | while slice.len() > 0 && &slice[0] < value { slice = &slice[1..]; } 119 | slice.len() > 0 && &slice[0] == value 120 | }); 121 | } 122 | } 123 | } 124 | 125 | // intended to advance slice to start at the first element >= value. 126 | 127 | #[inline(always)] 128 | pub fn gallop<'a, T: Ord>(mut slice: &'a [T], value: &T) -> &'a [T] { 129 | // if empty slice, or already >= element, return 130 | if slice.len() > 0 && &slice[0] < value { 131 | let mut step = 1; 132 | while step < slice.len() && &slice[step] < value { 133 | slice = &slice[step..]; 134 | step = step << 1; 135 | } 136 | 137 | step = step >> 1; 138 | while step > 0 { 139 | if step < slice.len() && &slice[step] < value { 140 | slice = &slice[step..]; 141 | } 142 | step = step >> 1; 143 | } 144 | 145 | slice = &slice[1..]; // advance one, as we always stayed < value 146 | } 147 | 148 | return slice; 149 | } 150 | -------------------------------------------------------------------------------- /static_graph/src/lib.rs: -------------------------------------------------------------------------------- 1 | // #![allow(dead_code)] 2 | 3 | extern crate timely; 4 | extern crate time; 5 | extern crate mmap; 6 | 7 | use std::rc::Rc; 8 | 9 | use timely::dataflow::*; 10 | use timely::dataflow::operators::*; 11 | use timely::dataflow::channels::pact::Exchange; 12 | use timely::ExchangeData as Data; 13 | 14 | pub mod graph; 15 | 16 | mod typedrw; 17 | pub use typedrw::TypedMemoryMap; 18 | 19 | // Algorithm 3 is an implementation of an instance of GenericJoin, a worst-case optimal join algorithm. 20 | 21 | // The algorithm orders the attributes of the resulting relation, and for each prefix of these attributes 22 | // produces the set of viable prefixes of output relations. The set of prefixes is updated by a new attribute 23 | // by having each relation with that attribute propose extensions for each prefix, based on matching existing 24 | // attributes within their relation. Proposals are then intersected, and surviving extended prefixes form the 25 | // basis of the next iteration 26 | 27 | 28 | // Informally, the algorithm looks like: 29 | // 0. Let X be an empty relation over 0 attributes 30 | // 1. For each output attribute A: 31 | // 0. Let T be an initially empty set. 32 | // a. For each relation R containing A: 33 | // i. For each element x of X, let p(R, x) be the set of distinct values of A in pi_A(R join x), 34 | // that is, the distinct symbols R would propose to extend x. 35 | // b. For each element x of X, let r(x) be the relation R with the smallest p(R, x). 36 | // c. For each relation R containing A: 37 | // i. For each element x of X with r(x) = R, add (x join p(R, x)) to T. 38 | // d. For each relation R containing A: 39 | // i. For each element (x, y) of T, remove (x, y) if y is not in p(R, x). 40 | // 41 | // The important part of this algorithm is that step d.i should take roughly constant time. 42 | 43 | 44 | // record-by-record prefix extension functionality 45 | pub trait PrefixExtender { 46 | type Prefix; 47 | type Extension; 48 | 49 | // these are the parts required for the join algorithm 50 | fn count(&self, &Self::Prefix) -> u64; 51 | fn propose(&self, &Self::Prefix, &mut Vec); 52 | fn intersect(&self, &Self::Prefix, &mut Vec); 53 | 54 | // these are needed to tell timely dataflow how to route prefixes. 55 | // this object will be shared under an Rc> so we want 56 | // to give back a function, rather than provide a method ourself. 57 | type RoutingFunction: Fn(&Self::Prefix)->u64+'static; 58 | fn logic(&self) -> Rc; 59 | } 60 | 61 | // functionality required by the GenericJoin layer 62 | pub trait StreamPrefixExtender { 63 | type Prefix: Data; 64 | type Extension: Data; 65 | 66 | fn count(&self, Stream, u64) -> Stream; 67 | fn propose(&self, Stream) -> Stream)>; 68 | fn intersect(&self, Stream)>) -> Stream)>; 69 | } 70 | 71 | // implementation of StreamPrefixExtender for any (wrapped) PrefixExtender 72 | // TODO : Add a Rc>>> to recycle allocations 73 | impl StreamPrefixExtender for Rc 74 | where PE::Prefix: Data, 75 | PE::Extension: Data, { 76 | type Prefix = PE::Prefix; 77 | type Extension = PE::Extension; 78 | 79 | fn count(&self, stream: Stream, ident: u64) -> Stream { 80 | let clone = self.clone(); 81 | let logic = self.logic(); 82 | let exch = Exchange::new(move |&(ref x,_,_)| (*logic)(x)); 83 | let mut vector = Vec::new(); 84 | stream.unary(exch, "Count", move |_,_| move |input, output| { 85 | while let Some((time, data)) = input.next() { 86 | data.swap(&mut vector); 87 | for &mut (ref p, ref mut c, ref mut i) in vector.iter_mut() { 88 | let nc = (*clone).count(p); 89 | if &nc < c { 90 | *c = nc; 91 | *i = ident; 92 | } 93 | } 94 | vector.retain(|x| x.1 > 0); 95 | output.session(&time).give_vec(&mut vector); 96 | } 97 | }) 98 | } 99 | 100 | fn propose(&self, stream: Stream) -> Stream)> { 101 | let clone = self.clone(); 102 | let logic = self.logic(); 103 | let exch = Exchange::new(move |x| (*logic)(x)); 104 | let mut vector = Vec::new(); 105 | stream.unary(exch, "Propose", move |_,_| move |input, output| { 106 | let mut effort = 0; 107 | while let Some((time, data)) = input.next() { 108 | data.swap(&mut vector); 109 | effort += vector.len(); 110 | output.session(&time).give_iterator(vector.drain(..).map(|p| { 111 | let mut vec = Vec::new(); 112 | (*clone).propose(&p, &mut vec); 113 | (p, vec) 114 | })); 115 | if effort > 4096 { 116 | break; 117 | } 118 | } 119 | }) 120 | } 121 | fn intersect(&self, stream: Stream)>) -> Stream)> { 122 | let logic = self.logic(); 123 | let clone = self.clone(); 124 | let exch = Exchange::new(move |&(ref x,_)| (*logic)(x)); 125 | let mut vector = Vec::new(); 126 | stream.unary(exch, "Intersect", move |_,_| move |input, output| { 127 | while let Some((time, data)) = input.next() { 128 | data.swap(&mut vector); 129 | for &mut (ref prefix, ref mut extensions) in vector.iter_mut() { 130 | (*clone).intersect(prefix, extensions); 131 | } 132 | vector.retain(|x| x.1.len() > 0); 133 | output.session(&time).give_vec(&mut vector); 134 | } 135 | }) 136 | } 137 | } 138 | 139 | pub trait GenericJoinExt { 140 | fn extend(self, extenders: Vec<&StreamPrefixExtender>) 141 | -> Stream)>; 142 | } 143 | 144 | // A layer of GenericJoin, in which a collection of prefixes are extended by one attribute 145 | impl GenericJoinExt for Stream { 146 | fn extend(self, extenders: Vec<&StreamPrefixExtender>) 147 | -> Stream)> { 148 | 149 | let mut counts = self.map(|p| (p, 1 << 31, 0)); 150 | for (index,extender) in extenders.iter().enumerate() { 151 | counts = extender.count(counts, index as u64); 152 | } 153 | 154 | let parts = counts.partition(extenders.len() as u64, |(p, _, i)| (i, p)); 155 | 156 | let mut results = Vec::new(); 157 | for (index, nominations) in parts.into_iter().enumerate() { 158 | let mut extensions = extenders[index].propose(nominations); 159 | for other in (0..extenders.len()).filter(|&x| x != index) { 160 | extensions = extenders[other].intersect(extensions); 161 | } 162 | 163 | results.push(extensions); // save extensions 164 | } 165 | 166 | self.scope().concatenate(results) 167 | } 168 | } 169 | -------------------------------------------------------------------------------- /static_graph/src/typedrw.rs: -------------------------------------------------------------------------------- 1 | use std::mem; 2 | // use core::raw::Slice as RawSlice; 3 | use mmap::MapOption::{MapReadable, MapFd}; 4 | use mmap::MemoryMap; 5 | use std::os::unix::prelude::AsRawFd; 6 | use std::slice; 7 | use std::ops; 8 | use std::fs::File; 9 | use std::marker::PhantomData; 10 | 11 | pub struct TypedMemoryMap { 12 | map: MemoryMap, // mapped file 13 | len: usize, // in bytes (needed because map extends to full block) 14 | phn: PhantomData, 15 | } 16 | 17 | impl TypedMemoryMap { 18 | pub fn new(filename: String) -> TypedMemoryMap { 19 | let file = File::open(filename).unwrap(); 20 | let size = file.metadata().unwrap().len() as usize; 21 | TypedMemoryMap { 22 | map: MemoryMap::new(size, &[MapReadable, MapFd(file.as_raw_fd())]).unwrap(), 23 | len: size / mem::size_of::(), 24 | phn: PhantomData, 25 | } 26 | } 27 | } 28 | 29 | impl ops::Index for TypedMemoryMap { 30 | type Output = [T]; 31 | #[inline] 32 | fn index(&self, _index: ops::RangeFull) -> &[T] { 33 | // assert!(self.len <= self.map.len()); 34 | // unsafe { mem::transmute(RawSlice { 35 | // data: self.map.data() as *const u8, 36 | // len: self.len, 37 | // })} 38 | unsafe { slice::from_raw_parts(self.map.data() as *const T, self.len) } 39 | } 40 | } 41 | --------------------------------------------------------------------------------