Release 0.2.0

Co-authored-by: Michał Bartoszkiewicz <embe@pathway.com>
Co-authored-by: Jan Chorowski <janek@pathway.com>
Co-authored-by: Xavier Gendre <xavier@pathway.com>
Co-authored-by: Adrian Kosowski <adrian@pathway.com>
Co-authored-by: Jakub Kowalski <kuba@pathway.com>
Co-authored-by: Sergey Kulik <sergey@pathway.com>
Co-authored-by: Mateusz Lewandowski <mateusz@pathway.com>
Co-authored-by: Mohamed Malhou <mohamed@pathway.com>
Co-authored-by: Krzysztof Nowicki <krzysiek@pathway.com>
Co-authored-by: Richard Pelgrim <richard.pelgrim@pathway.com>
Co-authored-by: Kamil Piechowiak <kamil@pathway.com>
Co-authored-by: Paweł Podhajski <pawel.podhajski@pathway.com>
Co-authored-by: Olivier Ruas <olivier@pathway.com>
Co-authored-by: Przemysław Uznański <przemek@pathway.com>
Co-authored-by: Sebastian Włudzik <sebastian.wludzik@pathway.com>
GitOrigin-RevId: 1d3c004e1e06be9fa3506b32e4dac21486185ad7
This commit is contained in:
Manul from Pathway 2023-07-21 13:37:44 +02:00
commit 103cce6077
1036 changed files with 175188 additions and 0 deletions

4
.coveragerc Normal file
View File

@ -0,0 +1,4 @@
[report]
exclude_also =
if TYPE_CHECKING:
^\s*\.\.\.$

9
.gitignore vendored Normal file
View File

@ -0,0 +1,9 @@
target/
__pycache__/
.mypy_cache/
.pytest_cache/
.coverage_report*/
.test_*
*.lcov

16
CHANGELOG.md Normal file
View File

@ -0,0 +1,16 @@
# Changelog
All notable changes to this project will be documented in this file.
The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.1.0/),
and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.html).
## [Unreleased]
## [0.2.0] - 2023-07-20
### Added
<img src="https://d14l3brkh44201.cloudfront.net/PathwayManul.svg" alt="manul" width="50px"></img>

2745
Cargo.lock generated Normal file

File diff suppressed because it is too large Load Diff

85
Cargo.toml Normal file
View File

@ -0,0 +1,85 @@
[package]
name = "pathway"
version = "0.2.0"
edition = "2021"
publish = false
rust-version = "1.69"
license = "BUSL-1.1"
[lib]
name = "pathway_engine"
crate-type = ["cdylib", "lib"]
[dev-dependencies]
assert_matches = "1.5.0"
color-eyre = { version = "0.6.2", default-features = false }
eyre = "0.6.8"
tempfile = "3.6.0"
[dependencies]
arc-swap = "1.6.0"
arcstr = { version = "1.1.5", default-features = false, features = ["serde", "std"] }
base32 = "0.4.0"
bincode = "1.3.3"
cfg-if = "1.0.0"
chrono = { version = "0.4.26", features = ["std", "clock"], default-features = false }
chrono-tz = "0.8.3"
crossbeam-channel = "0.5.8"
csv = "1.2.2"
derivative = "2.2.0"
differential-dataflow = { path = "./external/differential-dataflow" }
elasticsearch = "8.5.0-alpha.1"
futures = "0.3.28"
hyper = { version = "0.14", features = ["server"] }
id-arena = "2.2.1"
itertools = "0.11.0"
jemallocator = { version = "0.5.0", features = ["stats", "disable_initial_exec_tls"] }
log = { version = "0.4.19", features = ["std"] }
ndarray = { version = "0.15.6", features = ["serde"] }
nix = { version = "0.26.2", default-features = false, features = ["fs"] }
num-integer = "0.1.45"
numpy = "0.18.0"
once_cell = "1.18.0"
ordered-float = { version = "3.7.0", features = ["serde"] }
pipe = "0.4.0"
postgres = "0.19.5"
prometheus-client = "0.21.2"
pyo3 = { version = "0.18.3", features = ["abi3-py38"] }
pyo3-asyncio = "0.18.0"
pyo3-log = "0.8.2"
rand = "0.8.5"
rdkafka = { version = "0.33.2", features = ["ssl-vendored", "cmake-build"] }
rust-s3 = { version = "0.33.0", features = ["sync-native-tls-vendored", "sync-native-tls", "fail-on-err"], default-features = false }
scopeguard = "1.1.0"
send_wrapper = "0.6.0"
serde = { version = "1.0.171", features = ["derive", "rc"] }
serde_json = "1.0"
smallvec = { version = "1.11.0", features = ["union", "const_generics"] }
thiserror = "1.0.43"
timely = { path = "./external/timely-dataflow/timely", features = ["bincode"] }
tokio = "1.29.1"
typed-arena = "2.0.2"
xxhash-rust = { version = "0.8.6", features = ["xxh3"] }
[target.'cfg(target_os = "linux")'.dependencies]
inotify = "0.10.1"
[features]
unlimited-workers = []
# Helpful for using external memory profilers
standard-allocator = []
# YOLO!
yolo-id64 = []
yolo-id32 = []
[profile.dev]
opt-level = 3
[profile.profiling]
inherits = "release"
debug = true
[build-dependencies]
pyo3-build-config = "0.18.3"

93
LICENSE.txt Normal file
View File

@ -0,0 +1,93 @@
License:
BSL 1.1
Licensor:
NavAlgo SAS
Licensed Work:
Pathway
Releases of Pathway covered by this License contain a copy of this License
as their license file, and are made available by the Licensor at
pathway.com, github.com/pathwaycom/, and by way of other distribution
channels.
The Licensed Work is © 2023 NavAlgo SAS
Additional Use Grant:
The Licensor grants you (the licensee) additional rights to the Licensed
Work, whereby you are entitled to run the Licensed Work in production
use, at no cost, subject to all of the following conditions:
(a) in a single installation of the Licensed Work you may run the Licensed
Work on only one machine, physical or virtual, and without exceeding the
number of worker threads and processes allowed by the configuration of the
software runner of the distribution; and
(b) the additional right to use granted herein applies to the exclusion
of use of any modified or derivative Licensed Work, as well as to the
exclusion of use of the Licensed Work for a Stream Data Processing
Service; and
(c) the production use of modified Licensed Work is permitted only where
the modifications of the Licensed Work are indispensable to fix bugs or
vulnerabilities which might otherwise alter the scope of functionalities
of the Licensed Work, as described in API documentation available at
https://pathway.com/developers/documentation/api-docs/pathway; and
(d) this entire License including its Additional Use Grant shall remain
in full force and effect for any use under this Additional Use Grant,
thus covering the Licensed Work and also binding any and all users of
the Licensed Work.
A “Stream Data Processing Service” is defined as any offering that allows
third parties (other than your employees or individual contractors) to
access the functionality of the Licensed Work by performing an action
directly or indirectly that causes the deployment, creation, or change to
the structure of a running computation graph of Pathway on any machine.
For the sake of clarity, a Stream Data Processing Service would include
providers of infrastructure services, such as cloud services, hosting
services, data center services and similarly situated third parties
(including affiliates of such entities) that would offer the Licensed
Work, possibly in connection with a broader service offering, to their
customers or subscribers.
Change Date:
Change date is four years from the date of code merge into the main
release branch of Pathway in the GitHub repo (and in no case earlier
than July 20, 2027). Please see GitHub commit history for exact
dates.
Change License:
Apache License, Version 2.0, as published by the Apache Foundation.
The Licensor hereby grants you the right to copy, modify, create derivative
works, redistribute, and make non-production use of the Licensed Work. The
Licensor may make an Additional Use Grant, above, permitting limited
production use.
Effective on the Change Date, or the fifth anniversary of the first publicly
available distribution of a specific version of the Licensed Work under this
License, whichever comes first, the Licensor hereby grants you rights under
the terms of the Change License, and the rights granted in the paragraph
above terminate.
If your use of the Licensed Work does not comply with the requirements
currently in effect as described in this License, you must purchase
a commercial license from the Licensor, its affiliated entities, or
authorized resellers, or you must refrain from using the Licensed Work.
All copies of the original and modified Licensed Work, and derivative works
of the Licensed Work, are subject to this License. This License applies
separately for each version of the Licensed Work and the Change Date may
vary for each version of the Licensed Work released by Licensor.
You must conspicuously display this License on each original or modified
copy of the Licensed Work. If you receive the Licensed Work in original or
modified form from a third party, the terms and conditions set forth in this
License apply to your use of that work.
Any use of the Licensed Work in violation of this License will automatically
terminate your rights under this License for the current and all other
versions of the Licensed Work.
This License does not grant you any right in any trademark or logo of
Licensor or its affiliates (provided that you may use a trademark or logo of
Licensor as expressly required by this License). TO THE EXTENT PERMITTED BY
APPLICABLE LAW, THE LICENSED WORK IS PROVIDED ON AN “AS IS” BASIS. LICENSOR
HEREBY DISCLAIMS ALL WARRANTIES AND CONDITIONS, EXPRESS OR IMPLIED,
INCLUDING (WITHOUT LIMITATION) WARRANTIES OF MERCHANTABILITY, FITNESS FOR A
PARTICULAR PURPOSE, NON-INFRINGEMENT, AND TITLE.

203
README.md Normal file
View File

@ -0,0 +1,203 @@
<div align="center">
<img src="https://pathway.com/logo-light.svg" /><br /><br />
</div>
<p align="center">
<img src="https://img.shields.io/badge/OS-Linux-green" alt="Linux"/>
<img src="https://img.shields.io/badge/OS-macOS-green" alt="macOS"/>
<a href="LICENSE.txt">
<img src="https://img.shields.io/badge/license-BSL-green" alt="License: BSL"/></a>
<a href="https://github.com/pathwaycom/pathway/graphs/contributors">
<img src="https://img.shields.io/github/contributors/pathwaycom/pathway" alt="Contributors"/></a>
<br>
<a href="https://discord.gg/pathway">
<img src="https://img.shields.io/discord/1042405378304004156?logo=discord"
alt="chat on Discord"></a>
<a href="https://twitter.com/intent/follow?screen_name=pathway_com">
<img src="https://img.shields.io/twitter/follow/pathway_com?style=social&logo=twitter"
alt="follow on Twitter"></a>
<a href="https://linkedin.com/company/pathway">
<img src="https://img.shields.io/badge/pathway-0077B5?style=social&logo=linkedin" alt="follow on LinkedIn"></a>
</p>
# Pathway
[Pathway](https://pathway.com) is an open framework for high-throughput and low-latency real-time data processing. It is used to create Python code which seamlessly combines batch processing, streaming, and real-time API's for LLM apps. Pathway's distributed runtime (🦀-🐍) provides fresh results of your data pipelines whenever new inputs and requests are received.
Pathway provides a high-level programming interface in Python for defining data transformations, aggregations, and other operations on data streams.
With Pathway, you can effortlessly design and deploy sophisticated data workflows that efficiently handle high volumes of data in real time.
Pathway is interoperable with various data sources and sinks such as Kafka, CSV files, SQL/noSQL databases, and REST API's, allowing you to connect and process data from different storage systems.
Typical use-cases of Pathway include realtime data processing, ETL (Extract, Transform, Load) pipelines, data analytics, monitoring, anomaly detection, and recommendation. Pathway can also independently provide the backbone of a light LLMops stack for [real-time LLM applications](https://github.com/pathwaycom/llm-app).
In Pathway, data is represented in the form of Tables. Live data streams are also treated as Tables. The library provides a rich set of operations like filtering, joining, grouping, and windowing.
For any questions, you will find the community and team behind the project [on Discord](https://discord.com/invite/pathway).
## Installation
Pathway requires Python 3.9 or above.
You can install the current release of Pathway using `pip`:
```
$ pip install -U pathway
```
⚠️ Pathway is available on MacOS and Linux. Users of other systems should run Pathway on a Virtual Machine.
## Getting started
To use Pathway, you only need to import it:
```python
import pathway as pw
```
Now, you can easily create your processing pipeline, and let Pathway handle the updates. Once your pipeline is created, you can launch the computation on streaming data with a one-line command:
```python
pw.run()
```
You can then run your Pathway project (say, `main.py`) just like a normal Python script: `$ python main.py`. Alternatively, use the pathway'ish version:
```
$ pathway spawn python main.py
```
Pathway natively supports multithreading.
To launch your application with 3 threads, you can do as follows:
```
$ pathway spawn --threads 3 python main.py
```
To jumpstart a Pathway project, you can use our [cookiecutter template](https://github.com/pathwaycom/cookiecutter-pathway).
### Example
```python
import pathway as pw
# Using the `demo` module to create a data stream
table = pw.demo.range_stream(nb_rows=50)
# Storing the stream into a CSV file
pw.io.csv.write(table, "output_table.csv")
# Summing all the values in a new table
sum_table = table.reduce(sum=pw.reducers.sum(pw.this.value))
# Storing the sum (which is a stream) in another CSV file
pw.io.csv.write(sum_table, "sum_table.csv")
# Now that the pipeline is built, the computation is started
pw.run()
```
Run this example [in Google Colab](https://colab.research.google.com/drive/1kLx5-vKKg0IeQ88ydS-ehtrxSujEZrXK?usp=sharing)!
## Monitoring Pathway
Pathway comes with a monitoring dashboard that allows you to keep track of the number of messages sent by each connector and the latency of the system. The dashboard also includes log messages.
This dashboard is enabled by default; you can disable it by passing `monitoring_level = pathway.MonitoringLevel.NONE` to `pathway.run()`.
<img src="https://d14l3brkh44201.cloudfront.net/pathway-dashboard.png" width="1326" alt="Pathway dashboard"/>
In addition to Pathway's built-in dashboard, you can [use Prometheus](https://pathway.com/developers/tutorials/prometheus-monitoring) to monitor your Pathway application.
## Resources
See also: **[Pathway Developer Resources](https://pathway.com/developers/)** webpage (including API Docs).
### Videos about Pathway
▶️ Building an LLM Application without a vector database (coming soon) - by [Jan Chorowski](https://scholar.google.com/citations?user=Yc94070AAAAJ)
[▶️ Linear regression on a Kafka Stream](https://vimeo.com/805069039) - by [Richard Pelgrim](https://twitter.com/richardpelgrim) (7min 53s)
[▶️ Introduction to reactive data processing](https://pathway.com/developers/user-guide/introduction/welcome) - by [Adrian Kosowski](https://scholar.google.com/citations?user=om8De_0AAAAJ) (27min 54s)
### Guides
- [Core concepts of Pathway](https://pathway.com/developers/user-guide/introduction/key-concepts/)
- [Basic operations](https://pathway.com/developers/user-guide/introduction/survival-guide/)
- [Joins](https://pathway.com/developers/user-guide/table-operations/join-manual/)
- [Groupby](https://pathway.com/developers/user-guide/table-operations/groupby-reduce-manual/)
- [Windowby](https://pathway.com/developers/user-guide/table-operations/windowby-reduce-manual/)
- [Transformer classes](https://pathway.com/developers/user-guide/transformer-classes/transformer-intro/)
- [Input and output connectors](https://pathway.com/developers/user-guide/input-and-output-streams/connectors/)
- [Coming from pandas](https://pathway.com/developers/user-guide/migrate-from-pandas/)
- [API docs](https://pathway.com/developers/api-docs/pathway)
- [Troubleshooting](https://pathway.com/developers/user-guide/introduction/troubleshooting/)
### Tutorials
- [Linear regression on a Kafka Stream](https://pathway.com/developers/tutorials/linear_regression_with_kafka/) ([video](https://vimeo.com/805069039))
- Joins:
- [Interval joins](https://pathway.com/developers/tutorials/fleet_eta_interval_join/)
- [Window joins](https://pathway.com/developers/tutorials/clickstream-window-join/)
- [ASOF joins](https://pathway.com/developers/tutorials/finance_ts_asof_join/)
- Connectors:
- [CSV connectors](https://pathway.com/developers/tutorials/connectors/csv_connectors/)
- [Database connectors](https://pathway.com/developers/tutorials/connectors/database-connectors/)
- [Kafka connectors](https://pathway.com/developers/tutorials/connectors/kafka_connectors/)
- [Custom Python connector](https://pathway.com/developers/tutorials/connectors/custom-python-connectors/)
- [Switching from Kafka to Redpanda](https://pathway.com/developers/tutorials/connectors/switching-to-redpanda/)
- [Monitoring Pathway with Prometheus](https://pathway.com/developers/tutorials/prometheus-monitoring/)
- [Time between events in a multi-topic event stream](https://pathway.com/developers/tutorials/event_stream_processing_time_between_occurrences/)
### Showcases
- [Realtime Twitter Analysis App](https://pathway.com/developers/showcases/twitter/)
- [Realtime classification with Nearest Neighbors](https://pathway.com/developers/showcases/lsh/lsh_chapter1/)
- [Realtime Fuzzy joins](https://pathway.com/developers/showcases/fuzzy_join/fuzzy_join_chapter1/)
### External and community content
- [Real-time linear regression (Data Engineering Weekly)](https://pathway.com/developers/tutorials/unlocking-data-stream-processing-1/)
- [Realtime server logs monitoring (Data Engineering Weekly)](https://pathway.com/developers/tutorials/unlocking-data-stream-processing-2/)
- [Data enrichment with fuzzy joins (Data Engineering Weekly)](https://pathway.com/developers/tutorials/unlocking-data-stream-processing-3/)
- [▶️ How to do Realtime Twitter Sentiment Analysis in Python (video)](https://www.youtube.com/watch?v=V7T3xHfjE4o)
If you would like to share with us some Pathway-related content, please give an admin a shout on [Discord](https://discord.gg/pathway).
### Manul conventions
Manuls (aka Pallas's Cats) [are creatures with fascinating habits](https://www.youtube.com/watch?v=rlSTBvViflc). As a tribute to them, we usually read `pw`, one of the most frequent tokens in Pathway code, as: `"paw"`.
<img src="https://d14l3brkh44201.cloudfront.net/PathwayManul.svg" alt="manul" width="50px"></img>
## Performance
Pathway is made to outperform state-of-the-art technologies designed for streaming and batch data processing tasks, including: Flink, Spark, and Kafka Streaming. It also makes it possible to implement a lot of algorithms/UDF's in streaming mode which are not readily supported by other streaming frameworks (especially: temporal joins, iterative graph algorithms, machine learning routines).
If you are curious, here are [some benchmarks to play with](https://github.com/pathwaycom/pathway-benchmarks).
<img src="https://github.com/pathwaycom/pathway-benchmarks/raw/main/images/bm-wordcount-lineplot.png" width="1326" alt="WordCount Graph"/>
If you try your own benchmarks, please don't hesitate to let us know. We investigate situations in which Pathway is underperforming on par with bugs (i.e., to our knowledge, they shouldn't happen...).
## Coming soon
Pathway continues to evolve and gain new capabilities. Here are some exciting new features that we plan to incorporate in the near future:
- Enhanced monitoring, observability, and data drift detection (integrates with Grafana visualization and other dashboarding tools).
- New connectors: interoperability with Delta Lake and Snowflake data sources.
- Easier connection setup for MongoDB.
- More performant garbage collection.
Stay up to date with the latest developments and news surrounding Pathway on [our blog](https://pathway.com/blog/), or [subscribe to our newsletter].
## Dependencies
Pathway is made to run in a "clean" Linux/MacOS + Python environment. When installing the pathway package with `pip` (from a wheel), you are likely to encounter a small number of Python package dependencies, such as sqlglot (used in the SQL API) and python-sat (useful for resolving dependencies during compilation). All necessary Rust crates are pre-built; the Rust compiler is not required to install Pathway, unless building from sources. A modified version of Timely/Differential Dataflow (which provides a dataflow assembly layer) is part of this repo.
## License
Pathway is distributed on a [BSL 1.1 License](LICENSE.txt) which allows for unlimited non-commercial use, as well as use of the Pathway package [for most commercial purposes](https://pathway.com/license/), free of charge. Code in this repository automatically converts to Open Source (Apache 2.0 License) after 4 years. Some [public repos](https://github.com/pathwaycom) which are complementary to this one (examples, libraries, connectors, etc.) are licensed as Open Source, under the MIT license.
## Contribution guidelines
If you develop a library or connector which you would like to integrate with this repo, we suggest releasing it first as a separate repo on a MIT/Apache 2.0 license.
For all concerns regarding core Pathway functionalities, Issues are encouraged. For further information, don't hesitate to engage with Pathway's [Discord community](https://discord.gg/pathway).

5
build.rs Normal file
View File

@ -0,0 +1,5 @@
fn main() {
println!("cargo:rerun-if-changed=build.rs");
pyo3_build_config::add_extension_module_link_args();
}

50
clippy.toml Normal file
View File

@ -0,0 +1,50 @@
[[disallowed-methods]]
path = "differential_dataflow::operators::Count::count"
reason = "use crate::dataflow::operators::MaybeTotal::count instead"
[[disallowed-methods]]
path = "differential_dataflow::operators::Threshold::distinct"
reason = "use crate::dataflow::operators::MaybeTotal::distinct instead"
[[disallowed-methods]]
path = "differential_dataflow::operators::arrange::arrangement::Arrange::arrange"
reason = "use crate::dataflow::operators::ArrangeWithTypes::arrange instead"
[[disallowed-methods]]
path = "differential_dataflow::operators::arrange::arrangement::Arrange::arrange_named"
reason = "use crate::dataflow::operators::ArrangeWithTypes::arrange_named instead"
[[disallowed-methods]]
path = "differential_dataflow::operators::arrange::arrangement::Arrange::arrange_core"
reason = "use crate::dataflow::operators::ArrangeWithTypes::arrange_named instead"
[[disallowed-methods]]
path = "differential_dataflow::operators::arrange::arrangement::ArrangeBySelf::arrange_by_self"
reason = "use crate::dataflow::operators::ArrangeWithTypes::arrange instead"
[[disallowed-methods]]
path = "differential_dataflow::operators::arrange::arrangement::ArrangeByKey::arrange_by_key"
reason = "use crate::dataflow::operators::ArrangeWithTypes::arrange instead"
[[disallowed-methods]]
path = "differential_dataflow::collection::Collection::map"
reason = "use crate::dataflow::operators::MapWrapped::map_ex or map_named instead"
[[disallowed-methods]]
path = "differential_dataflow::operators::join::Join::join_map"
reason = "use differential_dataflow::operators::join::JoinCore::join_core instead"
[[disallowed-methods]]
path = "differential_dataflow::operators::join::Join::join"
reason = "use differential_dataflow::operators::join::JoinCore::join_core instead"
[[disallowed-methods]]
path = "differential_dataflow::operators::join::Join::antijoin"
reason = "use differential_dataflow::operators::join::JoinCore::join_core instead"
[[disallowed-methods]]
path = "differential_dataflow::operators::join::Join::semijoin"
reason = "use differential_dataflow::operators::join::JoinCore::join_core instead"
# we do not export a Rust API yet
avoid-breaking-exported-api = false

View File

@ -0,0 +1,20 @@
language: rust
sudo: required
rust:
- stable
script:
- cargo build
- cargo test
- cargo bench
- cargo doc
after_success: |
[ $TRAVIS_BRANCH = master ] &&
[ $TRAVIS_PULL_REQUEST = false ] &&
cargo install mdbook &&
(cd mdbook; mdbook build) &&
sudo pip install ghp-import &&
ghp-import -n mdbook/book &&
git push -fq https://${GH_TOKEN}@github.com/${TRAVIS_REPO_SLUG}.git gh-pages
env:
global:
secure: d8tbB97FfXlLwGfzal2h/J2H7GdeR0dHBznqPfb/VxNPHevLJPtkD1Tnsqcs0/9w4piqmbKDzVWPAUXCA5Pg1e3f3pBqmgi12SYvzcFIHI1LLLbjF144S9vNNXhXc0IQE0bnaadX24A1JgRmmJJRt28qZnPxcCNdSJfrqV95h9Q=

View File

@ -0,0 +1,53 @@
[package]
name = "differential-dataflow"
version = "0.12.0"
authors = ["Frank McSherry <fmcsherry@me.com>"]
description = "An incremental data-parallel dataflow platform"
# These URLs point to more information about the repository
documentation = "https://docs.rs/differential-dataflow"
homepage = "https://github.com/TimelyDataflow/differential-dataflow"
repository = "https://github.com/TimelyDataflow/differential-dataflow.git"
keywords = ["differential", "dataflow"]
license = "MIT AND BUSL-1.1"
readme = "README.md"
autoexamples = true
[workspace]
members = [".", "dogsdogsdogs"]
[dev-dependencies]
bincode = "1.3.1"
rdkafka = "0.24"
indexmap = "1.0.1"
rand="0.4"
byteorder="1"
itertools="^0.7"
serde_json = "1.0"
graph_map = "0.1"
[dependencies]
serde = "1.0"
serde_derive = "1.0"
abomonation = "0.7"
abomonation_derive = "0.5"
timely = { path = "../timely-dataflow/timely", default-features = false }
fnv="1.0.2"
[features]
default = ["timely/getopts"]
columnation = ["timely/columnation"]
[[example]]
name = "columnation"
required-features = ["columnation"]
[profile.release]
opt-level = 3
debug = true
rpath = false
lto = true
debug-assertions = false
codegen-units = 4

View File

@ -0,0 +1,31 @@
This directory contains substantial portions of the "Differential Dataflow" project, as forked and modified by Pathway.
All Pathway code is distributed under the repository license: https://github.com/pathwaycom/pathway/LICENSE.
The original code of Differential Dataflow is distributed at https://github.com/TimelyDataflow/differential-dataflow,
and is licensed under The MIT License (MIT). We politely credit Frank McSherry and Contributors for this
amazing package. Conformant to the MIT license, an integral copy of the MIT license of the Differential Dataflow
project is provided in the Appendix below.
Contributions by Andrea Lattuada <andreal@student.ethz.ch> are Copyright (c) 2016 Andrea Lattuada, ETH Zürich.
Appendix: The MIT License (MIT)
Copyright (c) 2015 Frank McSherry
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all
copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
SOFTWARE.

View File

@ -0,0 +1,139 @@
// #![feature(vec_push_all)]
// #![feature(test)]
// #![feature(collections)]
//
// extern crate differential_dataflow;
// extern crate rand;
// extern crate test;
//
// use rand::{Rng, SeedableRng, StdRng, Rand};
// use test::Bencher;
//
// use differential_dataflow::sort::*;
// use differential_dataflow::sort::radix::*;
//
// fn random_vec<T: Rand>(size: usize) -> Vec<T> {
// let seed: &[_] = &[1, 2, 3, 4];
// let mut rng: StdRng = SeedableRng::from_seed(seed);
// // println!("{}", rng.gen::<f64>());
//
// let mut result = Vec::with_capacity(size);
// for _ in 0..size {
// result.push(rng.gen());
// }
//
// result
// }
//
// #[bench]
// fn bench_rand_exp8(bencher: &mut Bencher) {
// let mut data = random_vec::<(u32, u32)>(1 << 16);
//
// let mut src = Vec::new();
// for _ in 0..256 { src.push(Vec::with_capacity(data.len())); }
//
// let mut dst = Vec::new();
// for _ in 0..256 { dst.push(Vec::with_capacity(data.len())); }
//
// bencher.bytes = 1 << 16;
// bencher.iter(|| {
// for s in &mut src { s.clear(); }
// for d in &mut dst { d.clear(); }
// src[0].push_all(&data[..]);
//
// rsort_experimental8_buf(&mut src, &mut dst, &|x| x.0 as u64);
// rsort_experimental8_buf(&mut dst, &mut src, &|x| x.0 as u64 >> 8);
// rsort_experimental8_buf(&mut src, &mut dst, &|x| x.0 as u64 >> 16);
// rsort_experimental8_buf(&mut dst, &mut src, &|x| x.0 as u64 >> 24);
// for s in &mut src { isort_by(s, &|x| x); }
// });
// }
//
//
// fn bench_rand<T: Ord+Rand+Clone, F: FnMut(&mut[(u32, T)])>(bencher: &mut Bencher, mut func: F) {
// let mut data = random_vec::<(u32, T)>(1 << 16);
// let mut data2 = Vec::new();
//
// bencher.bytes = 1 << 16;
// bencher.iter(|| {
// data2.clear();
// data2.push_all(&data[..]);
// func(&mut data2)
// });
// }
//
// // fn bench_sort<T: Ord+Rand+Clone, F: FnMut(&mut[(u32, T)])>(bencher: &mut Bencher, mut func: F) {
// // let mut data = random_vec::<(u32, T)>(1 << 16);
// // let mut data2 = Vec::new();
// //
// // func(&mut data);
// //
// // bencher.bytes = 1 << 16;
// // bencher.iter(|| {
// // data2.clear();
// // data2.push_all(&data[..]);
// // func(&mut data2)
// // });
// // }
// // fn bench_same<T: Ord+Rand+Clone, F: FnMut(&mut[(u32, T)])>(bencher: &mut Bencher, mut func: F) {
// // let mut data = random_vec::<(u32, T)>(1 << 16);
// // let mut data2 = Vec::new();
// //
// // for i in 0..data.len() { data[i].0 = 0; }
// //
// // bencher.bytes = 1 << 16;
// // bencher.iter(|| {
// // data2.clear();
// // data2.push_all(&data[..]);
// // func(&mut data2)
// // });
// // }
//
// // #[bench] fn rand_sort(bencher: &mut Bencher) {
// // bench_rand::<u32, _>(bencher, |x| x.sort());
// // }
// //
// // #[bench] fn rand_qsort(bencher: &mut Bencher) {
// // bench_rand::<u32, _>(bencher, |x| qsort_by(x, &|y|&y.0));
// // }
//
// // #[bench] fn rand_rsort(bencher: &mut Bencher) {
// // bench_rand::<u32, _>(bencher, |x| rsort_msb(x, &|y|y.0 as u64, &|z| qsort_by(z, &|w|&w.0)));
// // }
// //
// // #[bench] fn rand_rsort_safe(bencher: &mut Bencher) {
// // bench_rand::<u32, _>(bencher, |x| rsort_msb_safe(x, &|y|y.0 as u64, &|z| qsort_by(z, &|w|&w.0)));
// // }
//
// #[bench] fn rand_rsort_bsaf(bencher: &mut Bencher) {
// let mut buffer = vec![(0u32, 0u32); 8 * 256];
// bench_rand::<u32, _>(bencher, |x| rsort_msb_buf(x, &mut buffer[..], &|y|y.0 as u64, &|z| qsort_by(z, &|q| q)));
// }
//
// #[bench] fn rand_rsort_c(bencher: &mut Bencher) {
// bench_rand::<u32, _>(bencher, |x| rsort_msb_clv(x, &|y|y.0 as u64, &|z| qsort_by(z, &|q| q)));
// }
//
// // #[bench] fn same_sort(bencher: &mut Bencher) {
// // bench_same::<(u32, u32), _>(bencher, |x| x.sort());
// // }
// //
// // #[bench] fn same_qsort(bencher: &mut Bencher) {
// // bench_same::<(u32, u32), _>(bencher, |x| qsort_by(x, &|y|&y.0));
// // }
// //
// // #[bench] fn same_rsort(bencher: &mut Bencher) {
// // bench_same::<(u32, u32), _>(bencher, |x| rsort_msb(x, &|y|y.0 as u64, &|z| qsort_by(z, &|w|&w.0)));
// // }
// //
// // #[bench] fn sort_sort(bencher: &mut Bencher) {
// // bench_sort::<(u32, u32), _>(bencher, |x| x.sort());
// // }
// //
// // #[bench] fn sort_qsort(bencher: &mut Bencher) {
// // bench_sort::<(u32, u32), _>(bencher, |x| qsort_by(x, &|y|&y.0));
// // }
// //
// // #[bench] fn sort_rsort(bencher: &mut Bencher) {
// // bench_sort::<(u32, u32), _>(bencher, |x| rsort_msb(x, &|y|y.0 as u64, &|z| qsort_by(z, &|w|&w.0)));
// // }

View File

@ -0,0 +1,19 @@
[package]
name = "dogsdogsdogs"
version = "0.1.0"
authors = ["Frank McSherry <fmcsherry@me.com>"]
license = "MIT"
[dependencies]
abomonation = "0.7"
abomonation_derive = "0.5"
timely = { git = "https://github.com/TimelyDataflow/timely-dataflow", default-features = false }
differential-dataflow = { path = "../", default-features = false }
serde = "1"
serde_derive = "1"
[dev-dependencies]
graph_map = "0.1"
[features]
default = ["timely/getopts"]

View File

@ -0,0 +1,205 @@
## Worst-case optimal joins and delta queries in differential dataflow
This project collects differential dataflow operators sufficient to implement memory-efficient delta queries, and worst-case optimal delta queries in the framework of [differential dataflow](https://github.com/frankmcsherry/differential-dataflow). Many of the ideas are a transportation of work done in the [`dataflow-join`](https://github.com/frankmcsherry/dataflow-join) project.
---
Imagine we have a collection of `(src, dst)` pairs describing a graph, and we want to determine the set of triangles:
triangles(a,b,c) := edge(a,b), edge(b,c), edge(a,c)
Differential dataflow provides a binary `join` operator that matches records in its two inputs with the same key, and produces the pairs of values. While this operator serves its role well, its naive implementation needs to maintain each of its inputs in indexed form to respond quickly to changes on either input.
In the example above, we might write
```rust
edges.join(&edges)
.map(|(a,(b,c))| ((b,c),a))
.semijoin(&edges)
.map(|((b,c),a)| (a,b,c));
```
This produces all triples `(a,b,c)` satisfying the triangle definition above, and it will update incrementally for arbitrary changes to the `edges` collection. Unfortunately, the first input to `semijoin` has size proportional to the sum of the squares of the degrees in the graph, which can be quite large.
The database community has a few remedies for this problem, and we will be borrowing two of them.
### Delta queries
Starting from the triangles query above, let's treat each of the three uses of `edges` as distinct relations.
triangles(a,b,c) := edge1(a,b), edge2(b,c), edge3(a,c)
We can write relational "delta queries" for how the `triangles` collection should change as a function of changes to any of the input relations.
d_tris1(a,b,c) := d_edge1(a,b), edge2(b,c), edge3(a,c)
d_tris2(a,b,c) := d_edge2(b,c), edge1(a,b), edge3(a,c)
d_tris3(a,b,c) := d_edge3(a,c), edge1(a,b), edge2(b,c)
All I've done here is written three rules by taking each relation in turn, replacing it by a version with a `d_` prefix, and then listing the remaining relations. Each of these queries tell us how to respond to a change to the associated relation.
Delta queries are appealing because they can be implemented in a stateless manner, maintaining only the indexed representations of `edges` (probably indexed both by source and by destination). They do *not* need to maintain intermediate collections, as differential dataflow does by default.
There is an important subtle detail, which the above example demonstrates: if there are simultaneous updates to many relations, we can't simply apply all delta queries at the same time. One change to `edges` could cause the three rules above to derive the same change multiple times. Instead, we need to execute the delta queries "in order", perhaps the order written above. Of course, we don't plan on actually doing that, but we can use timely dataflow's timestamps to impose a logical if not physical order on the execution.
In [examples/delta_query.rs](https://github.com/frankmcsherry/differential-dataflow/blob/master/dogsdogsdogs/examples/delta_query.rs#L52-L79) we can write these three rules as so:
```rust
// d_tris1(a,b,c) := d_edge1(a,b), edge2(b,c), edge3(a,c)
let d_tris1 = forward
.propose_using(&mut neu_forward.extend_using(|(_a,b)| *b))
.validate_using(&mut neu_forward.extend_using(|(a,_b)| *a))
.map(|((a,b),c)| (a,b,c));
// d_tris2(a,b,c) := d_edge2(b,c), edge1(a,b), edge3(a,c)
let d_tris2 = forward
.propose_using(&mut alt_reverse.extend_using(|(b,_c)| *b))
.validate_using(&mut neu_reverse.extend_using(|(_b,c)| *c))
.map(|((b,c),a)| (a,b,c));
// d_tris3(a,b,c) := d_edge3(a,c), edge1(a,b), edge2(b,c)
let d_tris3 = forward
.propose_using(&mut alt_forward.extend_using(|(a,_c)| *a))
.validate_using(&mut alt_reverse.extend_using(|(_a,c)| *c))
.map(|((a,c),b)| (a,b,c));
```
where the `alt` and `neu` prefixes refer to whether the data are old or new (old data can't "see" new data until the next timestamp), and the `forward` and `reverse` suffixes are the direction the data are indexed.
There is clearly a lot of code jargon going on here, but this all works and maintains a fixed memory footprint, proportional to the number of edges rather than the number of intermediate triangle candidates.
### Worst-case optimal joins
If you don't know what these are you are in for a treat.
Worst-case optimal joins are a new way of join processing that operate attribute-at-a-time, rather than relation-at-a-time, and which take time proportional to the worst-case possible result size set. For something like triangle computation, this is at most the number of edges raised to the power 1.5, which is better than the power 2, which is what a traditional relational engine would guarantee.
There are several worst-case optimal join algorithms, but my favorite comes from [the dataflow-join repository](https://github.com/frankmcsherry/dataflow-join). The gist here is that whenever you have a tuple like `(a,b)` to which you are thinking about adding a new attribute `c`, best to take stock of the possible ways that the new attribute `c` might be added. For example, in
d_tris1(a,b,c) := d_edge1(a,b), edge2(b,c), edge3(a,c)
the new `c` needs to be present in both `edge2` and in `edge3`. However, for a specific `(a,b)` pair, either of these two relations might propose more or fewer candidate `c` values. What if you were smart and first asked each relation to report the number of distinct values they would propose? Then you could have that relation propose the values, and the other relations validate these choices in time proportional to the number of proposals.
You can perform this in native differential dataflow over in the [examples/ngo.rs](https://github.com/frankmcsherry/differential-dataflow/blob/master/examples/ngo.rs) example. While cool, this version has the defect that the differential implementation again maintains intermediate state. This state can be much smaller than the naive joins, but it can also be much larger than the input relations.
Of course, we can combine the delta queries above with the worst-case optimal join processing, using all of the sweet loot in this project here. In [examples/delta_query_wcoj.rs](https://github.com/frankmcsherry/differential-dataflow/blob/master/dogsdogsdogs/examples/delta_query_wcoj.rs#L61-L87) we write the same delta queries, but implemented using worst-case optimal joins.
```rust
// d_tris1(a,b,c) := d_edge1(a,b), edge2(b,c), edge3(a,c)
let d_tris1 = forward
.extend(&mut [
&mut neu_forward.extend_using(|(_a,b)| *b),
&mut neu_forward.extend_using(|(a,_b)| *a),
])
.map(|((a,b),c)| (a,b,c));
// d_tris2(a,b,c) := d_edge2(b,c), edge1(a,b), edge3(a,c)
let d_tris2 = forward
.extend(&mut [
&mut alt_reverse.extend_using(|(b,_c)| *b),
&mut neu_reverse.extend_using(|(_b,c)| *c),
])
.map(|((b,c),a)| (a,b,c));
// d_tris3(a,b,c) := d_edge3(a,c), edge1(a,b), edge2(b,c)
let d_tris3 = forward
.extend(&mut [
&mut alt_forward.extend_using(|(a,_c)| *a),
&mut alt_reverse.extend_using(|(_a,c)| *c),
])
.map(|((a,c),b)| (a,b,c));
```
### Some numbers
Let's take these two implementations for a performance spin. None of these numbers are going to be nearly as exciting as for hand-rolled triangle counting implementations, sorry! However, they are fully differential, which means you could use them inside of an iterative Datalog computation, for example. Or with high refresh frequency, or distributed on multiple workers, etc. It's up to you to decide if these things are worth doing.
We have a few examples, including a naive differential WCOJ (examples/ngo.rs), and the delta queries and worst-case optimal delta queries. For the moment, let's just look at the second two.
If we spin up `examples/delta_query.rs` with the livejournal graph, asking it to introduce nodes (and all their edges) one at a time, we see
```
Echidnatron% cargo run --release --example delta_query -- ~/Projects/Datasets/livejournal 1
Finished release [optimized] target(s) in 0.07s
Running `target/release/examples/delta_query /Users/mcsherry/Projects/Datasets/livejournal 1`
3.429001ms Round 1 complete
32.624059ms Round 2 complete
38.686335ms Round 3 complete
41.348212ms Round 4 complete
44.279022ms Round 5 complete
...
```
which looks like a great start. However, it goes really fast and quickly gets "stuck". If we wait a moment, we see
```
425.859742ms Round 85 complete
426.276961ms Round 86 complete
426.550825ms Round 87 complete
26.317746169s Round 88 complete
26.318698709s Round 89 complete
26.319160175s Round 90 complete
...
```
which is pretty bad news.
What happened here is a great example of why you want worst-case optimal joins. It turns out that node 87 has 13,127 out-going edges. That means that the third update rule in the delta query
```rust
// d_tris3(a,b,c) := d_edge3(a,c), edge1(a,b), edge2(b,c)
let d_tris3 = forward
.propose_using(&mut alt_forward.extend_using(|(a,_c)| *a))
.validate_using(&mut alt_reverse.extend_using(|(_a,c)| *c))
.map(|((a,c),b)| (a,b,c));
```
takes in 13,127 changes and produces 172,318,129 proposals, being that number of changes *squared*. In this case, it was a horrible idea to ask `edge1` to propose changes to extend `d_edge3`. It would have been much better to ask `edge2` to do the proposals, especially because we are loading the graph in node order and there aren't so many reverse edges yet. Of course we don't want to bake that in as a rule, but the worst-case optimal implementation can figure this out automatically.
Let's see how `examples/delta_query_wcoj.rs` does:
```
Echidnatron% cargo run --release --example delta_query_wcoj -- ~/Projects/Datasets/livejournal 1
Finished release [optimized] target(s) in 0.14s
Running `target/release/examples/delta_query_wcoj /Users/mcsherry/Projects/Datasets/livejournal 1`
3.649981ms Round 1 complete
5.279384ms Round 2 complete
6.196575ms Round 3 complete
6.990896ms Round 4 complete
7.662358ms Round 5 complete
...
```
Lots faster, you might have noticed. Even without the horrible behavior seen above, most of these nodes have degree in the hundreds, and the implementation is smart enough to use the reverse directions when appropriate.
```
84.9538ms Round 85 complete
87.320066ms Round 86 complete
88.923623ms Round 87 complete
144.530066ms Round 88 complete
145.450224ms Round 89 complete
146.103021ms Round 90 complete
...
```
Node 87 still causes some mischief, but this has more to do with our computation ingesting those 13,127 edges and checking each of them to see what needs to be done. We still perform linear work, just not quadratic in this case.
You can take the examples out for spins with bigger batching if you want higher throughput. I wouldn't recommend this for the vanilla delta query (it has problems enough) but you can do e.g.
```
Echidnatron% cargo run --release --example delta_query_wcoj -- ~/Projects/Datasets/livejournal 1000
Finished release [optimized] target(s) in 0.07s
Running `target/release/examples/delta_query_wcoj /Users/mcsherry/Projects/Datasets/livejournal 1000`
675.443522ms Round 1000 complete
1.534094533s Round 2000 complete
2.341095966s Round 3000 complete
3.192567920s Round 4000 complete
3.952057645s Round 5000 complete
4.619403029s Round 6000 complete
5.617067697s Round 7000 complete
6.260885764s Round 8000 complete
7.649967738s Round 9000 complete
8.279152976s Round 10000 complete
...
```
This is showing us that we can perform over one thousand distinct updates each second (note: not a batch of one thousand updates), though this number will probably drop off as we let it run (the graph fills out). There is still plenty to improve and work on, and at the moment this is more about unique functionality rather than raw performance.

Binary file not shown.

After

Width:  |  Height:  |  Size: 802 KiB

View File

@ -0,0 +1,160 @@
extern crate timely;
extern crate graph_map;
extern crate differential_dataflow;
extern crate dogsdogsdogs;
use timely::dataflow::Scope;
use timely::dataflow::operators::probe::Handle;
use differential_dataflow::input::Input;
use differential_dataflow::operators::JoinCore;
use graph_map::GraphMMap;
use dogsdogsdogs::altneu::AltNeu;
use dogsdogsdogs::calculus::{Differentiate, Integrate};
fn main() {
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
let batching = std::env::args().nth(2).unwrap().parse::<usize>().unwrap();
let inspect = std::env::args().any(|x| x == "inspect");
timely::execute_from_args(std::env::args().skip(2), move |worker| {
let timer = std::time::Instant::now();
let graph = GraphMMap::new(&filename);
let peers = worker.peers();
let index = worker.index();
let mut probe = Handle::new();
let mut input = worker.dataflow::<usize,_,_>(|scope| {
let (edges_input, edges) = scope.new_collection();
// Graph oriented both ways, indexed by key.
use differential_dataflow::operators::arrange::ArrangeByKey;
let forward_key = edges.arrange_by_key();
let reverse_key = edges.map(|(x,y)| (y,x))
.arrange_by_key();
// Graph oriented both ways, indexed by (key, val).
use differential_dataflow::operators::arrange::ArrangeBySelf;
let forward_self = edges.arrange_by_self();
let reverse_self = edges.map(|(x,y)| (y,x))
.arrange_by_self();
// // Graph oriented both ways, counts of distinct vals for each key.
// // Not required without worst-case-optimal join strategy.
// let forward_count = edges.map(|(x,y)| x).arrange_by_self();
// let reverse_count = edges.map(|(x,y)| y).arrange_by_self();
// Q(a,b,c) := E1(a,b), E2(b,c), E3(a,c)
let (triangles_prev, triangles_next) = scope.scoped::<AltNeu<usize>,_,_>("DeltaQuery (Triangles)", |inner| {
// Grab the stream of changes.
let changes = edges.enter(inner);
// Each relation we'll need.
let forward_key_alt = forward_key.enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), |t| t.time.saturating_sub(1));
let reverse_key_alt = reverse_key.enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), |t| t.time.saturating_sub(1));
let forward_key_neu = forward_key.enter_at(inner, |_,_,t| AltNeu::neu(t.clone()), |t| t.time.saturating_sub(1));
// let reverse_key_neu = reverse_key.enter_at(inner, |_,_,t| AltNeu::neu(t.clone()), |t| t.time.saturating_sub(1));
// let forward_self_alt = forward_self.enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), |t| t.time.saturating_sub(1));
let reverse_self_alt = reverse_self.enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), |t| t.time.saturating_sub(1));
let forward_self_neu = forward_self.enter_at(inner, |_,_,t| AltNeu::neu(t.clone()), |t| t.time.saturating_sub(1));
let reverse_self_neu = reverse_self.enter_at(inner, |_,_,t| AltNeu::neu(t.clone()), |t| t.time.saturating_sub(1));
// For each relation, we form a delta query driven by changes to that relation.
//
// The sequence of joined relations are such that we only introduce relations
// which share some bound attributes with the current stream of deltas.
// Each joined relation is delayed { alt -> neu } if its position in the
// sequence is greater than the delta stream.
// Each joined relation is directed { forward, reverse } by whether the
// bound variable occurs in the first or second position.
let key1 = |x: &(u32, u32)| x.0;
let key2 = |x: &(u32, u32)| x.1;
use dogsdogsdogs::operators::propose;
use dogsdogsdogs::operators::validate;
// Prior technology
// dQ/dE1 := dE1(a,b), E2(b,c), E3(a,c)
let changes1 = propose(&changes, forward_key_neu.clone(), key2.clone());
let changes1 = validate(&changes1, forward_self_neu.clone(), key1.clone());
let changes1 = changes1.map(|((a,b),c)| (a,b,c));
// dQ/dE2 := dE2(b,c), E1(a,b), E3(a,c)
let changes2 = propose(&changes, reverse_key_alt.clone(), key1.clone());
let changes2 = validate(&changes2, reverse_self_neu.clone(), key2.clone());
let changes2 = changes2.map(|((b,c),a)| (a,b,c));
// dQ/dE3 := dE3(a,c), E1(a,b), E2(b,c)
let changes3 = propose(&changes, forward_key_alt.clone(), key1.clone());
let changes3 = validate(&changes3, reverse_self_alt.clone(), key2.clone());
let changes3 = changes3.map(|((a,c),b)| (a,b,c));
let prev_changes = changes1.concat(&changes2).concat(&changes3).leave();
// New ideas
let d_edges = edges.differentiate(inner);
// dQ/dE1 := dE1(a,b), E2(b,c), E3(a,c)
let changes1 =
d_edges
.map(|(x,y)| (y,x))
.join_core(&forward_key_neu, |b,a,c| Some(((*a, *c), *b)))
.join_core(&forward_self_neu, |(a,c), b, &()| Some((*a,*b,*c)));
// dQ/dE2 := dE2(b,c), E1(a,b), E3(a,c)
let changes2 =
d_edges
.join_core(&reverse_key_alt, |b,c,a| Some(((*a, *c), *b)))
.join_core(&forward_self_neu, |(a,c), b, &()| Some((*a,*b,*c)));
// dQ/dE3 := dE3(a,c), E1(a,b), E2(b,c)
let changes3 =
d_edges
.join_core(&forward_key_alt, |a,c,b| Some(((*c, *b), *a)))
.join_core(&reverse_self_alt, |(c,b), a, &()| Some((*a,*b,*c)));
let next_changes = changes1.concat(&changes2).concat(&changes3).integrate();
(prev_changes, next_changes)
});
// Test if our two methods do the same thing.
triangles_prev.assert_eq(&triangles_next);
triangles_prev
.filter(move |_| inspect)
.inspect(|x| println!("\tTriangle: {:?}", x))
.probe_with(&mut probe);
edges_input
});
let mut index = index;
while index < graph.nodes() {
input.advance_to(index);
for &edge in graph.edges(index) {
input.insert((index as u32, edge));
}
index += peers;
input.advance_to(index);
input.flush();
if (index / peers) % batching == 0 {
while probe.less_than(input.time()) {
worker.step();
}
println!("{:?}\tRound {} complete", timer.elapsed(), index);
}
}
}).unwrap();
}

View File

@ -0,0 +1,85 @@
extern crate timely;
extern crate graph_map;
extern crate differential_dataflow;
extern crate dogsdogsdogs;
use timely::dataflow::Scope;
use timely::order::Product;
use timely::dataflow::operators::probe::Handle;
use timely::dataflow::operators::UnorderedInput;
use timely::dataflow::operators::Map;
use differential_dataflow::AsCollection;
fn main() {
timely::execute_from_args(std::env::args().skip(2), move |worker| {
let mut probe = Handle::new();
let (mut i1, mut i2, c1, c2) = worker.dataflow::<usize,_,_>(|scope| {
// Nested scope as `Product<usize, usize>` doesn't refine `()`, because .. coherence.
scope.scoped("InnerScope", |inner| {
use timely::dataflow::operators::unordered_input::UnorderedHandle;
let ((input1, capability1), data1): ((UnorderedHandle<Product<usize, usize>, ((usize, usize), Product<usize, usize>, isize)>, _), _) = inner.new_unordered_input();
let ((input2, capability2), data2): ((UnorderedHandle<Product<usize, usize>, ((usize, usize), Product<usize, usize>, isize)>, _), _) = inner.new_unordered_input();
let edges1 = data1.as_collection();
let edges2 = data2.as_collection();
// Graph oriented both ways, indexed by key.
use differential_dataflow::operators::arrange::ArrangeByKey;
let forward1 = edges1.arrange_by_key();
let forward2 = edges2.arrange_by_key();
// Grab the stream of changes. Stash the initial time as payload.
let changes1 = edges1.inner.map(|((k,v),t,r)| ((k,v,t.clone()),t,r)).as_collection();
let changes2 = edges2.inner.map(|((k,v),t,r)| ((k,v,t.clone()),t,r)).as_collection();
use dogsdogsdogs::operators::half_join;
// pick a frontier that will not mislead TOTAL ORDER comparisons.
let closure = |time: &Product<usize, usize>| Product::new(time.outer.saturating_sub(1), time.inner.saturating_sub(1));
let path1 =
half_join(
&changes1,
forward2,
closure.clone(),
|t1,t2| t1.lt(t2), // This one ignores concurrent updates.
|key, val1, val2| (key.clone(), (val1.clone(), val2.clone())),
);
let path2 =
half_join(
&changes2,
forward1,
closure.clone(),
|t1,t2| t1.le(t2), // This one can "see" concurrent updates.
|key, val1, val2| (key.clone(), (val2.clone(), val1.clone())),
);
// Delay updates until the worked payload time.
// This should be at least the ignored update time.
path1.concat(&path2)
.inner.map(|(((k,v),t),_,r)| ((k,v),t,r)).as_collection()
.inspect(|x| println!("{:?}", x))
.probe_with(&mut probe);
(input1, input2, capability1, capability2)
})
});
i1
.session(c1.clone())
.give(((5, 6), Product::new(0, 13), 1));
i2
.session(c2.clone())
.give(((5, 7), Product::new(11, 0), 1));
}).unwrap();
}

View File

@ -0,0 +1,116 @@
extern crate timely;
extern crate graph_map;
extern crate differential_dataflow;
extern crate dogsdogsdogs;
use timely::dataflow::Scope;
use timely::dataflow::operators::probe::Handle;
use differential_dataflow::input::Input;
use graph_map::GraphMMap;
use dogsdogsdogs::{CollectionIndex, altneu::AltNeu};
use dogsdogsdogs::{ProposeExtensionMethod};
fn main() {
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
let batching = std::env::args().nth(2).unwrap().parse::<usize>().unwrap();
let inspect = std::env::args().any(|x| x == "inspect");
timely::execute_from_args(std::env::args().skip(2), move |worker| {
let timer = std::time::Instant::now();
let graph = GraphMMap::new(&filename);
let peers = worker.peers();
let index = worker.index();
let mut probe = Handle::new();
let mut input = worker.dataflow::<usize,_,_>(|scope| {
let (edges_input, edges) = scope.new_collection();
let forward = edges.clone();
let reverse = edges.map(|(x,y)| (y,x));
// Q(a,b,c) := E1(a,b), E2(b,c), E3(a,c)
let triangles = scope.scoped::<AltNeu<usize>,_,_>("DeltaQuery (Triangles)", |inner| {
// Each relation we'll need.
let forward = forward.enter(inner);
let reverse = reverse.enter(inner);
// Without using wrappers yet, maintain an "old" and a "new" copy of edges.
let alt_forward = CollectionIndex::index(&forward);
let alt_reverse = CollectionIndex::index(&reverse);
let neu_forward = CollectionIndex::index(&forward.delay(|time| AltNeu::neu(time.time.clone())));
let neu_reverse = CollectionIndex::index(&reverse.delay(|time| AltNeu::neu(time.time.clone())));
// For each relation, we form a delta query driven by changes to that relation.
//
// The sequence of joined relations are such that we only introduce relations
// which share some bound attributes with the current stream of deltas.
// Each joined relation is delayed { alt -> neu } if its position in the
// sequence is greater than the delta stream.
// Each joined relation is directed { forward, reverse } by whether the
// bound variable occurs in the first or second position.
// dQ/dE1 := dE1(a,b), E2(b,c), E3(a,c)
let changes1 =
forward
.extend(&mut [
&mut neu_forward.extend_using(|(_a,b)| *b),
&mut neu_forward.extend_using(|(a,_b)| *a),
])
.map(|((a,b),c)| (a,b,c));
// dQ/dE2 := dE2(b,c), E1(a,b), E3(a,c)
let changes2 =
forward
.extend(&mut [
&mut alt_reverse.extend_using(|(b,_c)| *b),
&mut neu_reverse.extend_using(|(_b,c)| *c),
])
.map(|((b,c),a)| (a,b,c));
// dQ/dE3 := dE3(a,c), E1(a,b), E2(b,c)
let changes3 = forward
.extend(&mut [
&mut alt_forward.extend_using(|(a,_c)| *a),
&mut alt_reverse.extend_using(|(_a,c)| *c),
])
.map(|((a,c),b)| (a,b,c));
changes1.concat(&changes2).concat(&changes3).leave()
});
triangles
.filter(move |_| inspect)
.inspect(|x| println!("\tTriangle: {:?}", x))
.probe_with(&mut probe);
edges_input
});
let mut index = index;
while index < graph.nodes() {
input.advance_to(index);
for &edge in graph.edges(index) {
input.insert((index as u32, edge));
}
index += peers;
input.advance_to(index);
input.flush();
if (index / peers) % batching == 0 {
while probe.less_than(input.time()) {
worker.step();
}
println!("{:?}\tRound {} complete", timer.elapsed(), index);
}
}
}).unwrap();
}

View File

@ -0,0 +1,95 @@
extern crate timely;
extern crate graph_map;
extern crate differential_dataflow;
extern crate dogsdogsdogs;
use timely::dataflow::operators::{ToStream, Partition, Accumulate, Inspect, Probe};
use timely::dataflow::operators::probe::Handle;
use differential_dataflow::{Collection, AsCollection};
use differential_dataflow::input::Input;
use graph_map::GraphMMap;
use dogsdogsdogs::{CollectionIndex, PrefixExtender};
fn main() {
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
let batching = std::env::args().nth(2).unwrap().parse::<usize>().unwrap();
timely::execute_from_args(std::env::args().skip(2), move |worker| {
// let timer = std::time::Instant::now();
let peers = worker.peers();
let index = worker.index();
// // What you might do if you used GraphMMap:
let graph = GraphMMap::new(&filename);
let nodes = graph.nodes();
let edges = (0..nodes).filter(move |node| node % peers == index)
.flat_map(|node| graph.edges(node).iter().cloned().map(move |dst| ((node as u32, dst))))
.map(|(src, dst)| ((src, dst), Default::default(), 1))
.collect::<Vec<_>>();
let edges2 = edges.clone();
println!("loaded {} nodes, {} edges", nodes, edges.len());
let index = worker.dataflow::<usize,_,_>(|scope| {
CollectionIndex::index(&Collection::new(edges.to_stream(scope)))
});
let mut index_xz = index.extend_using(|&(ref x, ref _y)| *x);
let mut index_yz = index.extend_using(|&(ref _x, ref y)| *y);
let mut probe = Handle::new();
let mut edges = worker.dataflow::<usize,_,_>(|scope| {
let (edges_input, edges) = scope.new_collection();
// determine stream of (prefix, count, index) indicating relation with fewest extensions.
let counts = edges.map(|p| (p, usize::max_value(), usize::max_value()));
let counts0 = index_xz.count(&counts, 0);
let counts1 = index_yz.count(&counts0, 1);
// partition by index.
let parts = counts1.inner.partition(2, |((p, _c, i),t,d)| (i as u64,(p,t,d)));
// propose extensions using relation based on index.
let propose0 = index_xz.propose(&parts[0].as_collection());
let propose1 = index_yz.propose(&parts[1].as_collection());
// validate proposals with the other index.
let validate0 = index_yz.validate(&propose0);
let validate1 = index_xz.validate(&propose1);
validate0
.concat(&validate1)
.inner
.count()
.inspect(move |x| println!("{:?}", x))
// .inspect(move |x| println!("{:?}:\t{:?}", timer.elapsed(), x))
.probe_with(&mut probe);
edges_input
});
let mut index = 0;
while index < edges2.len() {
let limit = std::cmp::min(batching, edges2.len() - index);
for offset in 0 .. limit {
edges.insert(edges2[index + offset].0);
edges.advance_to(index + offset + 1);
}
index += limit;
edges.flush();
while probe.less_than(edges.time()) {
worker.step();
}
}
}).unwrap();
}

View File

@ -0,0 +1,98 @@
extern crate timely;
extern crate graph_map;
extern crate differential_dataflow;
use std::hash::Hash;
use timely::dataflow::*;
use timely::dataflow::operators::*;
use differential_dataflow::Collection;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::*;
use differential_dataflow::operators::arrange::ArrangeBySelf;
use differential_dataflow::operators::arrange::ArrangeByKey;
use graph_map::GraphMMap;
type Node = u32;
type Edge = (Node, Node);
fn main() {
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
timely::execute_from_args(std::env::args().skip(1), move |worker| {
let peers = worker.peers();
let index = worker.index();
// // What you might do if you used GraphMMap:
let graph = GraphMMap::new(&filename);
let nodes = graph.nodes();
let edges = (0..nodes).filter(move |node| node % peers == index)
.flat_map(|node| graph.edges(node).iter().cloned().map(move |dst| ((node as u32, dst))))
.map(|(src, dst)| ((src, dst), Default::default(), 1))
.collect::<Vec<_>>();
println!("loaded {} nodes, {} edges", nodes, edges.len());
worker.dataflow::<(),_,_>(|scope| {
triangles(&Collection::new(edges.to_stream(scope))).inner.count().inspect(|x| println!("{:?}", x));
});
}).unwrap();
}
fn triangles<G: Scope>(edges: &Collection<G, Edge>) -> Collection<G, (Node, Node, Node)>
where G::Timestamp: Lattice+Hash+Ord {
// only use forward-pointing edges.
let edges = edges.filter(|&(src, dst)| src < dst);
// arrange the edge relation three ways.
let as_self = edges.arrange_by_self();
let forward = edges.arrange_by_key();
let reverse = edges.map_in_place(|x| ::std::mem::swap(&mut x.0, &mut x.1))
.arrange_by_key();
// arrange the count of extensions from each source.
let counts = edges.map(|(src, _dst)| src)
.arrange_by_self();
// extract ((src, dst), idx) tuples with weights equal to the number of extensions.
let cand_count1 = forward.join_core(&counts, |&src, &dst, &()| Some(((src, dst), 1)));
let cand_count2 = reverse.join_core(&counts, |&dst, &src, &()| Some(((src, dst), 2)));
// determine for each (src, dst) tuple which index would propose the fewest extensions.
let winners = cand_count1.concat(&cand_count2)
.reduce(|_srcdst, counts, output| {
if counts.len() == 2 {
let mut min_cnt = isize::max_value();
let mut min_idx = usize::max_value();
for &(&idx, cnt) in counts.iter() {
if min_cnt > cnt {
min_idx = idx;
min_cnt = cnt;
}
}
output.push((min_idx, 1));
}
});
// select tuples with the first relation minimizing the proposals, join, then intersect.
let winners1 = winners.flat_map(|((src, dst), index)| if index == 1 { Some((src, dst)) } else { None })
.join_core(&forward, |&src, &dst, &ext| Some(((dst, ext), src)))
.join_core(&as_self, |&(dst, ext), &src, &()| Some(((dst, ext), src)))
.map(|((dst, ext), src)| (src, dst, ext));
// select tuples with the second relation minimizing the proposals, join, then intersect.
let winners2 = winners.flat_map(|((src, dst), index)| if index == 2 { Some((dst, src)) } else { None })
.join_core(&forward, |&dst, &src, &ext| Some(((src, ext), dst)))
.join_core(&as_self, |&(src, ext), &dst, &()| Some(((src, ext), dst)))
.map(|((src, ext), dst)| (src, dst, ext));
// collect and return results.
winners1.concat(&winners2)
}

View File

@ -0,0 +1,100 @@
//! A lexicographically ordered pair of timestamps.
//!
//! Two timestamps (s1, t1) and (s2, t2) are ordered either if
//! s1 and s2 are ordered, or if s1 equals s2 and t1 and t2 are
//! ordered.
//!
//! The join of two timestamps should have as its first coordinate
//! the join of the first coordinates, and for its second coordinate
//! the join of the second coordinates for elements whose first
//! coordinate equals the computed join. That may be the minimum
//! element of the second lattice, if neither first element equals
//! the join.
/// A pair of timestamps, partially ordered by the product order.
#[derive(Debug, Hash, Default, Clone, Eq, PartialEq, Ord, PartialOrd, Abomonation, Serialize, Deserialize)]
pub struct AltNeu<T> {
pub time: T,
pub neu: bool, // alt < neu in timestamp comparisons.
}
impl<T> AltNeu<T> {
pub fn alt(time: T) -> Self { AltNeu { time, neu: false } }
pub fn neu(time: T) -> Self { AltNeu { time, neu: true } }
}
// Implement timely dataflow's `PartialOrder` trait.
use timely::order::PartialOrder;
impl<T: PartialOrder> PartialOrder for AltNeu<T> {
fn less_equal(&self, other: &Self) -> bool {
if self.time.eq(&other.time) {
self.neu <= other.neu
}
else {
self.time.less_equal(&other.time)
}
}
}
// Implement timely dataflow's `PathSummary` trait.
// This is preparation for the `Timestamp` implementation below.
use timely::progress::PathSummary;
impl<T: Timestamp> PathSummary<AltNeu<T>> for () {
fn results_in(&self, timestamp: &AltNeu<T>) -> Option<AltNeu<T>> {
Some(timestamp.clone())
}
fn followed_by(&self, other: &Self) -> Option<Self> {
Some(other.clone())
}
}
// Implement timely dataflow's `Timestamp` trait.
use timely::progress::Timestamp;
impl<T: Timestamp> Timestamp for AltNeu<T> {
type Summary = ();
fn minimum() -> Self { AltNeu::alt(T::minimum()) }
}
use timely::progress::timestamp::Refines;
impl<T: Timestamp> Refines<T> for AltNeu<T> {
fn to_inner(other: T) -> Self {
AltNeu::alt(other)
}
fn to_outer(self: AltNeu<T>) -> T {
self.time
}
fn summarize(_path: ()) -> <T as Timestamp>::Summary {
Default::default()
}
}
// Implement differential dataflow's `Lattice` trait.
// This extends the `PartialOrder` implementation with additional structure.
use differential_dataflow::lattice::Lattice;
impl<T: Lattice> Lattice for AltNeu<T> {
fn join(&self, other: &Self) -> Self {
let time = self.time.join(&other.time);
let mut neu = false;
if time == self.time {
neu = neu || self.neu;
}
if time == other.time {
neu = neu || other.neu;
}
AltNeu { time, neu }
}
fn meet(&self, other: &Self) -> Self {
let time = self.time.meet(&other.time);
let mut neu = true;
if time == self.time {
neu = neu && self.neu;
}
if time == other.time {
neu = neu && other.neu;
}
AltNeu { time, neu }
}
}

View File

@ -0,0 +1,65 @@
//! Traits and implementations for differentiating and integrating collections.
//!
//! The `Differentiate` and `Integrate` traits allow us to move between standard differential
//! dataflow collections, and collections that describe their instantaneous change. The first
//! trait converts a collection to one that contains each change at the moment it occurs, but
//! then immediately retracting it. The second trait takes such a representation are recreates
//! the collection from its instantaneous changes.
//!
//! These two traits together allow us to build dataflows that maintain computates over inputs
//! that are the instantaneous changes, and then to reconstruct collections from them. The most
//! clear use case for this are "delta query" implementations of relational joins, where linearity
//! allows us to write dataflows based on instantaneous changes, whose "accumluated state" is
//! almost everywhere empty (and so has a low memory footprint, if the system works as planned).
use timely::dataflow::Scope;
use timely::dataflow::scopes::Child;
use timely::dataflow::operators::{Filter, Map};
use differential_dataflow::{AsCollection, Collection, Data};
use differential_dataflow::difference::Abelian;
use crate::altneu::AltNeu;
/// Produce a collection containing the changes at the moments they happen.
pub trait Differentiate<G: Scope, D: Data, R: Abelian> {
fn differentiate<'a>(&self, child: &Child<'a, G, AltNeu<G::Timestamp>>) -> Collection<Child<'a, G, AltNeu<G::Timestamp>>, D, R>;
}
/// Collect instantaneous changes back in to a collection.
pub trait Integrate<G: Scope, D: Data, R: Abelian> {
fn integrate(&self) -> Collection<G, D, R>;
}
impl<G, D, R> Differentiate<G, D, R> for Collection<G, D, R>
where
G: Scope,
D: Data,
R: Abelian,
{
// For each (data, Alt(time), diff) we add a (data, Neu(time), -diff).
fn differentiate<'a>(&self, child: &Child<'a, G, AltNeu<G::Timestamp>>) -> Collection<Child<'a, G, AltNeu<G::Timestamp>>, D, R> {
self.enter(child)
.inner
.flat_map(|(data, time, diff)| {
let neu = (data.clone(), AltNeu::neu(time.time.clone()), diff.clone().negate());
let alt = (data, time, diff);
Some(alt).into_iter().chain(Some(neu))
})
.as_collection()
}
}
impl<'a, G, D, R> Integrate<G, D, R> for Collection<Child<'a, G, AltNeu<G::Timestamp>>, D, R>
where
G: Scope,
D: Data,
R: Abelian,
{
// We discard each `neu` variant and strip off the `alt` wrapper.
fn integrate(&self) -> Collection<G, D, R> {
self.inner
.filter(|(_d,t,_r)| !t.neu)
.as_collection()
.leave()
}
}

View File

@ -0,0 +1,218 @@
#[macro_use]
extern crate abomonation_derive;
extern crate abomonation;
extern crate timely;
extern crate differential_dataflow;
#[macro_use]
extern crate serde_derive;
extern crate serde;
use std::hash::Hash;
use timely::dataflow::Scope;
use timely::progress::Timestamp;
use timely::dataflow::operators::Partition;
use timely::dataflow::operators::Concatenate;
use differential_dataflow::{ExchangeData, Collection, AsCollection};
use differential_dataflow::operators::Threshold;
use differential_dataflow::difference::{Monoid, Multiply};
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::arrange::TraceAgent;
use differential_dataflow::operators::arrange::{ArrangeBySelf, ArrangeByKey};
pub mod altneu;
pub mod calculus;
pub mod operators;
/// A type capable of extending a stream of prefixes.
///
/**
Implementors of `PrefixExtension` provide types and methods for extending a differential dataflow collection,
via the three methods `count`, `propose`, and `validate`.
**/
pub trait PrefixExtender<G: Scope, R: Monoid+Multiply<Output = R>> {
/// The required type of prefix to extend.
type Prefix;
/// The type to be produced as extension.
type Extension;
/// Annotates prefixes with the number of extensions the relation would propose.
fn count(&mut self, prefixes: &Collection<G, (Self::Prefix, usize, usize), R>, index: usize) -> Collection<G, (Self::Prefix, usize, usize), R>;
/// Extends each prefix with corresponding extensions.
fn propose(&mut self, prefixes: &Collection<G, Self::Prefix, R>) -> Collection<G, (Self::Prefix, Self::Extension), R>;
/// Restricts proposed extensions by those the extender would have proposed.
fn validate(&mut self, extensions: &Collection<G, (Self::Prefix, Self::Extension), R>) -> Collection<G, (Self::Prefix, Self::Extension), R>;
}
pub trait ProposeExtensionMethod<G: Scope, P: ExchangeData+Ord, R: Monoid+Multiply<Output = R>> {
fn propose_using<PE: PrefixExtender<G, R, Prefix=P>>(&self, extender: &mut PE) -> Collection<G, (P, PE::Extension), R>;
fn extend<E: ExchangeData+Ord>(&self, extenders: &mut [&mut dyn PrefixExtender<G,R,Prefix=P,Extension=E>]) -> Collection<G, (P, E), R>;
}
impl<G, P, R> ProposeExtensionMethod<G, P, R> for Collection<G, P, R>
where
G: Scope,
P: ExchangeData+Ord,
R: Monoid+Multiply<Output = R>,
{
fn propose_using<PE>(&self, extender: &mut PE) -> Collection<G, (P, PE::Extension), R>
where
PE: PrefixExtender<G, R, Prefix=P>
{
extender.propose(self)
}
fn extend<E>(&self, extenders: &mut [&mut dyn PrefixExtender<G,R,Prefix=P,Extension=E>]) -> Collection<G, (P, E), R>
where
E: ExchangeData+Ord
{
if extenders.len() == 1 {
extenders[0].propose(&self.clone())
}
else {
let mut counts = self.map(|p| (p, 1 << 31, 0));
for (index,extender) in extenders.iter_mut().enumerate() {
counts = extender.count(&counts, index);
}
let parts = counts.inner.partition(extenders.len() as u64, |((p, _, i),t,d)| (i as u64, (p,t,d)));
let mut results = Vec::new();
for (index, nominations) in parts.into_iter().enumerate() {
let mut extensions = extenders[index].propose(&nominations.as_collection());
for other in (0..extenders.len()).filter(|&x| x != index) {
extensions = extenders[other].validate(&extensions);
}
results.push(extensions.inner); // save extensions
}
self.scope().concatenate(results).as_collection()
}
}
}
pub trait ValidateExtensionMethod<G: Scope, R: Monoid+Multiply<Output = R>, P, E> {
fn validate_using<PE: PrefixExtender<G, R, Prefix=P, Extension=E>>(&self, extender: &mut PE) -> Collection<G, (P, E), R>;
}
impl<G: Scope, R: Monoid+Multiply<Output = R>, P, E> ValidateExtensionMethod<G, R, P, E> for Collection<G, (P, E), R> {
fn validate_using<PE: PrefixExtender<G, R, Prefix=P, Extension=E>>(&self, extender: &mut PE) -> Collection<G, (P, E), R> {
extender.validate(self)
}
}
// These are all defined here so that users can be assured a common layout.
use differential_dataflow::trace::implementations::ord::{OrdKeySpine, OrdValSpine};
type TraceValHandle<K,V,T,R> = TraceAgent<OrdValSpine<K,V,T,R>>;
type TraceKeyHandle<K,T,R> = TraceAgent<OrdKeySpine<K,T,R>>;
pub struct CollectionIndex<K, V, T, R>
where
K: ExchangeData,
V: ExchangeData,
T: Lattice+ExchangeData+Timestamp,
R: Monoid+Multiply<Output = R>+ExchangeData,
{
/// A trace of type (K, ()), used to count extensions for each prefix.
count_trace: TraceKeyHandle<K, T, isize>,
/// A trace of type (K, V), used to propose extensions for each prefix.
propose_trace: TraceValHandle<K, V, T, R>,
/// A trace of type ((K, V), ()), used to validate proposed extensions.
validate_trace: TraceKeyHandle<(K, V), T, R>,
}
impl<K, V, T, R> Clone for CollectionIndex<K, V, T, R>
where
K: ExchangeData+Hash,
V: ExchangeData+Hash,
T: Lattice+ExchangeData+Timestamp,
R: Monoid+Multiply<Output = R>+ExchangeData,
{
fn clone(&self) -> Self {
CollectionIndex {
count_trace: self.count_trace.clone(),
propose_trace: self.propose_trace.clone(),
validate_trace: self.validate_trace.clone(),
}
}
}
impl<K, V, T, R> CollectionIndex<K, V, T, R>
where
K: ExchangeData+Hash,
V: ExchangeData+Hash,
T: Lattice+ExchangeData+Timestamp,
R: Monoid+Multiply<Output = R>+ExchangeData,
{
pub fn index<G: Scope<Timestamp = T>>(collection: &Collection<G, (K, V), R>) -> Self {
// We need to count the number of (k, v) pairs and not rely on the given Monoid R and its binary addition operation.
// counts and validate can share the base arrangement
let arranged = collection.arrange_by_self();
let counts = arranged
.distinct()
.map(|(k, _v)| k)
.arrange_by_self()
.trace;
let propose = collection.arrange_by_key().trace;
let validate = arranged.trace;
CollectionIndex {
count_trace: counts,
propose_trace: propose,
validate_trace: validate,
}
}
pub fn extend_using<P, F: Fn(&P)->K+Clone>(&self, logic: F) -> CollectionExtender<K, V, T, R, P, F> {
CollectionExtender {
phantom: std::marker::PhantomData,
indices: self.clone(),
key_selector: logic,
}
}
}
pub struct CollectionExtender<K, V, T, R, P, F>
where
K: ExchangeData,
V: ExchangeData,
T: Lattice+ExchangeData+Timestamp,
R: Monoid+Multiply<Output = R>+ExchangeData,
F: Fn(&P)->K+Clone,
{
phantom: std::marker::PhantomData<P>,
indices: CollectionIndex<K, V, T, R>,
key_selector: F,
}
impl<G, K, V, R, P, F> PrefixExtender<G, R> for CollectionExtender<K, V, G::Timestamp, R, P, F>
where
G: Scope,
K: ExchangeData+Hash+Default,
V: ExchangeData+Hash+Default,
P: ExchangeData,
G::Timestamp: Lattice+ExchangeData,
R: Monoid+Multiply<Output = R>+ExchangeData,
F: Fn(&P)->K+Clone+'static,
{
type Prefix = P;
type Extension = V;
fn count(&mut self, prefixes: &Collection<G, (P, usize, usize), R>, index: usize) -> Collection<G, (P, usize, usize), R> {
let counts = self.indices.count_trace.import(&prefixes.scope());
operators::count::count(prefixes, counts, self.key_selector.clone(), index)
}
fn propose(&mut self, prefixes: &Collection<G, P, R>) -> Collection<G, (P, V), R> {
let propose = self.indices.propose_trace.import(&prefixes.scope());
operators::propose::propose(prefixes, propose, self.key_selector.clone())
}
fn validate(&mut self, extensions: &Collection<G, (P, V), R>) -> Collection<G, (P, V), R> {
let validate = self.indices.validate_trace.import(&extensions.scope());
operators::validate::validate(extensions, validate, self.key_selector.clone())
}
}

View File

@ -0,0 +1,43 @@
use timely::dataflow::Scope;
use differential_dataflow::{ExchangeData, Collection, Hashable};
use differential_dataflow::difference::{Monoid, Multiply};
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::arrange::Arranged;
use differential_dataflow::trace::TraceReader;
/// Reports a number of extensions to a stream of prefixes.
///
/// This method takes as input a stream of `(prefix, count, index)` triples.
/// For each triple, it extracts a key using `key_selector`, and finds the
/// associated count in `arrangement`. If the found count is less than `count`,
/// the `count` and `index` fields are overwritten with their new values.
pub fn count<G, Tr, R, F, P>(
prefixes: &Collection<G, (P, usize, usize), R>,
arrangement: Arranged<G, Tr>,
key_selector: F,
index: usize,
) -> Collection<G, (P, usize, usize), R>
where
G: Scope,
G::Timestamp: Lattice,
Tr: TraceReader<Val=(), Time=G::Timestamp, R=isize>+Clone+'static,
Tr::Key: Ord+Hashable+Default,
R: Monoid+Multiply<Output = R>+ExchangeData,
F: Fn(&P)->Tr::Key+Clone+'static,
P: ExchangeData,
{
crate::operators::lookup_map(
prefixes,
arrangement,
move |p: &(P,usize,usize), k: &mut Tr::Key| { *k = key_selector(&p.0); },
move |(p,c,i), r, &(), s| {
let s = *s as usize;
if *c < s { ((p.clone(), *c, *i), r.clone()) }
else { ((p.clone(), s, index), r.clone()) }
},
Default::default(),
Default::default(),
Default::default(),
)
}

View File

@ -0,0 +1,261 @@
//! Dataflow operator for delta joins over partially ordered timestamps.
//!
//! Given multiple streams of updates `(data, time, diff)` that are each
//! defined over the same partially ordered `time`, we want to form the
//! full cross-join of all relations (we will *later* apply some filters
//! and instead equijoin on keys).
//!
//! The "correct" output is the outer join of these triples, where
//! 1. The `data` entries are just tuple'd up together,
//! 2. The `time` entries are subjected to the lattice `join` operator,
//! 3. The `diff` entries are multiplied.
//!
//! One way to produce the correct output is to form independent dataflow
//! fragments for each input stream, such that each intended output is then
//! produced by exactly one of these input streams.
//!
//! There are several incorrect ways one might do this, but here is one way
//! that I hope is not incorrect:
//!
//! Each input stream of updates is joined with each other input collection,
//! where each input update is matched against each other input update that
//! has a `time` that is less-than the input update's `time`, *UNDER A TOTAL
//! ORDER ON `time`*. The output are the `(data, time, diff)` entries that
//! follow the rules above, except that we additionally preserve the input's
//! initial `time` as well, for use in subsequent joins with the other input
//! collections.
//!
//! There are some caveats about ties, and we should treat each `time` for
//! each input as occuring at distinct times, one after the other (so that
//! ties are resolved by the index of the input). There is also the matter
//! of logical compaction, which should not be done in a way that prevents
//! the correct determination of the total order comparison.
use std::collections::HashMap;
use timely::dataflow::Scope;
use timely::dataflow::channels::pact::{Pipeline, Exchange};
use timely::dataflow::operators::Operator;
use timely::progress::Antichain;
use differential_dataflow::{ExchangeData, Collection, AsCollection, Hashable};
use differential_dataflow::difference::{Monoid, Semigroup};
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::arrange::Arranged;
use differential_dataflow::trace::{Cursor, TraceReader};
use differential_dataflow::consolidation::{consolidate, consolidate_updates};
/// A binary equijoin that responds to updates on only its first input.
///
/// This operator responds to inputs of the form
///
/// ```ignore
/// ((key, val1, time1), initial_time, diff1)
/// ```
///
/// where `initial_time` is less or equal to `time1`, and produces as output
///
/// ```ignore
/// ((output_func(key, val1, val2), lub(time1, time2)), initial_time, diff1 * diff2)
/// ```
///
/// for each `((key, val2), time2, diff2)` present in `arrangement`, where
/// `time2` is less than `initial_time` *UNDER THE TOTAL ORDER ON TIMES*.
/// This last constraint is important to ensure that we correctly produce
/// all pairs of output updates across multiple `half_join` operators.
///
/// Notice that the time is hoisted up into data. The expectation is that
/// once out of the "delta flow region", the updates will be `delay`d to the
/// times specified in the payloads.
pub fn half_join<G, V, Tr, FF, CF, DOut, S>(
stream: &Collection<G, (Tr::Key, V, G::Timestamp), Tr::R>,
arrangement: Arranged<G, Tr>,
frontier_func: FF,
comparison: CF,
mut output_func: S,
) -> Collection<G, (DOut, G::Timestamp), Tr::R>
where
G: Scope,
G::Timestamp: Lattice,
V: ExchangeData,
Tr: TraceReader<Time=G::Timestamp>+Clone+'static,
Tr::Key: Ord+Hashable+ExchangeData,
Tr::Val: Clone,
Tr::R: Monoid+ExchangeData,
FF: Fn(&G::Timestamp) -> G::Timestamp + 'static,
CF: Fn(&G::Timestamp, &G::Timestamp) -> bool + 'static,
DOut: Clone+'static,
Tr::R: std::ops::Mul<Tr::R, Output=Tr::R>,
S: FnMut(&Tr::Key, &V, &Tr::Val)->DOut+'static,
{
let output_func = move |k: &Tr::Key, v1: &V, v2: &Tr::Val, initial: &G::Timestamp, time: &G::Timestamp, diff1: &Tr::R, diff2: &Tr::R| {
let diff = diff1.clone() * diff2.clone();
let dout = (output_func(k, v1, v2), time.clone());
Some((dout, initial.clone(), diff))
};
half_join_internal_unsafe(stream, arrangement, frontier_func, comparison, |_timer, _count| false, output_func)
}
/// An unsafe variant of `half_join` where the `output_func` closure takes
/// additional arguments for `time` and `diff` as input and returns an iterator
/// over `(data, time, diff)` triplets. This allows for more flexibility, but
/// is more error-prone.
///
/// This operator responds to inputs of the form
///
/// ```ignore
/// ((key, val1, time1), initial_time, diff1)
/// ```
///
/// where `initial_time` is less or equal to `time1`, and produces as output
///
/// ```ignore
/// output_func(key, val1, val2, initial_time, lub(time1, time2), diff1, diff2)
/// ```
///
/// for each `((key, val2), time2, diff2)` present in `arrangement`, where
/// `time2` is less than `initial_time` *UNDER THE TOTAL ORDER ON TIMES*.
///
/// The `yield_function` allows the caller to indicate when the operator should
/// yield control, as a function of the elapsed time and the number of matched
/// records. Note this is not the number of *output* records, owing mainly to
/// the number of matched records being easiest to record with low overhead.
pub fn half_join_internal_unsafe<G, V, Tr, FF, CF, DOut, ROut, Y, I, S>(
stream: &Collection<G, (Tr::Key, V, G::Timestamp), Tr::R>,
mut arrangement: Arranged<G, Tr>,
frontier_func: FF,
comparison: CF,
yield_function: Y,
mut output_func: S,
) -> Collection<G, DOut, ROut>
where
G: Scope,
G::Timestamp: Lattice,
V: ExchangeData,
Tr: TraceReader<Time=G::Timestamp>+Clone+'static,
Tr::Key: Ord+Hashable+ExchangeData,
Tr::Val: Clone,
Tr::R: Monoid+ExchangeData,
FF: Fn(&G::Timestamp) -> G::Timestamp + 'static,
CF: Fn(&G::Timestamp, &G::Timestamp) -> bool + 'static,
DOut: Clone+'static,
ROut: Monoid,
Y: Fn(std::time::Instant, usize) -> bool + 'static,
I: IntoIterator<Item=(DOut, G::Timestamp, ROut)>,
S: FnMut(&Tr::Key, &V, &Tr::Val, &G::Timestamp, &G::Timestamp, &Tr::R, &Tr::R)-> I + 'static,
{
// No need to block physical merging for this operator.
arrangement.trace.set_physical_compaction(Antichain::new().borrow());
let mut arrangement_trace = Some(arrangement.trace);
let arrangement_stream = arrangement.stream;
let mut stash = HashMap::new();
let mut buffer = Vec::new();
let exchange = Exchange::new(move |update: &((Tr::Key, V, G::Timestamp),G::Timestamp,Tr::R)| (update.0).0.hashed().into());
// Stash for (time, diff) accumulation.
let mut output_buffer = Vec::new();
stream.inner.binary_frontier(&arrangement_stream, exchange, Pipeline, "HalfJoin", move |_,info| {
// Acquire an activator to reschedule the operator when it has unfinished work.
use timely::scheduling::Activator;
let activations = stream.scope().activations();
let activator = Activator::new(&info.address[..], activations);
move |input1, input2, output| {
// drain the first input, stashing requests.
input1.for_each(|capability, data| {
data.swap(&mut buffer);
stash.entry(capability.retain())
.or_insert(Vec::new())
.extend(buffer.drain(..))
});
// Drain input batches; although we do not observe them, we want access to the input
// to observe the frontier and to drive scheduling.
input2.for_each(|_, _| { });
// Local variables to track if and when we should exit early.
// The rough logic is that we fully process inputs and set their differences to zero,
// stopping at any point. We clean up all of the zeros in buffers that did any work,
// and reactivate at the end if the yield function still says so.
let mut yielded = false;
let timer = std::time::Instant::now();
let mut work = 0;
if let Some(ref mut trace) = arrangement_trace {
for (capability, proposals) in stash.iter_mut() {
// Avoid computation if we should already yield.
// TODO: Verify this is correct for TOTAL ORDER.
yielded = yielded || yield_function(timer, work);
if !yielded && !input2.frontier.less_equal(capability.time()) {
let mut session = output.session(capability);
// Sort requests by key for in-order cursor traversal.
consolidate_updates(proposals);
let (mut cursor, storage) = trace.cursor();
// Process proposals one at a time, stopping if we should yield.
for &mut ((ref key, ref val1, ref time), ref initial, ref mut diff1) in proposals.iter_mut() {
// Use TOTAL ORDER to allow the release of `time`.
yielded = yielded || yield_function(timer, work);
if !yielded && !input2.frontier.frontier().iter().any(|t| comparison(t, initial)) {
cursor.seek_key(&storage, &key);
if cursor.get_key(&storage) == Some(&key) {
while let Some(val2) = cursor.get_val(&storage) {
cursor.map_times(&storage, |t, d| {
if comparison(t, initial) {
output_buffer.push((t.join(time), d.clone()))
}
});
consolidate(&mut output_buffer);
work += output_buffer.len();
for (time, diff2) in output_buffer.drain(..) {
for dout in output_func(key, val1, val2, initial, &time, &diff1, &diff2) {
session.give(dout);
}
}
cursor.step_val(&storage);
}
cursor.rewind_vals(&storage);
}
*diff1 = Tr::R::zero();
}
}
proposals.retain(|ptd| !ptd.2.is_zero());
}
}
}
// If we yielded, re-activate the operator.
if yielded {
activator.activate();
}
// drop fully processed capabilities.
stash.retain(|_,proposals| !proposals.is_empty());
// The logical merging frontier depends on both input1 and stash.
let mut frontier = timely::progress::frontier::Antichain::new();
for time in input1.frontier().frontier().iter() {
frontier.insert(frontier_func(time));
}
for key in stash.keys() {
frontier.insert(frontier_func(key.time()));
}
arrangement_trace.as_mut().map(|trace| trace.set_logical_compaction(frontier.borrow()));
if input1.frontier().is_empty() && stash.is_empty() {
arrangement_trace = None;
}
}
}).as_collection()
}

View File

@ -0,0 +1,144 @@
use std::collections::HashMap;
use timely::PartialOrder;
use timely::dataflow::Scope;
use timely::dataflow::channels::pact::{Pipeline, Exchange};
use timely::dataflow::operators::Operator;
use timely::progress::Antichain;
use differential_dataflow::{ExchangeData, Collection, AsCollection, Hashable};
use differential_dataflow::difference::{Semigroup, Monoid};
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::arrange::Arranged;
use differential_dataflow::trace::{Cursor, TraceReader};
/// Proposes extensions to a stream of prefixes.
///
/// This method takes a stream of prefixes and for each determines a
/// key with `key_selector` and then proposes all pair af the prefix
/// and values associated with the key in `arrangement`.
pub fn lookup_map<G, D, R, Tr, F, DOut, ROut, S>(
prefixes: &Collection<G, D, R>,
mut arrangement: Arranged<G, Tr>,
key_selector: F,
mut output_func: S,
supplied_key0: Tr::Key,
supplied_key1: Tr::Key,
supplied_key2: Tr::Key,
) -> Collection<G, DOut, ROut>
where
G: Scope,
G::Timestamp: Lattice,
Tr: TraceReader<Time=G::Timestamp>+Clone+'static,
Tr::Key: Ord+Hashable,
Tr::Val: Clone,
Tr::R: Monoid+ExchangeData,
F: FnMut(&D, &mut Tr::Key)+Clone+'static,
D: ExchangeData,
R: ExchangeData+Monoid,
DOut: Clone+'static,
ROut: Monoid,
S: FnMut(&D, &R, &Tr::Val, &Tr::R)->(DOut, ROut)+'static,
{
// No need to block physical merging for this operator.
arrangement.trace.set_physical_compaction(Antichain::new().borrow());
let mut propose_trace = Some(arrangement.trace);
let propose_stream = arrangement.stream;
let mut stash = HashMap::new();
let mut logic1 = key_selector.clone();
let mut logic2 = key_selector.clone();
let mut buffer = Vec::new();
let mut key: Tr::Key = supplied_key0;
let exchange = Exchange::new(move |update: &(D,G::Timestamp,R)| {
logic1(&update.0, &mut key);
key.hashed().into()
});
let mut key1: Tr::Key = supplied_key1;
let mut key2: Tr::Key = supplied_key2;
prefixes.inner.binary_frontier(&propose_stream, exchange, Pipeline, "LookupMap", move |_,_| move |input1, input2, output| {
// drain the first input, stashing requests.
input1.for_each(|capability, data| {
data.swap(&mut buffer);
stash.entry(capability.retain())
.or_insert(Vec::new())
.extend(buffer.drain(..))
});
// Drain input batches; although we do not observe them, we want access to the input
// to observe the frontier and to drive scheduling.
input2.for_each(|_, _| { });
if let Some(ref mut trace) = propose_trace {
for (capability, prefixes) in stash.iter_mut() {
// defer requests at incomplete times.
// NOTE: not all updates may be at complete times, but if this test fails then none of them are.
if !input2.frontier.less_equal(capability.time()) {
let mut session = output.session(capability);
// sort requests for in-order cursor traversal. could consolidate?
prefixes.sort_by(|x,y| {
logic2(&x.0, &mut key1);
logic2(&y.0, &mut key2);
key1.cmp(&key2)
});
let (mut cursor, storage) = trace.cursor();
for &mut (ref prefix, ref time, ref mut diff) in prefixes.iter_mut() {
if !input2.frontier.less_equal(time) {
logic2(prefix, &mut key1);
cursor.seek_key(&storage, &key1);
if cursor.get_key(&storage) == Some(&key1) {
while let Some(value) = cursor.get_val(&storage) {
let mut count = Tr::R::zero();
cursor.map_times(&storage, |t, d| {
if t.less_equal(time) { count.plus_equals(d); }
});
if !count.is_zero() {
let (dout, rout) = output_func(prefix, diff, value, &count);
if !rout.is_zero() {
session.give((dout, time.clone(), rout));
}
}
cursor.step_val(&storage);
}
cursor.rewind_vals(&storage);
}
*diff = R::zero();
}
}
prefixes.retain(|ptd| !ptd.2.is_zero());
}
}
}
// drop fully processed capabilities.
stash.retain(|_,prefixes| !prefixes.is_empty());
// The logical merging frontier depends on both input1 and stash.
let mut frontier = timely::progress::frontier::Antichain::new();
for time in input1.frontier().frontier().to_vec() {
frontier.insert(time);
}
for key in stash.keys() {
frontier.insert(key.time().clone());
}
propose_trace.as_mut().map(|trace| trace.set_logical_compaction(frontier.borrow()));
if input1.frontier().is_empty() && stash.is_empty() {
propose_trace = None;
}
}).as_collection()
}

View File

@ -0,0 +1,12 @@
pub mod half_join;
pub mod lookup_map;
pub mod count;
pub mod propose;
pub mod validate;
pub use self::half_join::half_join;
pub use self::lookup_map::lookup_map;
pub use self::count::count;
pub use self::propose::{propose, propose_distinct};
pub use self::validate::validate;

View File

@ -0,0 +1,72 @@
use timely::dataflow::Scope;
use differential_dataflow::{ExchangeData, Collection, Hashable};
use differential_dataflow::difference::{Monoid, Multiply};
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::arrange::Arranged;
use differential_dataflow::trace::TraceReader;
/// Proposes extensions to a prefix stream.
///
/// This method takes a collection `prefixes` and an arrangement `arrangement` and for each
/// update in the collection joins with the accumulated arranged records at a time less or
/// equal to that of the update. Note that this is not a join by itself, but can be used to
/// create a join if the `prefixes` collection is also arranged and responds to changes that
/// `arrangement` undergoes. More complicated patterns are also appropriate, as in the case
/// of delta queries.
pub fn propose<G, Tr, F, P>(
prefixes: &Collection<G, P, Tr::R>,
arrangement: Arranged<G, Tr>,
key_selector: F,
) -> Collection<G, (P, Tr::Val), Tr::R>
where
G: Scope,
G::Timestamp: Lattice,
Tr: TraceReader<Time=G::Timestamp>+Clone+'static,
Tr::Key: Ord+Hashable+Default,
Tr::Val: Clone,
Tr::R: Monoid+Multiply<Output = Tr::R>+ExchangeData,
F: Fn(&P)->Tr::Key+Clone+'static,
P: ExchangeData,
{
crate::operators::lookup_map(
prefixes,
arrangement,
move |p: &P, k: &mut Tr::Key| { *k = key_selector(p); },
|prefix, diff, value, sum| ((prefix.clone(), value.clone()), diff.clone().multiply(sum)),
Default::default(),
Default::default(),
Default::default(),
)
}
/// Proposes distinct extensions to a prefix stream.
///
/// Unlike `propose`, this method does not scale the multiplicity of matched
/// prefixes by the number of matches in `arrangement`. This can be useful to
/// avoid the need to prepare an arrangement of distinct extensions.
pub fn propose_distinct<G, Tr, F, P>(
prefixes: &Collection<G, P, Tr::R>,
arrangement: Arranged<G, Tr>,
key_selector: F,
) -> Collection<G, (P, Tr::Val), Tr::R>
where
G: Scope,
G::Timestamp: Lattice,
Tr: TraceReader<Time=G::Timestamp>+Clone+'static,
Tr::Key: Ord+Hashable+Default,
Tr::Val: Clone,
Tr::R: Monoid+Multiply<Output = Tr::R>+ExchangeData,
F: Fn(&P)->Tr::Key+Clone+'static,
P: ExchangeData,
{
crate::operators::lookup_map(
prefixes,
arrangement,
move |p: &P, k: &mut Tr::Key| { *k = key_selector(p); },
|prefix, diff, value, _sum| ((prefix.clone(), value.clone()), diff.clone()),
Default::default(),
Default::default(),
Default::default(),
)
}

View File

@ -0,0 +1,40 @@
use std::hash::Hash;
use timely::dataflow::Scope;
use differential_dataflow::{ExchangeData, Collection};
use differential_dataflow::difference::{Monoid, Multiply};
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::arrange::Arranged;
use differential_dataflow::trace::TraceReader;
/// Proposes extensions to a stream of prefixes.
///
/// This method takes a stream of prefixes and for each determines a
/// key with `key_selector` and then proposes all pair af the prefix
/// and values associated with the key in `arrangement`.
pub fn validate<G, K, V, Tr, F, P>(
extensions: &Collection<G, (P, V), Tr::R>,
arrangement: Arranged<G, Tr>,
key_selector: F,
) -> Collection<G, (P, V), Tr::R>
where
G: Scope,
G::Timestamp: Lattice,
Tr: TraceReader<Key=(K,V), Val=(), Time=G::Timestamp>+Clone+'static,
K: Ord+Hash+Clone+Default,
V: ExchangeData+Hash+Default,
Tr::R: Monoid+Multiply<Output = Tr::R>+ExchangeData,
F: Fn(&P)->K+Clone+'static,
P: ExchangeData,
{
crate::operators::lookup_map(
extensions,
arrangement,
move |(pre,val),key| { *key = (key_selector(pre), val.clone()); },
|(pre,val),r,&(),_| ((pre.clone(), val.clone()), r.clone()),
Default::default(),
Default::default(),
Default::default(),
)
}

View File

@ -0,0 +1,9 @@
[package]
name = "doop"
version = "0.1.0"
authors = ["Frank McSherry <fmcsherry@me.com>"]
[dependencies]
indexmap = "1.0.1"
timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" }
differential-dataflow = { path = "../" }

View File

@ -0,0 +1,561 @@
// Main schema
.type Modifier
.type Type
.type PrimitiveType = Type
.type ReferenceType = Type
.type ArrayType = ReferenceType
.type ClassType = ReferenceType
.type InterfaceType = ReferenceType
.decl isType(?t:Type)
.decl isReferenceType(?t:ReferenceType)
.decl isArrayType(?t:ArrayType)
.decl isClassType(?t:ClassType)
.decl isInterfaceType(?t:InterfaceType)
.decl DirectSuperclass(?class:ClassType, ?superclass:ClassType)
.input DirectSuperclass(IO="file", filename="DirectSuperclass.facts", delimiter="\t")
.decl DirectSuperinterface(?ref:ReferenceType, ?interface:InterfaceType)
.input DirectSuperinterface(IO="file", filename="DirectSuperinterface.facts", delimiter="\t")
.decl ApplicationClass(?ref:ReferenceType)
.decl MainClass(?class:ClassType)
.input MainClass(IO="file", filename="MainClass.facts", delimiter="\t")
.type Field
.decl Field_DeclaringType(?field:Field, ?declaringClass:ReferenceType)
.type MethodDescriptor
.type Method
.decl Method_DeclaringType(?method:Method, ?declaringType:ReferenceType)
.decl Method_ReturnType(?method:Method, ?returnType:Type)
.decl Method_SimpleName(?method:Method, ?simpleName:symbol)
.decl Method_Params(?method:Method, ?params:symbol)
.decl Method_Descriptor(?method:Method, ?descriptor:MethodDescriptor)
.decl Method_Modifier(?mod:Modifier, ?method:Method)
.input Method_Modifier(IO="file", filename="Method-Modifier.facts", delimiter="\t")
.decl FormalParam(?index:number, ?method:Method, ?var:Var)
.input FormalParam(IO="file", filename="FormalParam.facts", delimiter="\t")
.decl ThisVar(?method:Method, ?var:Var)
.type Var
.decl Var_Type(?var:Var, ?type:Type)
.input Var_Type(IO="file", filename="Var-Type.facts", delimiter="\t")
.decl Var_DeclaringMethod(?var:Var, ?method:Method)
.type HeapAllocation
.decl HeapAllocation_Type(?heap:HeapAllocation, ?type:Type)
.decl MainMethodArgArray(?heap:HeapAllocation)
.decl MainMethodArgArrayContent(?heap:HeapAllocation)
.type Instruction
.decl Instruction_Method(?insn:Instruction, ?inMethod:Method)
.type FieldInstruction = Instruction
.type LoadInstanceField_Insn = FieldInstruction
.type StoreInstanceField_Insn = FieldInstruction
.type LoadStaticField_Insn = FieldInstruction
.type StoreStaticField_Insn = FieldInstruction
.type ArrayInstruction = Instruction
.type LoadArrayIndex_Insn = ArrayInstruction
.type StoreArrayIndex_Insn = ArrayInstruction
.type AssignInstruction = Instruction
.type AssignLocal_Insn = AssignInstruction
.type AssignCast_Insn = AssignInstruction
.type AssignHeapAllocation_Insn = AssignInstruction
.type ReturnInstruction = Instruction
.type ReturnNonvoid_Insn = ReturnInstruction
.type MethodInvocation = Instruction
.type VirtualMethodInvocation_Insn = MethodInvocation
.type StaticMethodInvocation_Insn = MethodInvocation
.decl isVirtualMethodInvocation_Insn(?insn:VirtualMethodInvocation_Insn)
.decl isStaticMethodInvocation_Insn(?insn:StaticMethodInvocation_Insn)
.decl FieldInstruction_Signature(?insn:FieldInstruction, ?sign:Field)
.decl LoadInstanceField_Base(?insn:LoadInstanceField_Insn, ?var:Var)
.decl LoadInstanceField_To(?insn:LoadInstanceField_Insn, ?var:Var)
.decl StoreInstanceField_From(?insn:StoreInstanceField_Insn, ?var:Var)
.decl StoreInstanceField_Base(?insn:StoreInstanceField_Insn, ?var:Var)
.decl LoadStaticField_To(?insn:LoadStaticField_Insn, ?var:Var)
.decl StoreStaticField_From(?insn:StoreStaticField_Insn, ?var:Var)
.decl ComponentType(?arrayType:ArrayType, ?componentType:Type)
.input ComponentType(IO="file", filename="ComponentType.facts", delimiter="\t")
.decl LoadArrayIndex_Base(?insn:LoadArrayIndex_Insn, ?var:Var)
.decl LoadArrayIndex_To(?insn:LoadArrayIndex_Insn, var:Var)
.decl StoreArrayIndex_From(?insn:StoreArrayIndex_Insn, ?var:Var)
.decl StoreArrayIndex_Base(?insn:StoreArrayIndex_Insn, ?var:Var)
.decl AssignInstruction_To(?insn:AssignInstruction, ?to:Var)
.decl AssignCast_From(?insn:AssignCast_Insn, ?from:Var)
.decl AssignCast_Type(?insn:AssignCast_Insn, ?type:Type)
.decl AssignLocal_From(?insn:AssignLocal_Insn, ?from:Var)
.decl AssignHeapAllocation_Heap(?insn:AssignHeapAllocation_Insn, ?heap:HeapAllocation)
.decl ReturnNonvoid_Var(?return:ReturnNonvoid_Insn, ?var:Var)
.decl MethodInvocation_Method(?invocation:MethodInvocation, ?signature:Method)
.decl AssignReturnValue(?invocation:MethodInvocation, ?to:Var)
.input AssignReturnValue(IO="file", filename="AssignReturnValue.facts", delimiter="\t")
.decl ActualParam(?index:number, ?invocation:MethodInvocation, ?var:Var)
.input ActualParam(IO="file", filename="ActualParam.facts", delimiter="\t")
.decl VirtualMethodInvocation_Base(?invocation:VirtualMethodInvocation_Insn, ?base:Var)
.decl VirtualMethodInvocation_SimpleName(?invocation:MethodInvocation, ?simplename:symbol)
.decl VirtualMethodInvocation_Descriptor(?invocation:MethodInvocation, ?descriptor:MethodDescriptor)
.decl SpecialMethodInvocation_Base(?invocation:MethodInvocation, ?base:Var)
.decl MethodInvocation_Base(?invocation:MethodInvocation, ?base:Var)
// Fat schema
.decl LoadInstanceField(?base:Var, ?sig:Field, ?to:Var, ?inmethod:Method)
.decl StoreInstanceField(?from:Var, ?base:Var, ?signature:Field, ?inmethod:Method)
.decl LoadStaticField(?sig:Field, ?to:Var, ?inmethod:Method)
.decl StoreStaticField(?from:Var, ?signature:Field, ?inmethod:Method)
.decl LoadArrayIndex(?base:Var, ?to:Var, ?inmethod:Method)
.decl StoreArrayIndex(?from:Var, ?base:Var, ?inmethod:Method)
.decl AssignCast(?type:Type, ?from:Var, ?to:Var, ?inmethod:Method)
.decl AssignLocal(?from:Var, ?to:Var, ?inmethod:Method)
.decl AssignHeapAllocation(?heap:HeapAllocation, ?to:Var, ?inmethod:Method)
.decl ReturnVar(?var:Var, ?method:Method)
.decl StaticMethodInvocation(?invocation:MethodInvocation, ?signature:Method, ?inmethod:Method)
// imports
.decl _ClassType(?class:symbol)
.input _ClassType(IO="file", filename="ClassType.facts", delimiter="\t")
isType(?class),
isReferenceType(?class),
isClassType(?class) :-
_ClassType(?class).
.decl _ArrayType(?arrayType:symbol)
.input _ArrayType(IO="file", filename="ArrayType.facts", delimiter="\t")
isType(?arrayType),
isReferenceType(?arrayType),
isArrayType(?arrayType) :-
_ArrayType(?arrayType).
.decl _InterfaceType(?interface:symbol)
.input _InterfaceType(IO="file", filename="InterfaceType.facts", delimiter="\t")
isType(?interface),
isReferenceType(?interface),
isInterfaceType(?interface) :-
_InterfaceType(?interface).
.decl _Var_DeclaringMethod(?var:symbol, ?method:symbol)
.input _Var_DeclaringMethod(IO="file", filename="Var-DeclaringMethod.facts", delimiter="\t")
Var_DeclaringMethod(?var, ?method) :-
_Var_DeclaringMethod(?var, ?method).
.decl _ApplicationClass(?type:symbol)
.input _ApplicationClass(IO="file", filename="ApplicationClass.facts", delimiter="\t")
isType(?type),
isReferenceType(?type),
ApplicationClass(?type) :-
_ApplicationClass(?type).
.decl _ThisVar(?method:symbol, ?var:symbol)
.input _ThisVar(IO="file", filename="ThisVar.facts", delimiter="\t")
ThisVar(?method, ?var) :-
_ThisVar(?method, ?var).
.decl _NormalHeap(?id:symbol, ?type:symbol)
.input _NormalHeap(IO="file", filename="NormalHeap.facts", delimiter="\t")
isType(?type),
HeapAllocation_Type(?id, ?type) :-
_NormalHeap(?id, ?type).
.decl _StringConstant(?id:symbol)
.input _StringConstant(IO="file", filename="StringConstant.facts", delimiter="\t")
HeapAllocation_Type(?id, "java.lang.String") :-
_StringConstant(?id).
.decl _AssignHeapAllocation(?instruction:symbol, ?index:number, ?heap:symbol, ?to:symbol, ?inmethod:symbol, ?linenumber:number)
.input _AssignHeapAllocation(IO="file", filename="AssignHeapAllocation.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
AssignInstruction_To(?instruction, ?to),
AssignHeapAllocation_Heap(?instruction, ?heap) :-
_AssignHeapAllocation(?instruction, ?index, ?heap, ?to, ?method, ?linenumber).
.decl _AssignLocal(?instruction:symbol, ?index:number, ?from:symbol, ?to:symbol, ?inmethod:symbol)
.input _AssignLocal(IO="file", filename="AssignLocal.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
AssignLocal_From(?instruction, ?from),
AssignInstruction_To(?instruction, ?to) :-
_AssignLocal(?instruction, ?index, ?from, ?to, ?method).
.decl _AssignCast(?instruction:symbol, ?index:number, ?from:symbol, ?to:symbol, ?type:symbol, ?inmethod:symbol)
.input _AssignCast(IO="file", filename="AssignCast.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
AssignCast_Type(?instruction, ?type),
AssignCast_From(?instruction, ?from),
AssignInstruction_To(?instruction, ?to) :-
_AssignCast(?instruction, ?index, ?from, ?to, ?type, ?method).
.decl _Field(?signature:symbol, ?declaringClass:symbol, ?simplename:symbol, ?type:symbol)
.input _Field(IO="file", filename="Field.facts", delimiter="\t")
Field_DeclaringType(?signature, ?declaringType) :-
_Field(?signature, ?declaringType, _, _).
MethodInvocation_Base(?invocation, ?base) :-
VirtualMethodInvocation_Base(?invocation, ?base) ;
SpecialMethodInvocation_Base(?invocation, ?base).
.decl _StaticMethodInvocation(?instruction:symbol, ?index:number, ?signature:symbol, ?method:symbol)
.input _StaticMethodInvocation(IO="file", filename="StaticMethodInvocation.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
isStaticMethodInvocation_Insn(?instruction),
MethodInvocation_Method(?instruction, ?signature) :-
_StaticMethodInvocation(?instruction, ?index, ?signature, ?method).
.decl _SpecialMethodInvocation(?instruction:symbol, ?index:number, ?signature:symbol, ?base:symbol, ?method:symbol)
.input _SpecialMethodInvocation(IO="file", filename="SpecialMethodInvocation.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
SpecialMethodInvocation_Base(?instruction, ?base),
MethodInvocation_Method(?instruction, ?signature) :-
_SpecialMethodInvocation(?instruction, ?index, ?signature, ?base, ?method).
.decl _VirtualMethodInvocation(?instruction:symbol, ?index:number, ?signature:symbol, ?base:symbol, ?method:symbol)
.input _VirtualMethodInvocation(IO="file", filename="VirtualMethodInvocation.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
isVirtualMethodInvocation_Insn(?instruction),
VirtualMethodInvocation_Base(?instruction, ?base),
MethodInvocation_Method(?instruction, ?signature) :-
_VirtualMethodInvocation(?instruction, ?index, ?signature, ?base, ?method).
.decl _Method(?method:symbol, ?simplename:symbol, ?descriptor:symbol, ?declaringType:symbol, ?returnType:symbol, ?jvmDescriptor:symbol, ?arity:number)
.input _Method(IO="file", filename="Method.facts", delimiter="\t")
Method_SimpleName(?method, ?simplename),
Method_Params(?method, ?params),
Method_DeclaringType(?method, ?declaringType),
Method_ReturnType(?method, ?returnType) :-
_Method(?method, ?simplename, ?params, ?declaringType, ?returnType, ?jvmDescriptor, ?arity).
Method_Descriptor(?method, ?descriptor) :-
Method_ReturnType(?method, ?returnType),
Method_Params(?method, ?params),
?descriptor = cat(?returnType, ?params).
.decl _StoreInstanceField(?instruction:symbol, ?index:number, ?from:symbol, ?base:symbol, ?signature:symbol, ?method:symbol)
.input _StoreInstanceField(IO="file", filename="StoreInstanceField.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
FieldInstruction_Signature(?instruction, ?signature),
StoreInstanceField_Base(?instruction, ?base),
StoreInstanceField_From(?instruction, ?from) :-
_StoreInstanceField(?instruction, ?index, ?from, ?base, ?signature, ?method).
.decl _LoadInstanceField(?instruction:symbol, ?index:number, ?to:symbol, ?base:symbol, ?signature:symbol, ?method:symbol)
.input _LoadInstanceField(IO="file", filename="LoadInstanceField.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
FieldInstruction_Signature(?instruction, ?signature),
LoadInstanceField_Base(?instruction, ?base),
LoadInstanceField_To(?instruction, ?to) :-
_LoadInstanceField(?instruction, ?index, ?to, ?base, ?signature, ?method).
.decl _StoreStaticField(?instruction:symbol, ?index:number, ?from:symbol, ?signature:symbol, ?method:symbol)
.input _StoreStaticField(IO="file", filename="StoreStaticField.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
FieldInstruction_Signature(?instruction, ?signature),
StoreStaticField_From(?instruction, ?from) :-
_StoreStaticField(?instruction, ?index, ?from, ?signature, ?method).
.decl _LoadStaticField(?instruction:symbol, ?index:number, ?to:symbol, ?signature:symbol, ?method:symbol)
.input _LoadStaticField(IO="file", filename="LoadStaticField.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
FieldInstruction_Signature(?instruction, ?signature),
LoadStaticField_To(?instruction, ?to) :-
_LoadStaticField(?instruction, ?index, ?to, ?signature, ?method).
.decl _StoreArrayIndex(?instruction:symbol, ?index:number, ?from:symbol, ?base:symbol, ?method:symbol)
.input _StoreArrayIndex(IO="file", filename="StoreArrayIndex.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
StoreArrayIndex_Base(?instruction, ?base),
StoreArrayIndex_From(?instruction, ?from) :-
_StoreArrayIndex(?instruction, ?index, ?from, ?base, ?method).
.decl _LoadArrayIndex(?instruction:symbol, ?index:number, ?to:symbol, ?base:symbol, ?method:symbol)
.input _LoadArrayIndex(IO="file", filename="LoadArrayIndex.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
LoadArrayIndex_Base(?instruction, ?base),
LoadArrayIndex_To(?instruction, ?to) :-
_LoadArrayIndex(?instruction, ?index, ?to, ?base, ?method).
.decl _Return(?instruction:symbol, ?index:number, ?var:symbol, ?method:symbol)
.input _Return(IO="file", filename="Return.facts", delimiter="\t")
Instruction_Method(?instruction, ?method),
ReturnNonvoid_Var(?instruction, ?var) :-
_Return(?instruction, ?index, ?var, ?method).
// fat schema population
LoadInstanceField(?base, ?sig, ?to, ?inmethod) :-
Instruction_Method(?insn, ?inmethod),
LoadInstanceField_Base(?insn, ?base),
FieldInstruction_Signature(?insn, ?sig),
LoadInstanceField_To(?insn, ?to).
StoreInstanceField(?from, ?base, ?sig, ?inmethod) :-
Instruction_Method(?insn, ?inmethod),
StoreInstanceField_From(?insn, ?from),
StoreInstanceField_Base(?insn, ?base),
FieldInstruction_Signature(?insn, ?sig).
LoadStaticField(?sig, ?to, ?inmethod) :-
Instruction_Method(?insn, ?inmethod),
FieldInstruction_Signature(?insn, ?sig),
LoadStaticField_To(?insn, ?to).
StoreStaticField(?from, ?sig, ?inmethod) :-
Instruction_Method(?insn, ?inmethod),
StoreStaticField_From(?insn, ?from),
FieldInstruction_Signature(?insn, ?sig).
LoadArrayIndex(?base, ?to, ?inmethod) :-
Instruction_Method(?insn, ?inmethod),
LoadArrayIndex_Base(?insn, ?base),
LoadArrayIndex_To(?insn, ?to).
StoreArrayIndex(?from, ?base, ?inmethod) :-
Instruction_Method(?insn, ?inmethod),
StoreArrayIndex_From(?insn, ?from),
StoreArrayIndex_Base(?insn, ?base).
AssignCast(?type, ?from, ?to, ?inmethod) :-
Instruction_Method(?insn, ?inmethod),
AssignCast_From(?insn, ?from),
AssignInstruction_To(?insn, ?to),
AssignCast_Type(?insn, ?type).
AssignLocal(?from, ?to, ?inmethod) :-
AssignInstruction_To(?insn, ?to),
Instruction_Method(?insn, ?inmethod),
AssignLocal_From(?insn, ?from).
AssignHeapAllocation(?heap, ?to, ?inmethod) :-
Instruction_Method(?insn, ?inmethod),
AssignHeapAllocation_Heap(?insn, ?heap),
AssignInstruction_To(?insn, ?to).
ReturnVar(?var, ?method) :-
Instruction_Method(?insn, ?method),
ReturnNonvoid_Var(?insn, ?var).
StaticMethodInvocation(?invocation, ?signature, ?inmethod) :-
isStaticMethodInvocation_Insn(?invocation),
Instruction_Method(?invocation, ?inmethod),
MethodInvocation_Method(?invocation, ?signature).
HeapAllocation_Type(?heap, ?type),
MainMethodArgArray(?heap) :-
?heap = "<<main method array>>",
?type = "java.lang.String[]".
HeapAllocation_Type(?heap, ?type),
MainMethodArgArrayContent(?heap) :-
?heap = "<<main method array content>>",
?type = "java.lang.String".
VirtualMethodInvocation_SimpleName(?invocation, ?simplename),
VirtualMethodInvocation_Descriptor(?invocation, ?descriptor) :-
isVirtualMethodInvocation_Insn(?invocation),
MethodInvocation_Method(?invocation, ?signature),
Method_SimpleName(?signature, ?simplename),
Method_Descriptor(?signature, ?descriptor).
// Basic (type-based) analysis
.comp Basic {
.decl MethodLookup(?simplename:symbol, ?descriptor:MethodDescriptor, ?type:Type, ?method:Method)
.decl MethodImplemented(?simplename:symbol, ?descriptor:MethodDescriptor, ?type:Type, ?method:Method)
.decl DirectSubclass(?a:Type, ?c:Type)
.decl Subclass(?c:Type, ?a:Type)
.decl Superclass(?c:Type, ?a:Type)
.decl Superinterface(?k:Type, ?c:Type)
.decl SubtypeOf(?subtype:Type, ?type:Type)
.decl SupertypeOf(?supertype:Type, ?type:Type)
.decl SubtypeOfDifferent(?subtype:Type, ?type:Type)
.decl MainMethodDeclaration(?method:Method)
MethodLookup(?simplename, ?descriptor, ?type, ?method) :-
MethodImplemented(?simplename, ?descriptor, ?type, ?method).
MethodLookup(?simplename, ?descriptor, ?type, ?method) :-
(DirectSuperclass(?type, ?supertype) ;
DirectSuperinterface(?type, ?supertype)),
MethodLookup(?simplename, ?descriptor, ?supertype, ?method),
! MethodImplemented(?simplename, ?descriptor, ?type, _).
MethodImplemented(?simplename, ?descriptor, ?type, ?method) :-
Method_SimpleName(?method, ?simplename),
Method_Descriptor(?method, ?descriptor),
Method_DeclaringType(?method, ?type),
! Method_Modifier("abstract", ?method).
DirectSubclass(?a, ?c) :-
DirectSuperclass(?a, ?c).
Subclass(?c, ?a) :-
DirectSubclass(?a, ?c).
Subclass(?c, ?a) :-
Subclass(?b, ?a),
DirectSubclass(?b, ?c).
Superclass(?c, ?a) :-
Subclass(?a, ?c).
Superinterface(?k, ?c) :-
DirectSuperinterface(?c, ?k).
Superinterface(?k, ?c) :-
DirectSuperinterface(?c, ?j),
Superinterface(?k, ?j).
Superinterface(?k, ?c) :-
DirectSuperclass(?c, ?super),
Superinterface(?k, ?super).
SupertypeOf(?s, ?t) :-
SubtypeOf(?t, ?s).
SubtypeOf(?s, ?s) :-
isClassType(?s).
SubtypeOf(?s, ?t) :-
Subclass(?t, ?s).
SubtypeOf(?s, ?t) :-
isClassType(?s),
Superinterface(?t, ?s).
SubtypeOf(?s, ?t) :-
isInterfaceType(?s),
isType(?t),
?t = "java.lang.Object".
SubtypeOf(?s, ?s) :-
isInterfaceType(?s).
SubtypeOf(?s, ?t) :-
isInterfaceType(?s),
Superinterface(?t, ?s).
SubtypeOf(?s, ?t) :-
isArrayType(?s),
isType(?t),
?t = "java.lang.Object".
SubtypeOf(?s, ?t) :-
ComponentType(?s, ?sc),
ComponentType(?t, ?tc),
isReferenceType(?sc),
isReferenceType(?tc),
SubtypeOf(?sc, ?tc).
SubtypeOf(?s, ?t) :-
isArrayType(?s),
isInterfaceType(?t),
isType(?t),
?t = "java.lang.Cloneable".
SubtypeOf(?s, ?t) :-
isArrayType(?s),
isInterfaceType(?t),
isType(?t),
?t = "java.io.Serializable".
SubtypeOf(?t, ?t) :-
isType(?t).
SubtypeOfDifferent(?s, ?t) :-
SubtypeOf(?s, ?t),
?s != ?t.
MainMethodDeclaration(?method) :-
MainClass(?type),
Method_DeclaringType(?method, ?type),
?method != "<java.util.prefs.Base64: void main(java.lang.String[])>",
?method != "<sun.java2d.loops.GraphicsPrimitiveMgr: void main(java.lang.String[])>",
?method != "<sun.security.provider.PolicyParser: void main(java.lang.String[])>",
Method_SimpleName(?method, "main"),
Method_Descriptor(?method, "void(java.lang.String[])"),
Method_Modifier("public", ?method),
Method_Modifier("static", ?method).
}
.init basic = Basic
// class initialization
.decl ClassInitializer(?type:Type, ?method:Method)
.decl InitializedClass(?classOrInterface:Type)
ClassInitializer(?type, ?method) :-
basic.MethodImplemented("<clinit>", "void()", ?type, ?method).
InitializedClass(?superclass) :-
InitializedClass(?class),
DirectSuperclass(?class, ?superclass).
InitializedClass(?superinterface) :-
InitializedClass(?classOrInterface),
DirectSuperinterface(?classOrInterface, ?superinterface).
InitializedClass(?class) :-
basic.MainMethodDeclaration(?method),
Method_DeclaringType(?method, ?class).
InitializedClass(?class) :-
Reachable(?inmethod),
AssignHeapAllocation(?heap, _, ?inmethod),
HeapAllocation_Type(?heap, ?class).
InitializedClass(?class) :-
Reachable(?inmethod),
Instruction_Method(?invocation, ?inmethod),
isStaticMethodInvocation_Insn(?invocation),
MethodInvocation_Method(?invocation, ?signature),
Method_DeclaringType(?signature, ?class).
InitializedClass(?classOrInterface) :-
Reachable(?inmethod),
StoreStaticField(_, ?signature, ?inmethod),
Field_DeclaringType(?signature, ?classOrInterface).
InitializedClass(?classOrInterface) :-
Reachable(?inmethod),
LoadStaticField(?signature, _, ?inmethod),
Field_DeclaringType(?signature, ?classOrInterface).
Reachable(?clinit) :-
InitializedClass(?class),
ClassInitializer(?class, ?clinit).
// Main (value-based) analysis
.decl Assign(?to:Var, ?from:Var)
.output Assign
.decl VarPointsTo(?heap:HeapAllocation, ?var:Var)
.output VarPointsTo
.decl InstanceFieldPointsTo(?heap:HeapAllocation , ?fld:Field, ?baseheap:HeapAllocation)
.output InstanceFieldPointsTo
.decl StaticFieldPointsTo(?heap:HeapAllocation, ?fld:Field)
.output StaticFieldPointsTo
.decl CallGraphEdge(?invocation:MethodInvocation, ?meth:Method)
.output CallGraphEdge
.decl ArrayIndexPointsTo(?baseheap:HeapAllocation, ?heap:HeapAllocation)
.output ArrayIndexPointsTo
.decl Reachable(?method:Method)
.output Reachable
Assign(?actual, ?formal) :-
CallGraphEdge(?invocation, ?method),
FormalParam(?index, ?method, ?formal),
ActualParam(?index, ?invocation, ?actual).
Assign(?return, ?local) :-
CallGraphEdge(?invocation, ?method),
ReturnVar(?return, ?method),
AssignReturnValue(?invocation, ?local).
VarPointsTo(?heap, ?var) :-
AssignHeapAllocation(?heap, ?var, ?inMethod),
Reachable(?inMethod).
VarPointsTo(?heap, ?to) :-
Assign(?from, ?to),
VarPointsTo(?heap, ?from).
VarPointsTo(?heap, ?to) :-
Reachable(?inmethod),
AssignLocal(?from, ?to, ?inmethod),
VarPointsTo(?heap, ?from).
.plan 1:(3,2,1)
VarPointsTo(?heap, ?to) :-
Reachable(?method),
AssignCast(?type, ?from, ?to, ?method),
basic.SupertypeOf(?type, ?heaptype),
HeapAllocation_Type(?heap, ?heaptype),
VarPointsTo(?heap, ?from).
.plan 1:(5,2,1,4,3)
ArrayIndexPointsTo(?baseheap, ?heap) :-
Reachable(?inmethod),
StoreArrayIndex(?from, ?base, ?inmethod),
VarPointsTo(?baseheap, ?base),
VarPointsTo(?heap, ?from),
HeapAllocation_Type(?heap, ?heaptype),
HeapAllocation_Type(?baseheap, ?baseheaptype),
ComponentType(?baseheaptype, ?componenttype),
basic.SupertypeOf(?componenttype, ?heaptype).
VarPointsTo(?heap, ?to) :-
Reachable(?inmethod),
LoadArrayIndex(?base, ?to, ?inmethod),
VarPointsTo(?baseheap, ?base),
ArrayIndexPointsTo(?baseheap, ?heap),
Var_Type(?to, ?type),
HeapAllocation_Type(?baseheap, ?baseheaptype),
ComponentType(?baseheaptype, ?basecomponenttype),
basic.SupertypeOf(?type, ?basecomponenttype).
.plan 1:(3,2,1,4,5,6,7,8), 2:(4,3,2,1,5,6,7,8)
VarPointsTo(?heap, ?to) :-
Reachable(?inmethod),
LoadInstanceField(?base, ?signature, ?to, ?inmethod),
VarPointsTo(?baseheap, ?base),
InstanceFieldPointsTo(?heap, ?signature, ?baseheap).
.plan 1:(3,2,1,4), 2:(4,2,1,3)
InstanceFieldPointsTo(?heap, ?fld, ?baseheap) :-
Reachable(?inmethod),
StoreInstanceField(?from, ?base, ?fld, ?inmethod),
VarPointsTo(?heap, ?from),
VarPointsTo(?baseheap, ?base).
.plan 1:(3,2,1,4), 2:(4,2,1,3)
VarPointsTo(?heap, ?to) :-
Reachable(?inmethod),
LoadStaticField(?fld, ?to, ?inmethod),
StaticFieldPointsTo(?heap, ?fld).
StaticFieldPointsTo(?heap, ?fld) :-
Reachable(?inmethod),
StoreStaticField(?from, ?fld, ?inmethod),
VarPointsTo(?heap, ?from).
VarPointsTo(?heap, ?this) :-
Reachable(?inMethod),
Instruction_Method(?invocation, ?inMethod),
VirtualMethodInvocation_Base(?invocation, ?base),
VarPointsTo(?heap, ?base),
HeapAllocation_Type(?heap, ?heaptype),
VirtualMethodInvocation_SimpleName(?invocation, ?simplename),
VirtualMethodInvocation_Descriptor(?invocation, ?descriptor),
basic.MethodLookup(?simplename, ?descriptor, ?heaptype, ?toMethod),
ThisVar(?toMethod, ?this).
.plan 1:(4,3,2,1,5,6,7,8,9)
Reachable(?toMethod),
CallGraphEdge(?invocation, ?toMethod) :-
Reachable(?inMethod),
Instruction_Method(?invocation, ?inMethod),
VirtualMethodInvocation_Base(?invocation, ?base),
VarPointsTo(?heap, ?base),
HeapAllocation_Type(?heap, ?heaptype),
VirtualMethodInvocation_SimpleName(?invocation, ?simplename),
VirtualMethodInvocation_Descriptor(?invocation, ?descriptor),
basic.MethodLookup(?simplename, ?descriptor, ?heaptype, ?toMethod).
.plan 1:(4,3,2,1,5,6,7,8)
Reachable(?tomethod),
CallGraphEdge(?invocation, ?tomethod) :-
Reachable(?inmethod),
StaticMethodInvocation(?invocation, ?tomethod, ?inmethod).
Reachable(?tomethod),
CallGraphEdge(?invocation, ?tomethod),
VarPointsTo(?heap, ?this) :-
Reachable(?inmethod),
Instruction_Method(?invocation, ?inmethod),
SpecialMethodInvocation_Base(?invocation, ?base),
VarPointsTo(?heap, ?base),
MethodInvocation_Method(?invocation, ?tomethod),
ThisVar(?tomethod, ?this).
.plan 1:(4,3,2,1,5,6)
Reachable(?method) :-
basic.MainMethodDeclaration(?method).

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,55 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
use rand::{Rng, SeedableRng, StdRng};
use differential_dataflow::input::Input;
use differential_dataflow::operators::Consolidate;
fn main() {
let keys: usize = std::env::args().nth(1).unwrap().parse().unwrap();
let batch: usize = 10_000;
// This computation demonstrates in-place accumulation of arbitrarily large
// volumes of input data, consuming space bounded by the number of distinct keys.
timely::execute_from_args(std::env::args().skip(2), move |worker| {
let index = worker.index();
let peers = worker.peers();
let mut input = worker.dataflow::<(), _, _>(|scope| {
let (input, data) = scope.new_collection::<_, isize>();
data.consolidate();
input
});
let seed: &[_] = &[1, 2, 3, 4];
let mut rng: StdRng = SeedableRng::from_seed(seed);
let timer = ::std::time::Instant::now();
let mut counter = 0;
let mut last_sec = 0;
loop {
for _ in 0 .. batch {
input.insert(rng.gen_range(0, keys as u32));
}
counter += batch;
worker.step();
let elapsed = timer.elapsed();
if elapsed.as_secs() as usize > last_sec {
let secs = elapsed.as_secs() as f64 + (elapsed.subsec_nanos() as f64)/1000000000.0;
if last_sec % peers == index {
println!("tuples: {:?},\telts/sec: {:?}", peers * counter, (peers * counter) as f64 / secs);
}
last_sec = elapsed.as_secs() as usize;
}
}
}).unwrap();
}

View File

@ -0,0 +1,186 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::operators::*;
use timely::order::Product;
use timely::scheduling::Scheduler;
use differential_dataflow::input::Input;
use differential_dataflow::AsCollection;
use differential_dataflow::operators::arrange::ArrangeByKey;
use differential_dataflow::operators::reduce::Reduce;
use differential_dataflow::operators::join::JoinCore;
use differential_dataflow::operators::Iterate;
use differential_dataflow::operators::Consolidate;
fn main() {
let nodes: u32 = std::env::args().nth(1).unwrap().parse().unwrap();
let edges: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let batch: usize = std::env::args().nth(3).unwrap().parse().unwrap();
let pre: usize = std::env::args().nth(4).unwrap().parse().unwrap();
let inspect: bool = std::env::args().nth(5).unwrap() == "inspect";
// define a new timely dataflow computation.
timely::execute_from_args(std::env::args().skip(6), move |worker| {
let timer = ::std::time::Instant::now();
let index = worker.index();
let peers = worker.peers();
let mut probe = timely::dataflow::operators::probe::Handle::new();
// create a dataflow managing an ever-changing edge collection.
let mut graph = worker.dataflow::<Product<(),usize>,_,_>(|scope| {
// create a source operator which will produce random edges and delete them.
timely::dataflow::operators::generic::source(scope, "RandomGraph", |mut capability, info| {
let activator = scope.activator_for(&info.address[..]);
let seed: &[_] = &[1, 2, 3, index];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for edge deletions
let mut additions = 0;
let mut deletions = 0;
let handle = probe.clone();
move |output| {
// do nothing if the probe is not caught up to us
if !handle.less_than(capability.time()) {
let mut time = capability.time().clone();
// println!("{:?}\tintroducing edges for batch starting {:?}", timer.elapsed(), time);
{ // scope to allow session to drop, un-borrow.
let mut session = output.session(&capability);
// we want to send at times.inner + (0 .. batch).
for _ in 0 .. batch {
while additions < time.inner + edges {
if additions % peers == index {
let src = rng1.gen_range(0, nodes);
let dst = rng1.gen_range(0, nodes);
session.give(((src, dst), time, 1));
}
additions += 1;
}
while deletions < time.inner {
if deletions % peers == index {
let src = rng2.gen_range(0, nodes);
let dst = rng2.gen_range(0, nodes);
session.give(((src, dst), time, -1));
}
deletions += 1;
}
time.inner += 1;
}
}
// println!("downgrading {:?} to {:?}", capability, time);
capability.downgrade(&time);
}
activator.activate();
}
})
.probe_with(&mut probe)
.as_collection()
.arrange_by_key()
// .arrange::<OrdValSpineAbom>()
.trace
});
println!("{:?}:\tloading edges", timer.elapsed());
for _ in 0 .. pre {
worker.step();
}
println!("{:?}\tedges loaded; building query dataflows", timer.elapsed());
let mut roots = worker.dataflow(|scope| {
let edges = graph.import(scope);
let (input, roots) = scope.new_collection();
let roots = roots.map(|x| (x, 0));
// repeatedly update minimal distances each node can be reached from each root
roots.iterate(|dists| {
let edges = edges.enter(&dists.scope());
let roots = roots.enter(&dists.scope());
dists.arrange_by_key()
.join_core(&edges, |_k,l,d| Some((*d, l+1)))
.concat(&roots)
.reduce(|_, s, t| t.push((*s[0].0, 1)))
})
.map(|(_node, dist)| dist)
.consolidate()
.inspect(|x| println!("distance update: {:?}", x))
.probe_with(&mut probe);
input
});
let mut query = worker.dataflow(|scope| {
let edges = graph.import(scope);
let (input, query) = scope.new_collection();
query.map(|x| (x, x))
.join_core(&edges, |_n, &q, &d| Some((d, q)))
.join_core(&edges, |_n, &q, &d| Some((d, q)))
.join_core(&edges, |_n, &q, &d| Some((d, q)))
.filter(move |_| inspect)
.map(|x| x.1)
.consolidate()
.inspect(|x| println!("{:?}", x))
.probe_with(&mut probe);
input
});
println!("{:?}\tquery dataflows built; querying", timer.elapsed());
// the trace will not compact unless we release capabilities.
// we drop rather than continually downgrade them as we run.
drop(graph);
if batch > 0 {
for round in 0 .. {
let mut time = query.time().clone();
// roots.insert(round % nodes);
if index == 0 {
query.insert(round % nodes);
}
time.inner += batch;
roots.advance_to(time); roots.flush();
query.advance_to(time); query.flush();
if index == 0 {
query.remove(round % nodes);
}
worker.step_while(|| probe.less_than(&time));
println!("{:?}\tquery round {:?} complete", timer.elapsed(), round);
}
}
}).unwrap();
}

View File

@ -0,0 +1,131 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::*;
use timely::dataflow::operators::probe::Handle;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::logging::DifferentialEvent;
type Node = u32;
type Edge = (Node, Node);
fn main() {
let nodes: u32 = std::env::args().nth(1).unwrap().parse().unwrap();
let edges: u32 = std::env::args().nth(2).unwrap().parse().unwrap();
let batch: u32 = std::env::args().nth(3).unwrap().parse().unwrap();
let rounds: u32 = std::env::args().nth(4).unwrap().parse().unwrap();
let inspect: bool = std::env::args().nth(5).unwrap() == "inspect";
// define a new computational scope, in which to run BFS
timely::execute_from_args(std::env::args(), move |worker| {
if let Ok(addr) = ::std::env::var("DIFFERENTIAL_LOG_ADDR") {
eprintln!("enabled DIFFERENTIAL logging to {}", addr);
if let Ok(stream) = ::std::net::TcpStream::connect(&addr) {
let writer = ::timely::dataflow::operators::capture::EventWriter::new(stream);
let mut logger = ::timely::logging::BatchLogger::new(writer);
worker.log_register().insert::<DifferentialEvent,_>("differential/arrange", move |time, data|
logger.publish_batch(time, data)
);
}
else {
panic!("Could not connect to differential log address: {:?}", addr);
}
}
let timer = ::std::time::Instant::now();
// define BFS dataflow; return handles to roots and edges inputs
let mut probe = Handle::new();
let (mut roots, mut graph) = worker.dataflow(|scope| {
let (root_input, roots) = scope.new_collection();
let (edge_input, graph) = scope.new_collection();
let mut result = bfs(&graph, &roots);
if !inspect {
result = result.filter(|_| false);
}
result.map(|(_,l)| l)
.consolidate()
.inspect(|x| println!("\t{:?}", x))
.probe_with(&mut probe);
(root_input, edge_input)
});
let seed: &[_] = &[1, 2, 3, 4];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for edge deletions
roots.insert(0);
roots.close();
println!("performing BFS on {} nodes, {} edges:", nodes, edges);
if worker.index() == 0 {
for _ in 0 .. edges {
graph.insert((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)));
}
}
println!("{:?}\tloaded", timer.elapsed());
graph.advance_to(1);
graph.flush();
worker.step_while(|| probe.less_than(graph.time()));
println!("{:?}\tstable", timer.elapsed());
for round in 0 .. rounds {
for element in 0 .. batch {
if worker.index() == 0 {
graph.insert((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)));
graph.remove((rng2.gen_range(0, nodes), rng2.gen_range(0, nodes)));
}
graph.advance_to(2 + round * batch + element);
}
graph.flush();
let timer2 = ::std::time::Instant::now();
worker.step_while(|| probe.less_than(&graph.time()));
if worker.index() == 0 {
let elapsed = timer2.elapsed();
println!("{:?}\t{:?}:\t{}", timer.elapsed(), round, elapsed.as_secs() * 1000000000 + (elapsed.subsec_nanos() as u64));
}
}
println!("finished; elapsed: {:?}", timer.elapsed());
}).unwrap();
}
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn bfs<G: Scope>(edges: &Collection<G, Edge>, roots: &Collection<G, Node>) -> Collection<G, (Node, u32)>
where G::Timestamp: Lattice+Ord {
// initialize roots as reaching themselves at distance 0
let nodes = roots.map(|x| (x, 0));
// repeatedly update minimal distances each node can be reached from each root
nodes.iterate(|inner| {
let edges = edges.enter(&inner.scope());
let nodes = nodes.enter(&inner.scope());
inner.join_map(&edges, |_k,l,d| (*d, l+1))
.concat(&nodes)
.reduce(|_, s, t| t.push((*s[0].0, 1)))
})
}

View File

@ -0,0 +1,319 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
extern crate serde;
extern crate rdkafka;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::*;
use timely::dataflow::operators::probe::Handle;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::lattice::Lattice;
type Node = u32;
type Edge = (Node, Node);
fn main() {
let nodes: u32 = std::env::args().nth(1).unwrap().parse().unwrap();
let edges: u32 = std::env::args().nth(2).unwrap().parse().unwrap();
let batch: u32 = std::env::args().nth(3).unwrap().parse().unwrap();
let topic = std::env::args().nth(4).unwrap();
let write = std::env::args().any(|x| x == "write");
let read = std::env::args().any(|x| x == "read");
// define a new computational scope, in which to run BFS
timely::execute_from_args(std::env::args(), move |worker| {
let timer = ::std::time::Instant::now();
// define BFS dataflow; return handles to roots and edges inputs
let mut probe = Handle::new();
let (mut roots, mut graph, _write_token, _read_token) = worker.dataflow(|scope| {
let (root_input, roots) = scope.new_collection();
let (edge_input, graph) = scope.new_collection();
let result = bfs(&graph, &roots);
let result =
result.map(|(_,l)| l)
.consolidate()
.probe_with(&mut probe);
let write_token = if write {
Some(kafka::create_sink(&result.inner, "localhost:9092", &topic))
} else { None };
let read_token = if read {
let (read_token, stream) = kafka::create_source(result.scope(), "localhost:9092", &topic, "group");
use differential_dataflow::AsCollection;
stream
.as_collection()
.negate()
.concat(&result)
.consolidate()
.inspect(|x| println!("In error: {:?}", x))
.probe_with(&mut probe)
.assert_empty()
;
Some(read_token)
} else { None };
(root_input, edge_input, write_token, read_token)
});
let seed: &[_] = &[1, 2, 3, 4];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for edge deletions
roots.insert(0);
roots.close();
println!("performing BFS on {} nodes, {} edges:", nodes, edges);
if worker.index() == 0 {
for _ in 0 .. edges {
graph.insert((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)));
}
}
println!("{:?}\tloaded", timer.elapsed());
graph.advance_to(1);
graph.flush();
worker.step_while(|| probe.less_than(graph.time()));
println!("{:?}\tstable", timer.elapsed());
for round in 0 .. {
if write {
std::thread::sleep(std::time::Duration::from_millis(100));
}
for element in 0 .. batch {
if worker.index() == 0 {
graph.insert((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)));
graph.remove((rng2.gen_range(0, nodes), rng2.gen_range(0, nodes)));
}
graph.advance_to(2 + round * batch + element);
}
graph.flush();
let timer2 = ::std::time::Instant::now();
worker.step_while(|| probe.less_than(&graph.time()));
if worker.index() == 0 {
let elapsed = timer2.elapsed();
println!("{:?}\t{:?}:\t{}", timer.elapsed(), round, elapsed.as_secs() * 1000000000 + (elapsed.subsec_nanos() as u64));
}
}
println!("finished; elapsed: {:?}", timer.elapsed());
}).unwrap();
}
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn bfs<G: Scope>(edges: &Collection<G, Edge>, roots: &Collection<G, Node>) -> Collection<G, (Node, u32)>
where G::Timestamp: Lattice+Ord {
// initialize roots as reaching themselves at distance 0
let nodes = roots.map(|x| (x, 0));
// repeatedly update minimal distances each node can be reached from each root
nodes.iterate(|inner| {
let edges = edges.enter(&inner.scope());
let nodes = nodes.enter(&inner.scope());
inner.join_map(&edges, |_k,l,d| (*d, l+1))
.concat(&nodes)
.reduce(|_, s, t| t.push((*s[0].0, 1)))
})
}
pub mod kafka {
use serde::{Serialize, Deserialize};
use timely::scheduling::SyncActivator;
use rdkafka::{ClientContext, config::ClientConfig};
use rdkafka::consumer::{BaseConsumer, ConsumerContext};
use rdkafka::error::{KafkaError, RDKafkaError};
use differential_dataflow::capture::Writer;
use std::hash::Hash;
use timely::progress::Timestamp;
use timely::dataflow::{Scope, Stream};
use differential_dataflow::ExchangeData;
use differential_dataflow::lattice::Lattice;
/// Creates a Kafka source from supplied configuration information.
pub fn create_source<G, D, T, R>(scope: G, addr: &str, topic: &str, group: &str) -> (Box<dyn std::any::Any + Send + Sync>, Stream<G, (D, T, R)>)
where
G: Scope<Timestamp = T>,
D: ExchangeData + Hash + for<'a> serde::Deserialize<'a>,
T: ExchangeData + Hash + for<'a> serde::Deserialize<'a> + Timestamp + Lattice,
R: ExchangeData + Hash + for<'a> serde::Deserialize<'a>,
{
differential_dataflow::capture::source::build(scope, |activator| {
let source = KafkaSource::new(addr, topic, group, activator);
differential_dataflow::capture::YieldingIter::new_from(Iter::<D,T,R>::new_from(source), std::time::Duration::from_millis(10))
})
}
pub fn create_sink<G, D, T, R>(stream: &Stream<G, (D, T, R)>, addr: &str, topic: &str) -> Box<dyn std::any::Any>
where
G: Scope<Timestamp = T>,
D: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a>,
T: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a> + Timestamp + Lattice,
R: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a>,
{
use std::rc::Rc;
use std::cell::RefCell;
use differential_dataflow::hashable::Hashable;
let sink = KafkaSink::new(addr, topic);
let result = Rc::new(RefCell::new(sink));
let sink_hash = (addr.to_string(), topic.to_string()).hashed();
differential_dataflow::capture::sink::build(
&stream,
sink_hash,
Rc::downgrade(&result),
Rc::downgrade(&result),
);
Box::new(result)
}
pub struct KafkaSource {
consumer: BaseConsumer<ActivationConsumerContext>,
}
impl KafkaSource {
pub fn new(addr: &str, topic: &str, group: &str, activator: SyncActivator) -> Self {
let mut kafka_config = ClientConfig::new();
// This is mostly cargo-cult'd in from `source/kafka.rs`.
kafka_config.set("bootstrap.servers", &addr.to_string());
kafka_config
.set("enable.auto.commit", "false")
.set("auto.offset.reset", "earliest");
kafka_config.set("topic.metadata.refresh.interval.ms", "30000"); // 30 seconds
kafka_config.set("fetch.message.max.bytes", "134217728");
kafka_config.set("group.id", group);
kafka_config.set("isolation.level", "read_committed");
let activator = ActivationConsumerContext(activator);
let consumer = kafka_config.create_with_context::<_, BaseConsumer<_>>(activator).unwrap();
use rdkafka::consumer::Consumer;
consumer.subscribe(&[topic]).unwrap();
Self {
consumer,
}
}
}
pub struct Iter<D, T, R> {
pub source: KafkaSource,
phantom: std::marker::PhantomData<(D, T, R)>,
}
impl<D, T, R> Iter<D, T, R> {
/// Constructs a new iterator from a bytes source.
pub fn new_from(source: KafkaSource) -> Self {
Self {
source,
phantom: std::marker::PhantomData,
}
}
}
impl<D, T, R> Iterator for Iter<D, T, R>
where
D: for<'a>Deserialize<'a>,
T: for<'a>Deserialize<'a>,
R: for<'a>Deserialize<'a>,
{
type Item = differential_dataflow::capture::Message<D, T, R>;
fn next(&mut self) -> Option<Self::Item> {
use rdkafka::message::Message;
self.source
.consumer
.poll(std::time::Duration::from_millis(0))
.and_then(|result| result.ok())
.and_then(|message| {
message.payload().and_then(|message| bincode::deserialize::<differential_dataflow::capture::Message<D, T, R>>(message).ok())
})
}
}
/// An implementation of [`ConsumerContext`] that unparks the wrapped thread
/// when the message queue switches from nonempty to empty.
struct ActivationConsumerContext(SyncActivator);
impl ClientContext for ActivationConsumerContext { }
impl ActivationConsumerContext {
fn activate(&self) {
self.0.activate().unwrap();
}
}
impl ConsumerContext for ActivationConsumerContext {
fn message_queue_nonempty_callback(&self) {
self.activate();
}
}
use std::time::Duration;
use rdkafka::producer::DefaultProducerContext;
use rdkafka::producer::{BaseRecord, ThreadedProducer};
pub struct KafkaSink {
topic: String,
producer: ThreadedProducer<DefaultProducerContext>,
buffer: Vec<u8>,
}
impl KafkaSink {
pub fn new(addr: &str, topic: &str) -> Self {
let mut config = ClientConfig::new();
config.set("bootstrap.servers", &addr);
config.set("queue.buffering.max.kbytes", &format!("{}", 16 << 20));
config.set("queue.buffering.max.messages", &format!("{}", 10_000_000));
config.set("queue.buffering.max.ms", &format!("{}", 10));
let producer = config
.create_with_context::<_, ThreadedProducer<_>>(DefaultProducerContext)
.expect("creating kafka producer for kafka sinks failed");
Self {
producer,
topic: topic.to_string(),
buffer: Vec::new(),
}
}
}
impl<T: Serialize> Writer<T> for KafkaSink {
fn poll(&mut self, item: &T) -> Option<Duration> {
self.buffer.clear();
bincode::serialize_into(&mut self.buffer, item).expect("Writing to a `Vec<u8>` cannot fail");
let record = BaseRecord::<[u8], _>::to(&self.topic).payload(&self.buffer);
self.producer.send(record).err().map(|(e, _)| {
if let KafkaError::MessageProduction(RDKafkaError::QueueFull) = e {
Duration::from_secs(1)
} else {
// TODO(frank): report this error upwards so the user knows the sink is dead.
Duration::from_secs(1)
}
})
}
fn done(&self) -> bool {
self.producer.in_flight_count() == 0
}
}
}

View File

@ -0,0 +1,101 @@
extern crate timely;
extern crate differential_dataflow;
use timely::dataflow::operators::probe::Handle;
use differential_dataflow::input::Input;
fn main() {
let keys: usize = std::env::args().nth(1).unwrap().parse().unwrap();
let size: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let mode = std::env::args().any(|a| a == "new");
if mode {
println!("Running NEW arrangement");
}
else {
println!("Running OLD arrangement");
}
let timer1 = ::std::time::Instant::now();
let timer2 = timer1.clone();
// define a new computational scope, in which to run BFS
timely::execute_from_args(std::env::args(), move |worker| {
// define BFS dataflow; return handles to roots and edges inputs
let mut probe = Handle::new();
let (mut data_input, mut keys_input) = worker.dataflow(|scope| {
use differential_dataflow::operators::{arrange::Arrange, JoinCore};
use differential_dataflow::trace::implementations::ord::{OrdKeySpine, ColKeySpine};
let (data_input, data) = scope.new_collection::<String, isize>();
let (keys_input, keys) = scope.new_collection::<String, isize>();
if mode {
let data = data.arrange::<ColKeySpine<_,_,_>>();
let keys = keys.arrange::<ColKeySpine<_,_,_>>();
keys.join_core(&data, |_k, &(), &()| Option::<()>::None)
.probe_with(&mut probe);
}
else {
let data = data.arrange::<OrdKeySpine<_,_,_>>();
let keys = keys.arrange::<OrdKeySpine<_,_,_>>();
keys.join_core(&data, |_k, &(), &()| Option::<()>::None)
.probe_with(&mut probe);
}
(data_input, keys_input)
});
// Load up data in batches.
let mut counter = 0;
while counter < 10 * keys {
let mut i = worker.index();
while i < size {
let val = (counter + i) % keys;
data_input.insert(format!("{:?}", val));
i += worker.peers();
}
counter += size;
data_input.advance_to(data_input.time() + 1);
data_input.flush();
keys_input.advance_to(keys_input.time() + 1);
keys_input.flush();
while probe.less_than(data_input.time()) {
worker.step();
}
}
println!("{:?}\tloading complete", timer1.elapsed());
let mut queries = 0;
while queries < 10 * keys {
let mut i = worker.index();
while i < size {
let val = (queries + i) % keys;
keys_input.insert(format!("{:?}", val));
i += worker.peers();
}
queries += size;
data_input.advance_to(data_input.time() + 1);
data_input.flush();
keys_input.advance_to(keys_input.time() + 1);
keys_input.flush();
while probe.less_than(data_input.time()) {
worker.step();
}
}
println!("{:?}\tqueries complete", timer1.elapsed());
// loop { }
}).unwrap();
println!("{:?}\tshut down", timer2.elapsed());
}

View File

@ -0,0 +1,77 @@
extern crate timely;
extern crate differential_dataflow;
use differential_dataflow::input::Input;
use differential_dataflow::operators::Threshold;
fn main() {
let large: usize = std::env::args().nth(1).unwrap().parse().unwrap();
let small: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let batch: usize = std::env::args().nth(3).unwrap().parse().unwrap();
let total: usize = std::env::args().nth(4).unwrap().parse().unwrap();
// define a new timely dataflow computation.
timely::execute_from_args(std::env::args().skip(3), move |worker| {
let timer = ::std::time::Instant::now();
let mut probe = timely::dataflow::operators::probe::Handle::new();
// create a dataflow managing an ever-changing edge collection.
let mut handle = worker.dataflow(|scope| {
let (handle, input) = scope.new_collection();
input.distinct().probe_with(&mut probe);
handle
});
println!("{:?}:\tloading edges", timer.elapsed());
let mut next = batch;
let mut value = worker.index();
while value < total {
if value >= next {
handle.advance_to(next);
handle.flush();
next += batch;
while probe.less_than(handle.time()) { worker.step(); }
// println!("{:?}\tround {} loaded", timer.elapsed(), next);
}
handle.advance_to(value);
handle.insert(value % large);
handle.insert(value % small);
value += worker.peers();
}
handle.advance_to(total);
handle.flush();
while probe.less_than(handle.time()) { worker.step(); }
println!("{:?}\tdata loaded", timer.elapsed());
let mut next = batch;
let mut value = worker.index();
while value < total {
if value >= next {
handle.advance_to(total + next);
handle.flush();
next += batch;
while probe.less_than(handle.time()) { worker.step(); }
// println!("{:?}\tround {} unloaded", timer.elapsed(), next);
}
handle.advance_to(total + value);
handle.remove(value % large);
handle.remove(value % small);
value += worker.peers();
}
handle.advance_to(total + total);
handle.flush();
while probe.less_than(handle.time()) { worker.step(); }
println!("{:?}\tdata unloaded", timer.elapsed());
while worker.step_or_park(None) { }
}).unwrap();
}

View File

@ -0,0 +1,147 @@
//! An example illustrating the use of the cursors API to enumerate arrangements.
//!
//! Dataflow graph construction:
//! * Create a trivial dataflow graph with a single input collection that stores graph
//! edges of type `(u32, u32)`.
//! * Arrange this collection by the first coordinate of an edge; return the trace of
//! this arrangement, so that it can be searched and enumerated at runtime.
//!
//! At runtime:
//! * At every round `i`, insert a pair of edges `(i,i+1)`, `(i+1, i)` to the graph, and
//! delete one existing edge added at the previous round: `(i-1, i)`.
//! * At the end of the round, dump the entire content of the trace using the
//! `CursorDebug::to_vec` method.
//! * Validate the entire content of the trace after the last round.
//!
//! Example invocation (3 rounds, 4 workers): `cargo run --example cursors -- 3 -w 4`.
//!
//! Expected output:
//! ```
//! round 1, w0 1:(1, 2): [(0, 1)]
//! round 1, w3 2:(2, 1): [(0, 1)]
//! round 2, w2 3:(3, 2): [(1, 1)]
//! round 2, w3 2:(2, 1): [(0, 1)]
//! round 2, w3 2:(2, 3): [(1, 1)]
//! round 2, w0 1:(1, 2): [(0, 1), (1, -1)]
//! round 3, w3 2:(2, 1): [(2, 1)]
//! round 3, w3 2:(2, 3): [(2, 1), (2, -1)]
//! round 3, w2 3:(3, 2): [(2, 1)]
//! round 3, w2 3:(3, 4): [(2, 1)]
//! round 3, w1 4:(4, 3): [(2, 1)]
//! Final graph: {(2, 1): 1, (3, 2): 1, (3, 4): 1, (4, 3): 1}
//! ```
extern crate differential_dataflow;
extern crate timely;
use std::fmt::Debug;
use std::collections::BTreeMap;
use timely::dataflow::operators::probe::Handle;
use timely::progress::frontier::AntichainRef;
use timely::dataflow::operators::Probe;
use differential_dataflow::input::Input;
use differential_dataflow::operators::arrange::ArrangeByKey;
use differential_dataflow::trace::cursor::Cursor;
use differential_dataflow::trace::TraceReader;
type Node = u32;
type Edge = (Node, Node);
type Time = u32;
type Diff = isize;
fn main() {
let rounds: u32 = std::env::args().nth(1).unwrap().parse().unwrap();
let mut summaries = timely::execute_from_args(std::env::args(), move |worker| {
let mut probe = Handle::new();
let (mut graph, mut graph_trace) = worker.dataflow(|scope| {
let (graph_input, graph) = scope.new_collection();
let graph_arr = graph.map(|(x, y): Edge| (x, (x, y))).arrange_by_key();
let graph_trace = graph_arr.trace.clone();
/* Be sure to attach probe to arrangements we want to enumerate;
* so we know when all updates for a given epoch have been added to the arrangement. */
graph_arr
.stream
//.inspect(move |x| println!("{:?}", x))
.probe_with(&mut probe);
/* Return `graph_trace`, so we can obtain cursor for the arrangement at runtime. */
(graph_input, graph_trace)
});
if worker.index() != 0 {
graph.close();
for i in 1..rounds + 1 {
/* Advance the trace frontier to enable trace compaction. */
graph_trace.set_physical_compaction(AntichainRef::new(&[i]));
graph_trace.set_logical_compaction(AntichainRef::new(&[i]));
worker.step_while(|| probe.less_than(&i));
dump_cursor(i, worker.index(), &mut graph_trace);
}
} else {
/* Only worker 0 feeds inputs to the dataflow. */
for i in 1..rounds + 1 {
graph.insert((i, i + 1));
graph.insert((i + 1, i));
if i > 1 {
graph.remove((i - 1, i));
}
graph.advance_to(i);
graph.flush();
graph_trace.set_physical_compaction(AntichainRef::new(&[i]));
graph_trace.set_logical_compaction(AntichainRef::new(&[i]));
worker.step_while(|| probe.less_than(graph.time()));
dump_cursor(i, worker.index(), &mut graph_trace);
}
}
/* Return trace content after the last round. */
let (mut cursor, storage) = graph_trace.cursor();
cursor.to_vec(&storage)
})
.unwrap().join();
/* Aggregate trace summaries from individual workers to reconstrust the content of the graph. */
let mut graph_content: BTreeMap<Edge, Diff> = BTreeMap::new();
for summary in summaries.drain(..) {
let mut summary_vec: Vec<((Node, Edge), Vec<(Time, Diff)>)> = summary.unwrap();
for ((_, edge), timestamps) in summary_vec.drain(..) {
/* Sum up all diffs to get the number of occurrences of `edge` at the end of the
* computation. */
let diff: Diff = timestamps.iter().map(|(_,d)|d).sum();
if diff != 0 {
*graph_content.entry(edge).or_insert(0) += diff;
}
}
}
println!("Final graph: {:?}", graph_content);
/* Make sure that final graph content is correct. */
let mut expected_graph_content: BTreeMap<Edge, Diff> = BTreeMap::new();
for i in 1..rounds+1 {
expected_graph_content.insert((i+1, i), 1);
}
expected_graph_content.insert((rounds, rounds+1), 1);
assert_eq!(graph_content, expected_graph_content);
}
fn dump_cursor<Tr>(round: u32, index: usize, trace: &mut Tr)
where
Tr: TraceReader,
Tr::Key: Debug + Clone,
Tr::Val: Debug + Clone,
Tr::Time: Debug + Clone,
Tr::R: Debug + Clone,
{
let (mut cursor, storage) = trace.cursor();
for ((k, v), diffs) in cursor.to_vec(&storage).iter() {
println!("round {}, w{} {:?}:{:?}: {:?}", round, index, *k, *v, diffs);
}
}

View File

@ -0,0 +1,184 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
use rand::{Rng, SeedableRng, StdRng};
use differential_dataflow::input::Input;
use differential_dataflow::operators::count::CountTotal;
fn main() {
let nodes: u32 = std::env::args().nth(1).unwrap().parse().unwrap();
let edges: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let batch: usize = std::env::args().nth(3).unwrap().parse().unwrap();
let inspect: bool = std::env::args().nth(4).unwrap() == "inspect";
let open_loop: bool = std::env::args().nth(5).unwrap() == "open-loop";
// define a new computational scope, in which to run BFS
timely::execute_from_args(std::env::args().skip(6), move |worker| {
let timer = ::std::time::Instant::now();
let index = worker.index();
let peers = worker.peers();
// create a degree counting differential dataflow
let (mut input, probe) = worker.dataflow(|scope| {
// create edge input, count a few ways.
let (input, edges) = scope.new_collection();
let degrs = edges.map(|(src, _dst)| src)
.count_total()
;
// // pull of count, and count.
// let distr = degrs.map(|(_src, cnt)| cnt as usize)
// .count_total();
// show us something about the collection, notice when done.
let probe = if inspect {
degrs.inspect(|x| println!("observed: {:?}", x))
.probe()
}
else { degrs.probe() };
(input, probe)
});
let seed: &[_] = &[1, 2, 3, index];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
// load up graph dataz
for _ in 0 .. (edges / peers) + if index < (edges % peers) { 1 } else { 0 } {
input.insert((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)))
}
input.advance_to(1u64);
input.flush();
worker.step_while(|| probe.less_than(input.time()));
println!("round 0 finished after {:?} (loading)", timer.elapsed());
if batch > 0 {
if !open_loop {
let mut counts = vec![0u64; 64];
// let mut changed = 0;
// closed-loop latency-throughput test, parameterized by batch size.
let timer = ::std::time::Instant::now();
let mut wave = 1;
while timer.elapsed().as_secs() < 10 {
for round in 0 .. batch {
input.advance_to((((wave * batch) + round) * peers + index) as u64);
input.insert((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)));
input.remove((rng2.gen_range(0, nodes), rng2.gen_range(0, nodes)));
}
// changed += 2 * batch;
wave += 1;
input.advance_to((wave * batch * peers) as u64);
input.flush();
let elapsed1 = timer.elapsed();
let elapsed1_ns = elapsed1.as_secs() * 1_000_000_000 + (elapsed1.subsec_nanos() as u64);
worker.step_while(|| probe.less_than(input.time()));
let elapsed2 = timer.elapsed();
let elapsed2_ns = elapsed2.as_secs() * 1_000_000_000 + (elapsed2.subsec_nanos() as u64);
let count_index = (elapsed2_ns - elapsed1_ns).next_power_of_two().trailing_zeros() as usize;
counts[count_index] += 1;
}
let elapsed = timer.elapsed();
let seconds = elapsed.as_secs() as f64 + (elapsed.subsec_nanos() as f64) / 1000000000.0;
println!("{:?}, {:?}", seconds / (wave - 1) as f64, ((wave - 1) * batch * peers) as f64 / seconds);
println!("latencies:");
for index in 0 .. counts.len() {
if counts[index] > 0 {
println!("\tcount[{}]:\t{}", index, counts[index]);
}
}
}
else {
let requests_per_sec = batch;
let ns_per_request = 1_000_000_000 / requests_per_sec;
let mut request_counter = peers + index; // skip first request for each.
let mut ack_counter = peers + index;
let mut counts = vec![0u64; 64];
let timer = ::std::time::Instant::now();
let mut inserted_ns = 1;
loop {
// Open-loop latency-throughput test, parameterized by offered rate `ns_per_request`.
let elapsed = timer.elapsed();
let elapsed_ns = elapsed.as_secs() * 1_000_000_000 + (elapsed.subsec_nanos() as u64);
// Determine completed ns.
let acknowledged_ns: u64 = probe.with_frontier(|frontier| frontier[0]);
// any un-recorded measurements that are complete should be recorded.
while ((ack_counter * ns_per_request) as u64) < acknowledged_ns {
let requested_at = (ack_counter * ns_per_request) as u64;
let count_index = (elapsed_ns - requested_at).next_power_of_two().trailing_zeros() as usize;
counts[count_index] += 1;
ack_counter += peers;
if (ack_counter & ((1 << 20) - 1)) == 0 {
println!("latencies:");
for index in 0 .. counts.len() {
if counts[index] > 0 {
println!("\tcount[{}]:\t{}", index, counts[index]);
}
}
counts = vec![0u64; 64];
}
}
// Now, should we introduce more records before stepping the worker?
//
// Thinking: inserted_ns - acknowledged_ns is some amount of time that
// is currently outstanding in the system, and we needn't advance our
// inputs unless by this order of magnitude.
//
// The more sophisticated plan is: we compute the next power of two
// greater than inserted_ns - acknowledged_ns and look for the last
// multiple of this number in the interval [inserted_ns, elapsed_ns].
// If such a multiple exists, we introduce records to that point and
// advance the input.
let scale = (inserted_ns - acknowledged_ns).next_power_of_two();
// last multiple of `scale` less than `elapsed_ns`.
let target_ns = elapsed_ns & !(scale - 1);
if inserted_ns < target_ns {
// println!("acknowledged_ns:\t{:?}\t{:X}", acknowledged_ns, acknowledged_ns);
// println!("inserted_ns:\t\t{:?}\t{:X}", inserted_ns, inserted_ns);
// println!("target_ns:\t\t{:?}\t{:X}", target_ns, target_ns);
// println!("elapsed_ns:\t\t{:?}\t{:X}", elapsed_ns, elapsed_ns);
// println!();
while ((request_counter * ns_per_request) as u64) < target_ns {
input.advance_to((request_counter * ns_per_request) as u64);
input.insert((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)));
input.remove((rng2.gen_range(0, nodes), rng2.gen_range(0, nodes)));
request_counter += peers;
}
input.advance_to(target_ns);
input.flush();
inserted_ns = target_ns;
}
worker.step();
}
}
}
}).unwrap();
}

View File

@ -0,0 +1,112 @@
extern crate timely;
extern crate differential_dataflow;
use timely::dataflow::operators::probe::Handle;
use timely::dataflow::operators::Map;
use differential_dataflow::input::Input;
use differential_dataflow::AsCollection;
use differential_dataflow::operators::*;
use differential_dataflow::operators::arrange::{ArrangeByKey, ArrangeBySelf};
use differential_dataflow::trace::wrappers::freeze::freeze;
fn main() {
// define a new computational scope, in which to run BFS
timely::execute_from_args(std::env::args(), move |worker| {
// define BFS dataflow; return handles to roots and edges inputs
let mut probe = Handle::new();
let (mut rules, mut graph) = worker.dataflow(|scope| {
let (rule_input, rules) = scope.new_collection();
let (edge_input, graph) = scope.new_collection();
let result = graph.iterate(|inner| {
let rules = rules.enter(&inner.scope());
let arranged = inner.arrange_by_key();
// rule 0: remove self-loops:
let freeze0 = freeze(&arranged, |t| {
if t.inner <= 0 {
let mut t = t.clone();
t.inner = 0;
Some(t)
}
else { None }
});
let rule0 = freeze0.as_collection(|&k,&v| (k,v))
.filter(|x| x.0 == x.1)
.negate()
.inspect(|x| println!("rule0:\t{:?}", x));
// subtract self loops once, not each round.
let rule0 = &rule0.inner
.map_in_place(|dtr| { dtr.1.inner += 1; })
.as_collection()
.negate()
.concat(&rule0);
// rule 1: overwrite keys present in `rules`
let freeze1 = freeze(&arranged, |t| {
if t.inner <= 1 {
let mut t = t.clone();
t.inner = 1;
Some(t)
}
else { None }
});
let rule1 = freeze1.join_core(&rules.map(|(x,_y)| x).distinct().arrange_by_self(), |&k, &x, &()| Some((k,x)))
.negate()
.concat(&rules.inner.map_in_place(|dtr| dtr.1.inner = 1).as_collection())
.inspect(|x| println!("rule1:\t{:?}", x));
let rule1 = &rule1.inner
.map_in_place(|dtr| { dtr.1.inner += 1; })
.as_collection()
.negate()
.concat(&rule1);
inner
.concat(&rule0)
.concat(&rule1)
.consolidate()
.inspect(|x| println!("inner:\t{:?}", x))
});
result.consolidate()
.inspect(|x| println!("output\t{:?}", x))
.probe_with(&mut probe);
(rule_input, edge_input)
});
println!("starting up");
graph.insert((0, 1));
graph.insert((1, 1));
graph.insert((2, 1));
graph.insert((2, 3));
graph.advance_to(1); graph.flush();
rules.advance_to(1); rules.flush();
while probe.less_than(graph.time()) { worker.step(); }
println!("round 0 complete");
graph.insert((3, 3));
graph.advance_to(2); graph.flush();
rules.advance_to(2); rules.flush();
while probe.less_than(graph.time()) { worker.step(); }
println!("round 1 complete");
rules.insert((2, 2));
graph.advance_to(3); graph.flush();
rules.advance_to(3); rules.flush();
while probe.less_than(graph.time()) { worker.step(); }
println!("round 2 complete");
}).unwrap();
}

View File

@ -0,0 +1,277 @@
extern crate indexmap;
extern crate timely;
extern crate differential_dataflow;
use std::io::{BufRead, BufReader};
use std::fs::File;
use indexmap::IndexMap;
use timely::progress::Timestamp;
use timely::order::Product;
use timely::dataflow::Scope;
use timely::dataflow::scopes::ScopeParent;
use differential_dataflow::Collection;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::input::{Input, InputSession};
use differential_dataflow::operators::arrange::{ArrangeByKey, ArrangeBySelf};
use differential_dataflow::operators::iterate::Variable;
use differential_dataflow::operators::{Threshold, JoinCore, Consolidate};
type Node = usize;
type Edge = (Node, Node);
type Iter = usize;
type Diff = isize;
/// A direction we might traverse an edge.
#[derive(Debug)]
pub enum Relation {
/// The forward direction of the relation, A(x,y).
Forward(String),
/// The reverse direction of the relation, A(y,x).
Reverse(String),
}
/// A rule for producing edges based on paths.
///
/// The intent is that we should add to the left hand relation any pair `(x,y)` for which
/// there exists a path x -> z0 -> z1 -> .. -> y through the specified relations.
#[derive(Debug)]
pub struct Production {
/// The name of the edge set to populate.
pub left_hand: String,
/// A sequence of edges to follow.
pub relations: Vec<Relation>,
}
impl<'a> From<&'a str> for Production {
fn from(text: &'a str) -> Production {
let mut names = text.split_whitespace();
// Extract the target.
let left_hand = names.next().expect("All rules must have a target.").to_string();
// Collect all remaining names.
let relations = names.map(|name| {
if name.starts_with('~') { Relation::Reverse(name.split_at(1).1.to_string()) }
else { Relation::Forward(name.to_string()) }
}).collect();
Production { left_hand, relations }
}
}
#[derive(Debug)]
pub struct Query {
/// A sequence of productions.
///
/// Empty productions are a great way to define input relations, but are otherwise ignored.
pub productions: Vec<Production>,
}
use differential_dataflow::trace::implementations::ord::{OrdValSpine, OrdKeySpine};
use differential_dataflow::operators::arrange::{Arranged, TraceAgent};
type TraceKeyHandle<K,T,R> = TraceAgent<OrdKeySpine<K, T, R>>;
type TraceValHandle<K,V,T,R> = TraceAgent<OrdValSpine<K, V, T, R>>;
type Arrange<G,K,V,R> = Arranged<G, TraceValHandle<K, V, <G as ScopeParent>::Timestamp, R>>;
/// An evolving set of edges.
///
/// An edge variable represents the distinct set of edges found in any of a set of productions
/// which the variable. A newly created edge variable supports the addition of new productions
/// up until the variable is "completed" at which point the definition is locked in.
///
/// An edge variable provides arranged representations of its contents, even before they are
/// completely defined, in support of recursively defined productions.
pub struct EdgeVariable<G: Scope> where G::Timestamp : Lattice {
variable: Variable<G, Edge, Diff>,
current: Collection<G, Edge, Diff>,
forward: Option<Arrange<G, Node, Node, Diff>>,
reverse: Option<Arrange<G, Node, Node, Diff>>,
}
impl<G: Scope> EdgeVariable<G> where G::Timestamp : Lattice {
/// Creates a new variable initialized with `source`.
pub fn from(source: &Collection<G, Edge>, step: <G::Timestamp as Timestamp>::Summary) -> Self {
let variable = Variable::new(&mut source.scope(), step);
EdgeVariable {
variable: variable,
current: source.clone(),
forward: None,
reverse: None,
}
}
/// Concatenates `production` into the definition of the variable.
pub fn add_production(&mut self, production: &Collection<G, Edge, Diff>) {
self.current = self.current.concat(production);
}
/// Finalizes the variable, connecting its recursive definition.
///
/// Failure to call `complete` on a variable results in a non-recursively defined
/// collection, whose contents are just its initial `source` data.
pub fn complete(self) {
let distinct = self.current.distinct();
// distinct.map(|_| ()).consolidate().inspect(|x| println!("{:?}", x));
self.variable.set(&distinct);
}
/// The collection arranged in the forward direction.
pub fn forward(&mut self) -> &Arrange<G, Node, Node, Diff> {
if self.forward.is_none() {
self.forward = Some(self.variable.arrange_by_key());
}
self.forward.as_ref().unwrap()
}
/// The collection arranged in the reverse direction.
pub fn reverse(&mut self) -> &Arrange<G, Node, Node, Diff> {
if self.reverse.is_none() {
self.reverse = Some(self.variable.map(|(x,y)| (y,x)).arrange_by_key());
}
self.reverse.as_ref().unwrap()
}
}
/// Handles to inputs and outputs of a computation.
pub struct RelationHandles<T: Timestamp+Lattice> {
/// An input handle supporting arbitrary changes.
pub input: InputSession<T, Edge, Diff>,
/// An output trace handle which can be used in other computations.
pub trace: TraceKeyHandle<Edge, T, Diff>,
}
impl Query {
/// Creates a new query from a sequence of productions.
///
/// Each production is a `String` of the form
///
/// Target Rel1 Rel2 ..
///
/// Where each Rel may begin with a '~' character, indicating the reverse direction.
/// Target must not begin with a '~'; rewrite the rule in the other order, silly.
pub fn build_from<'a>(iterator: impl IntoIterator<Item=&'a str>) -> Self {
Query { productions: iterator.into_iter().map(|text| Production::from(text)).collect() }
}
/// Creates a dataflow implementing the query, and returns input and trace handles.
pub fn render_in<G: Scope>(&self, scope: &mut G) -> IndexMap<String, RelationHandles<G::Timestamp>>
where G::Timestamp: Lattice+::timely::order::TotalOrder {
// Create new input (handle, stream) pairs
let mut input_map = IndexMap::new();
for production in self.productions.iter() {
input_map.entry(production.left_hand.clone()).or_insert_with(|| scope.new_collection());
}
// We need a subscope to allow iterative development of variables.
scope.iterative::<Iter,_,_>(|subscope| {
// create map from relation name to input handle and collection.
let mut result_map = IndexMap::new();
let mut variable_map = IndexMap::new();
// create variables and result handles for each named relation.
for (name, (input, collection)) in input_map.drain(..) {
let edge_variable = EdgeVariable::from(&collection.enter(subscope), Product::new(Default::default(), 1));
let trace = edge_variable.variable.leave().arrange_by_self().trace;
result_map.insert(name.clone(), RelationHandles { input, trace });
variable_map.insert(name.clone(), edge_variable);
}
// For each rule, add to the productions for the relation.
for production in self.productions.iter() {
let name = &production.left_hand;
let rule = &production.relations;
// We need to start somewhere; ignore empty rules.
if rule.len() > 0 {
// We'll track the path transposed, so that it is indexed by *destination* rather than source.
let mut transposed = match &rule[0] {
Relation::Forward(name) => variable_map[name].reverse().clone(),
Relation::Reverse(name) => variable_map[name].forward().clone(),
};
for relation in rule[1..].iter() {
let to_join = match relation {
Relation::Forward(name) => variable_map[name].forward(),
Relation::Reverse(name) => variable_map[name].reverse(),
};
transposed =
transposed
.join_core(to_join, |_k,&x,&y| Some((y,x)))
.arrange_by_key();
}
// Reverse the direction before adding it as a production.
variable_map[name].add_production(&transposed.as_collection(|&dst,&src| (src,dst)));
}
}
for (_name, variable) in variable_map.drain(..) {
variable.complete();
}
result_map
})
}
}
fn main() {
timely::execute_from_args(std::env::args(), move |worker| {
let timer = ::std::time::Instant::now();
let peers = worker.peers();
let index = worker.index();
let query_filename = std::env::args().nth(1).expect("Argument 1 (query filename) missing.");
let query_text = std::fs::read_to_string(query_filename).expect("Failed to read query file");
let query = Query::build_from(query_text.lines());
let mut relation_map = worker.dataflow::<(),_,_>(|scope| query.render_in(scope));
if index == 0 { println!("{:?}:\tDataflow assembled for {:?}", timer.elapsed(), query); }
// Build a dataflow to report final sizes.
worker.dataflow(|scope| {
for (name, data) in relation_map.iter_mut() {
let name = name.to_string();
data.trace
.import(scope)
.as_collection(|&_kv,&()| ())
.consolidate()
.inspect(move |x| println!("{:?}\tfinal size of relation '{}': {:?}", timer.elapsed(), name, x.2));
}
});
// snag a filename to use for the input graph.
let data_filename = std::env::args().nth(2).expect("Argument 2 (data filename) missing.");
let file = BufReader::new(File::open(data_filename).expect("Failed to read data file"));
for readline in file.lines() {
let line = readline.ok().expect("read error");
if !line.starts_with('#') && line.len() > 0 {
let mut elts = line[..].split_whitespace();
let src = elts.next().expect("data line with no src (1st) element").parse().expect("malformed src");
if (src as usize) % peers == index {
let dst = elts.next().expect("data line with no dst (2nd) element").parse().expect("malformed dst");
let val: &str = elts.next().expect("data line with no val (3rd) element");
if let Some(handle) = relation_map.get_mut(val) {
handle.input.insert((src, dst));
}
else {
panic!("couldn't find the named relation: {:?}", val);
}
}
}
}
if index == 0 { println!("{:?}:\tData loaded", timer.elapsed()); }
}).expect("Timely computation did not complete cleanly");
}

View File

@ -0,0 +1,93 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
use rand::{Rng, SeedableRng, StdRng};
use differential_dataflow::input::Input;
// use differential_dataflow::operators::Count;
use differential_dataflow::operators::count::CountTotal;
fn main() {
let mut args = std::env::args();
args.next();
let nodes: u32 = args.next().unwrap().parse().unwrap();
let edges: usize = args.next().unwrap().parse().unwrap();
let batch: u32 = args.next().unwrap().parse().unwrap();
let inspect: bool = args.next().unwrap() == "inspect";
// define a new computational scope, in which to run BFS
timely::execute_from_args(std::env::args().skip(5), move |worker| {
let timer = ::std::time::Instant::now();
let index = worker.index();
let peers = worker.peers();
// create a degree counting differential dataflow
let (mut input, probe) = worker.dataflow::<u32,_,_>(|scope| {
// create edge input, count a few ways.
let (input, edges) = scope.new_collection::<_,i32>();
let out_degr_distr =
edges
.map(|(src, _dst)| src)
.count_total()
.map(|(_src, cnt)| cnt as usize)
.count_total();
// show us something about the collection, notice when done.
let probe =
out_degr_distr
.filter(move |_| inspect)
.inspect(|x| println!("observed: {:?}", x))
.probe();
(input, probe)
});
let seed: &[_] = &[1, 2, 3, index];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for edge deletions
// Load up graph data. Round-robin among workers.
for _ in 0 .. (edges / peers) + if index < (edges % peers) { 1 } else { 0 } {
input.update((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)), 1)
}
input.advance_to(1);
input.flush();
worker.step_while(|| probe.less_than(input.time()));
if index == 0 {
println!("round 0 finished after {:?} (loading)", timer.elapsed());
}
if batch > 0 {
// Just have worker zero drive input production.
if index == 0 {
let mut next = batch;
for round in 1 .. {
input.advance_to(round);
input.update((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)), 1);
input.update((rng2.gen_range(0, nodes), rng2.gen_range(0, nodes)), -1);
if round > next {
let timer = ::std::time::Instant::now();
input.flush();
while probe.less_than(input.time()) {
worker.step();
}
println!("round {} finished after {:?}", next, timer.elapsed());
next += batch;
}
}
}
}
}).unwrap();
}

View File

@ -0,0 +1,108 @@
extern crate timely;
extern crate graph_map;
extern crate differential_dataflow;
use std::hash::Hash;
use timely::dataflow::*;
use timely::dataflow::operators::*;
use differential_dataflow::Collection;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::*;
use differential_dataflow::operators::arrange::ArrangeBySelf;
use differential_dataflow::operators::arrange::ArrangeByKey;
use graph_map::GraphMMap;
type Node = u32;
type Edge = (Node, Node);
fn main() {
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
timely::execute_from_args(std::env::args().skip(2), move |worker| {
let peers = worker.peers();
let index = worker.index();
// // What you might do if you used GraphMMap:
let graph = GraphMMap::new(&filename);
let nodes = graph.nodes();
let edges = (0..nodes).filter(move |node| node % peers == index)
.flat_map(|node| graph.edges(node).iter().cloned().map(move |dst| ((node as u32, dst))))
.map(|(src, dst)| ((src, dst), Default::default(), 1))
.collect::<Vec<_>>();
println!("loaded {} nodes, {} edges", nodes, edges.len());
worker.dataflow::<(),_,_>(|scope| {
interpret(&Collection::new(edges.to_stream(scope)), &[(0,2), (1,2)]);
});
}).unwrap();
}
fn interpret<G: Scope>(edges: &Collection<G, Edge>, relations: &[(usize, usize)]) -> Collection<G, Vec<Node>>
where G::Timestamp: Lattice+Hash+Ord {
// arrange the edge relation three ways.
let as_self = edges.arrange_by_self();
let forward = edges.arrange_by_key();
let reverse = edges.map_in_place(|x| ::std::mem::swap(&mut x.0, &mut x.1))
.arrange_by_key();
let mut field_present = ::std::collections::HashSet::new();
let mut results = edges.map(|(x,y)| vec![x, y]);
field_present.insert(0);
field_present.insert(1);
for &(src, dst) in relations.iter() {
let src_present = field_present.contains(&src);
let dst_present = field_present.contains(&dst);
results = match (src_present, dst_present) {
(true, true) => {
// Both variables are bound, so this is a semijoin.
results
.map(move |vec| ((vec[src], vec[dst]), vec))
.join_core(&as_self, |_key, vec, &()| Some(vec.clone()))
}
(true, false) => {
// Only `src` is bound, so we must use `forward` to propose `dst`.
field_present.insert(dst);
results
.map(move |vec| (vec[src], vec))
.join_core(&forward, move |_src_val, vec, &dst_val| {
let mut temp = vec.clone();
while temp.len() <= dst { temp.push(0); }
temp[dst] = dst_val;
Some(temp)
})
}
(false, true) => {
// Only `dst` is bound, so we must use `reverse` to propose `src`.
field_present.insert(src);
results
.map(move |vec| (vec[dst], vec))
.join_core(&reverse, move |_dst_val, vec, &src_val| {
let mut temp = vec.clone();
while temp.len() <= src { temp.push(0); }
temp[src] = src_val;
Some(temp)
})
}
(false, false) => {
// Neither variable is bound, which we treat as user error.
panic!("error: joining with unbound variables");
}
};
}
results
}

View File

@ -0,0 +1,184 @@
extern crate timely;
extern crate differential_dataflow;
extern crate rand;
use differential_dataflow::input::InputSession;
use differential_dataflow::operators::{Join,CountTotal,Count};
use differential_dataflow::operators::arrange::ArrangeByKey;
use differential_dataflow::operators::join::JoinCore;
use rand::{Rng, SeedableRng, StdRng};
// A differential version of item-based collaborative filtering using Jaccard similarity for
// comparing item interaction histories. See Algorithm 1 in https://ssc.io/pdf/amnesia.pdf
// for details.
fn main() {
timely::execute_from_args(std::env::args(), move |worker| {
let mut interactions_input = InputSession::new();
let probe = worker.dataflow(|scope| {
let interactions = interactions_input.to_collection(scope);
// Find all users with less than 500 interactions
let users_with_enough_interactions = interactions
.map(|(user, _item)| user)
.count_total()
.filter(move |(_user, count): &(u32, isize)| *count < 500)
.map(|(user, _count)| user);
// Remove users with too many interactions
let remaining_interactions = interactions
.semijoin(&users_with_enough_interactions);
let num_interactions_per_item = remaining_interactions
.map(|(_user, item)| item)
.count_total();
let arranged_remaining_interactions = remaining_interactions.arrange_by_key();
// Compute the number of cooccurrences of each item pair
let cooccurrences = arranged_remaining_interactions
.join_core(&arranged_remaining_interactions, |_user, &item_a, &item_b| {
if item_a > item_b { Some((item_a, item_b)) } else { None }
})
.count();
let arranged_num_interactions_per_item = num_interactions_per_item.arrange_by_key();
// Compute the jaccard similarity between item pairs (= number of users that interacted
// with both items / number of users that interacted with at least one of the items)
let jaccard_similarities = cooccurrences
// Find the number of interactions for item_a
.map(|((item_a, item_b), num_cooc)| (item_a, (item_b, num_cooc)))
.join_core(
&arranged_num_interactions_per_item,
|&item_a, &(item_b, num_cooc), &occ_a| Some((item_b, (item_a, num_cooc, occ_a)))
)
// Find the number of interactions for item_b
.join_core(
&arranged_num_interactions_per_item,
|&item_b, &(item_a, num_cooc, occ_a), &occ_b| {
Some(((item_a, item_b), (num_cooc, occ_a, occ_b)))
},
)
// Compute Jaccard similarty, has to be done in a map due to the lack of a
// total order for f64 (which seems to break the consolidation in join)
.map(|((item_a, item_b), (num_cooc, occ_a, occ_b))| {
let jaccard = num_cooc as f64 / (occ_a + occ_b - num_cooc) as f64;
((item_a, item_b), jaccard)
});
// We threshold the similarity matrix
let thresholded_similarities = jaccard_similarities
.filter(|(_item_pair, jaccard)| *jaccard > 0.05);
thresholded_similarities.probe()
});
let num_interactions: usize = std::env::args().nth(1)
.or_else(|| Some(String::from("10000"))).unwrap().parse().unwrap();
if worker.index() == 0 {
println!("Generating {} synthetic interactions...", num_interactions);
}
let seed: &[_] = &[1, 2, 3, 4];
let mut rng: StdRng = SeedableRng::from_seed(seed);
let synthetic_interactions = generate_interactions(num_interactions, &mut rng);
let num_users = synthetic_interactions.iter().map(|(user, _item)| user).max().unwrap() + 1;
let users_to_remove = rand::seq::sample_iter(&mut rng, 0..num_users, 20).unwrap();
let interactions_to_remove: Vec<(u32,u32)> = synthetic_interactions.iter()
.filter(|(user, _item)| users_to_remove.contains(&user))
.map(|(user, item)| (*user, *item))
.collect();
let timer = worker.timer();
for (user, item) in synthetic_interactions.iter() {
if *user as usize % worker.peers() == worker.index() {
interactions_input.insert((*user, *item));
}
}
interactions_input.advance_to(1);
interactions_input.flush();
worker.step_while(|| probe.less_than(interactions_input.time()));
let initial_model_time = timer.elapsed();
println!("Model trained in {:?} on worker {:?}", initial_model_time, worker.index());
if worker.index() == 0 {
println!("Removing {} interactions...", interactions_to_remove.len());
}
for (user, item) in interactions_to_remove.iter() {
if *user as usize % worker.peers() == worker.index() {
interactions_input.remove((*user, *item));
}
}
interactions_input.advance_to(2);
interactions_input.flush();
worker.step_while(|| probe.less_than(interactions_input.time()));
let removal_time = timer.elapsed();
println!(
"Model updated after {:?} on worker {:?}",
(removal_time - initial_model_time),
worker.index(),
);
}).unwrap();
}
struct CRP { alpha: f64, discount: f64, weight: f64, weights: Vec<f64> }
impl CRP {
fn new(alpha: f64, discount: f64) -> Self {
CRP { alpha, discount, weight: 0.0, weights: Vec::new() }
}
fn sample<R>(&mut self, rng: &mut R) -> u32 where R: Rng {
let mut u = rng.gen::<f64>() * (self.alpha + self.weight);
for j in 0 .. self.weights.len() {
if u < self.weights[j] - self.discount {
self.weights[j] += 1.0;
self.weight += 1.0;
return j as u32;
} else {
u = u - self.weights[j] - self.discount;
}
}
self.weights.push(1.0);
self.weight = self.weight + 1.0;
(self.weights.len() - 1) as u32
}
}
// Generate synthetic interactions with a skewed distribution
fn generate_interactions<R>(how_many: usize, rng: &mut R) -> Vec<(u32,u32)> where R: Rng {
let mut interactions = Vec::with_capacity(how_many);
let mut user_sampler = CRP::new(6000.0, 0.35);
let mut item_sampler = CRP::new(6000.0, 0.35);
for _ in 0 .. how_many {
let user = user_sampler.sample(rng);
let item = item_sampler.sample(rng);
interactions.push((user, item));
}
interactions
}

View File

@ -0,0 +1,168 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
#[macro_use]
extern crate abomonation_derive;
extern crate abomonation;
#[macro_use]
extern crate serde_derive;
extern crate serde;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::*;
use timely::dataflow::operators::probe::Handle;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::lattice::Lattice;
type Node = u32;
type Edge = (Node, Node);
#[derive(Abomonation, Copy, Ord, PartialOrd, Eq, PartialEq, Debug, Clone, Serialize, Deserialize, Hash)]
pub struct MinSum {
value: u32,
}
use differential_dataflow::difference::{Semigroup, Multiply};
impl Semigroup for MinSum {
fn plus_equals(&mut self, rhs: &Self) {
self.value = std::cmp::min(self.value, rhs.value);
}
fn is_zero(&self) -> bool { false }
}
impl Multiply<Self> for MinSum {
type Output = Self;
fn multiply(self, rhs: &Self) -> Self {
MinSum { value: self.value + rhs.value }
}
}
fn main() {
let nodes: u32 = std::env::args().nth(1).unwrap().parse().unwrap();
let edges: u32 = std::env::args().nth(2).unwrap().parse().unwrap();
let weight: u32 = std::env::args().nth(3).unwrap().parse().unwrap();
let batch: u32 = std::env::args().nth(4).unwrap().parse().unwrap();
let rounds: u32 = std::env::args().nth(5).unwrap().parse().unwrap();
let inspect: bool = std::env::args().nth(6).unwrap() == "inspect";
// define a new computational scope, in which to run BFS
timely::execute_from_args(std::env::args(), move |worker| {
let timer = ::std::time::Instant::now();
// define BFS dataflow; return handles to roots and edges inputs
let mut probe = Handle::new();
let (mut roots, mut graph) = worker.dataflow(|scope| {
let (root_input, roots) = scope.new_collection();
let (edge_input, graph) = scope.new_collection();
let mut result = bfs(&graph, &roots);
if !inspect {
result = result.filter(|_| false);
}
result.count()
.map(|(_,l)| l)
.consolidate()
.inspect(|x| println!("\t{:?}", x))
.probe_with(&mut probe);
(root_input, edge_input)
});
let seed: &[_] = &[1, 2, 3, 4];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
roots.update_at(0, Default::default(), MinSum { value: 0 });
roots.close();
println!("performing BFS on {} nodes, {} edges:", nodes, edges);
if worker.index() == 0 {
for _ in 0 .. edges {
graph.update_at(
(rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)),
Default::default(),
MinSum { value: rng1.gen_range(0, weight) },
);
}
}
println!("{:?}\tloaded", timer.elapsed());
graph.advance_to(1);
graph.flush();
worker.step_while(|| probe.less_than(graph.time()));
println!("{:?}\tstable", timer.elapsed());
for round in 0 .. rounds {
for element in 0 .. batch {
if worker.index() == 0 {
graph.update_at(
(rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)),
1 + round * batch + element,
MinSum { value: rng1.gen_range(0, weight) },
);
}
graph.advance_to(2 + round * batch + element);
}
graph.flush();
let timer2 = ::std::time::Instant::now();
worker.step_while(|| probe.less_than(&graph.time()));
if worker.index() == 0 {
let elapsed = timer2.elapsed();
println!("{:?}\t{:?}:\t{}", timer.elapsed(), round, elapsed.as_secs() * 1000000000 + (elapsed.subsec_nanos() as u64));
}
}
println!("finished; elapsed: {:?}", timer.elapsed());
}).unwrap();
}
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn bfs<G: Scope>(edges: &Collection<G, Edge, MinSum>, roots: &Collection<G, Node, MinSum>) -> Collection<G, Node, MinSum>
where G::Timestamp: Lattice+Ord {
// repeatedly update minimal distances each node can be reached from each root
roots.scope().iterative::<u32,_,_>(|scope| {
use differential_dataflow::operators::iterate::SemigroupVariable;
use differential_dataflow::operators::reduce::ReduceCore;
use differential_dataflow::trace::implementations::ord::OrdKeySpine as DefaultKeyTrace;
use timely::order::Product;
let variable = SemigroupVariable::new(scope, Product::new(Default::default(), 1));
let edges = edges.enter(scope);
let roots = roots.enter(scope);
let result =
variable
.map(|n| (n,()))
.join_map(&edges, |_k,&(),d| *d)
.concat(&roots)
.map(|x| (x,()))
.reduce_core::<_,DefaultKeyTrace<_,_,_>>("Reduce", |_key, input, output, updates| {
if output.is_empty() || input[0].1 < output[0].1 {
updates.push(((), input[0].1));
}
})
.as_collection(|k,()| *k);
variable.set(&result);
result.leave()
})
}

View File

@ -0,0 +1,319 @@
#[macro_use]
extern crate abomonation_derive;
extern crate abomonation;
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
use std::io::BufRead;
use timely::dataflow::ProbeHandle;
use timely::dataflow::operators::unordered_input::UnorderedInput;
use timely::dataflow::operators::Probe;
use timely::progress::frontier::AntichainRef;
use timely::PartialOrder;
use differential_dataflow::AsCollection;
use differential_dataflow::operators::{Count, arrange::ArrangeBySelf};
use differential_dataflow::trace::{Cursor, TraceReader};
use pair::Pair;
fn main() {
timely::execute_from_args(std::env::args(), move |worker| {
// Used to determine if our output has caught up to our input.
let mut probe: ProbeHandle<Pair<isize, isize>> = ProbeHandle::new();
let (mut input, mut capability, mut trace) =
worker.dataflow(|scope| {
// Create "unordered" inputs which provide their capabilities to users.
// Here "capability" is a technical term, which is "permission to send
// data or after a certain timestamp". When this capability is dropped
// or downgraded, the input communicates that its possible timestamps
// have advanced, and the system can start to make progress.
let ((input, capability), data) = scope.new_unordered_input();
let arrangement =
data.as_collection()
.count()
.map(|(_value, count)| count)
.arrange_by_self();
arrangement.stream.probe_with(&mut probe);
(input, capability, arrangement.trace)
});
// Do not hold back physical compaction.
trace.set_physical_compaction(AntichainRef::new(&[]));
println!("Multi-temporal histogram; valid commands are (integer arguments):");
println!(" update value time1 time2 change");
println!(" advance-input time1 time2");
println!(" advance-output time1 time2");
println!(" query time1 time2");
let std_input = std::io::stdin();
for line in std_input.lock().lines().map(|x| x.unwrap()) {
let mut elts = line[..].split_whitespace();
if let Some(command) = elts.next() {
if let Ok(arguments) = read_integers(elts) {
match (command, arguments.len()) {
("update", 4) => {
let time = Pair::new(arguments[1], arguments[2]);
if capability.time().less_equal(&time) {
input
.session(capability.clone())
.give((arguments[0], time, arguments[3]));
} else {
println!("Requested time {:?} no longer open (input from {:?})", time, capability.time());
}
},
("advance-input", 2) => {
let time = Pair::new(arguments[0], arguments[1]);
if capability.time().less_equal(&time) {
capability.downgrade(&time);
while probe.less_than(capability.time()) {
worker.step();
}
} else {
println!("Requested time {:?} no longer open (input from {:?})", time, capability.time());
}
},
("advance-output", 2) => {
let time = Pair::new(arguments[0], arguments[1]);
if trace.get_logical_compaction().less_equal(&time) {
trace.set_logical_compaction(AntichainRef::new(&[time]));
while probe.less_than(capability.time()) {
worker.step();
}
} else {
println!("Requested time {:?} not readable (output from {:?})", time, trace.get_logical_compaction());
}
},
("query", 2) => {
// Check that the query times are not beyond the current capabilities.
let query_time = Pair::new(arguments[0], arguments[1]);
if capability.time().less_equal(&query_time) {
println!("Query time ({:?}) is still open (input from {:?}).", query_time, capability.time());
} else if !trace.get_logical_compaction().less_equal(&query_time) {
println!("Query time ({:?}) no longer available in output (output from {:?}).", query_time, trace.get_logical_compaction());
}
else {
println!("Report at {:?}", query_time);
// enumerate the contents of `trace` at `query_time`.
let (mut cursor, storage) = trace.cursor();
while let Some(key) = cursor.get_key(&storage) {
while let Some(_val) = cursor.get_val(&storage) {
let mut sum = 0;
cursor.map_times(&storage,
|time, diff| if time.less_equal(&query_time) { sum += diff; }
);
cursor.step_val(&storage);
if sum != 0 {
println!(" values with occurrence count {:?}: {:?}", key, sum);
}
}
cursor.step_key(&storage);
}
println!("Report complete");
}
},
_ => {
println!("Command not recognized: {:?} with {} arguments.", command, arguments.len());
}
}
}
else {
println!("Error parsing command arguments");
}
}
}
}).unwrap();
}
/// This module contains a definition of a new timestamp time, a "pair" or product.
///
/// This is a minimal self-contained implementation, in that it doesn't borrow anything
/// from the rest of the library other than the traits it needs to implement. With this
/// type and its implementations, you can use it as a timestamp type.
mod pair {
/// A pair of timestamps, partially ordered by the product order.
#[derive(Hash, Default, Clone, Eq, PartialEq, Ord, PartialOrd, Abomonation)]
pub struct Pair<S, T> {
pub first: S,
pub second: T,
}
impl<S, T> Pair<S, T> {
/// Create a new pair.
pub fn new(first: S, second: T) -> Self {
Pair { first, second }
}
}
// Implement timely dataflow's `PartialOrder` trait.
use timely::order::PartialOrder;
impl<S: PartialOrder, T: PartialOrder> PartialOrder for Pair<S, T> {
fn less_equal(&self, other: &Self) -> bool {
self.first.less_equal(&other.first) && self.second.less_equal(&other.second)
}
}
use timely::progress::timestamp::Refines;
impl<S: Timestamp, T: Timestamp> Refines<()> for Pair<S, T> {
fn to_inner(_outer: ()) -> Self { Self::minimum() }
fn to_outer(self) -> () { () }
fn summarize(_summary: <Self>::Summary) -> () { () }
}
// Implement timely dataflow's `PathSummary` trait.
// This is preparation for the `Timestamp` implementation below.
use timely::progress::PathSummary;
impl<S: Timestamp, T: Timestamp> PathSummary<Pair<S,T>> for () {
fn results_in(&self, timestamp: &Pair<S, T>) -> Option<Pair<S,T>> {
Some(timestamp.clone())
}
fn followed_by(&self, other: &Self) -> Option<Self> {
Some(other.clone())
}
}
// Implement timely dataflow's `Timestamp` trait.
use timely::progress::Timestamp;
impl<S: Timestamp, T: Timestamp> Timestamp for Pair<S, T> {
fn minimum() -> Self { Pair { first: S::minimum(), second: T::minimum() }}
type Summary = ();
}
// Implement differential dataflow's `Lattice` trait.
// This extends the `PartialOrder` implementation with additional structure.
use differential_dataflow::lattice::Lattice;
impl<S: Lattice, T: Lattice> Lattice for Pair<S, T> {
fn join(&self, other: &Self) -> Self {
Pair {
first: self.first.join(&other.first),
second: self.second.join(&other.second),
}
}
fn meet(&self, other: &Self) -> Self {
Pair {
first: self.first.meet(&other.first),
second: self.second.meet(&other.second),
}
}
}
use std::fmt::{Formatter, Error, Debug};
/// Debug implementation to avoid seeing fully qualified path names.
impl<TOuter: Debug, TInner: Debug> Debug for Pair<TOuter, TInner> {
fn fmt(&self, f: &mut Formatter) -> Result<(), Error> {
f.write_str(&format!("({:?}, {:?})", self.first, self.second))
}
}
}
/// This module contains a definition of a new timestamp time, a "pair" or product.
///
/// This is a minimal self-contained implementation, in that it doesn't borrow anything
/// from the rest of the library other than the traits it needs to implement. With this
/// type and its implementations, you can use it as a timestamp type.
mod vector {
/// A pair of timestamps, partially ordered by the product order.
#[derive(Hash, Default, Clone, Eq, PartialEq, Ord, PartialOrd, Abomonation, Debug)]
pub struct Vector<T> {
pub vector: Vec<T>,
}
impl<T> Vector<T> {
/// Create a new pair.
pub fn new(vector: Vec<T>) -> Self {
Vector { vector }
}
}
// Implement timely dataflow's `PartialOrder` trait.
use timely::order::PartialOrder;
impl<T: PartialOrder+Timestamp> PartialOrder for Vector<T> {
fn less_equal(&self, other: &Self) -> bool {
self.vector
.iter()
.enumerate()
.all(|(index, time)| time.less_equal(other.vector.get(index).unwrap_or(&T::minimum())))
}
}
use timely::progress::timestamp::Refines;
impl<T: Timestamp> Refines<()> for Vector<T> {
fn to_inner(_outer: ()) -> Self { Self { vector: Vec::new() } }
fn to_outer(self) -> () { () }
fn summarize(_summary: <Self>::Summary) -> () { () }
}
// Implement timely dataflow's `PathSummary` trait.
// This is preparation for the `Timestamp` implementation below.
use timely::progress::PathSummary;
impl<T: Timestamp> PathSummary<Vector<T>> for () {
fn results_in(&self, timestamp: &Vector<T>) -> Option<Vector<T>> {
Some(timestamp.clone())
}
fn followed_by(&self, other: &Self) -> Option<Self> {
Some(other.clone())
}
}
// Implement timely dataflow's `Timestamp` trait.
use timely::progress::Timestamp;
impl<T: Timestamp> Timestamp for Vector<T> {
fn minimum() -> Self { Self { vector: Vec::new() } }
type Summary = ();
}
// Implement differential dataflow's `Lattice` trait.
// This extends the `PartialOrder` implementation with additional structure.
use differential_dataflow::lattice::Lattice;
impl<T: Lattice+Timestamp+Clone> Lattice for Vector<T> {
fn join(&self, other: &Self) -> Self {
let min_len = ::std::cmp::min(self.vector.len(), other.vector.len());
let max_len = ::std::cmp::max(self.vector.len(), other.vector.len());
let mut vector = Vec::with_capacity(max_len);
for index in 0 .. min_len {
vector.push(self.vector[index].join(&other.vector[index]));
}
for time in &self.vector[min_len..] {
vector.push(time.clone());
}
for time in &other.vector[min_len..] {
vector.push(time.clone());
}
Self { vector }
}
fn meet(&self, other: &Self) -> Self {
let min_len = ::std::cmp::min(self.vector.len(), other.vector.len());
let mut vector = Vec::with_capacity(min_len);
for index in 0 .. min_len {
vector.push(self.vector[index].meet(&other.vector[index]));
}
Self { vector }
}
}
}
/// Read a command and its arguments.
fn read_integers<'a>(input: impl Iterator<Item=&'a str>) -> Result<Vec<isize>, std::num::ParseIntError> {
let mut integers = Vec::new();
for text in input {
integers.push(text.parse()?);
}
Ok(integers)
}

View File

@ -0,0 +1,138 @@
extern crate timely;
extern crate graph_map;
extern crate differential_dataflow;
use timely::order::Product;
use timely::dataflow::{*, operators::Filter};
use differential_dataflow::Collection;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::{*, iterate::Variable};
use differential_dataflow::input::InputSession;
use differential_dataflow::AsCollection;
use graph_map::GraphMMap;
type Node = u32;
type Edge = (Node, Node);
type Time = u32;
type Iter = u32;
type Diff = isize;
fn main() {
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
let iterations: Iter = std::env::args().nth(2).unwrap().parse().unwrap();
let inspect = std::env::args().nth(3) == Some("inspect".to_string());
timely::execute_from_args(std::env::args().skip(2), move |worker| {
let peers = worker.peers();
let index = worker.index();
let timer = worker.timer();
let mut input = InputSession::new();
let mut probe = ProbeHandle::new();
worker.dataflow::<Time,_,_>(|scope| {
let edges = input.to_collection(scope);
pagerank(iterations, &edges)
.filter(move |_| inspect)
.consolidate()
.inspect(|x| println!("{:?}", x))
.probe_with(&mut probe);
});
// // What you might do if you used GraphMMap:
let graph = GraphMMap::new(&filename);
let mut node = index;
while node < graph.nodes() {
for &edge in graph.edges(node) {
input.update((node as Node, edge as Node), 1);
}
node += peers;
}
input.advance_to(1);
input.flush();
while probe.less_than(input.time()) {
worker.step();
}
println!("{:?}\tinitial compute complete", timer.elapsed());
for node in 1 .. graph.nodes() {
if node % peers == index {
if !graph.edges(node).is_empty() {
input.update((node as Node, graph.edges(node)[0] as Node), -1);
}
}
input.advance_to((node + 1) as Time);
input.flush();
while probe.less_than(input.time()) {
worker.step();
}
println!("{:?}\tround {} complete", timer.elapsed(), node);
}
}).unwrap();
}
// Returns a weighted collection in which the weight of each node is proportional
// to its PageRank in the input graph `edges`.
fn pagerank<G>(iters: Iter, edges: &Collection<G, Edge, Diff>) -> Collection<G, Node, Diff>
where
G: Scope,
G::Timestamp: Lattice,
{
// initialize many surfers at each node.
let nodes =
edges.flat_map(|(x,y)| Some(x).into_iter().chain(Some(y)))
.distinct();
// snag out-degrees for each node.
let degrs = edges.map(|(src,_dst)| src)
.count();
edges.scope().iterative::<Iter,_,_>(|inner| {
// Bring various collections into the scope.
let edges = edges.enter(inner);
let nodes = nodes.enter(inner);
let degrs = degrs.enter(inner);
// Initial and reset numbers of surfers at each node.
let inits = nodes.explode(|node| Some((node, 6_000_000)));
let reset = nodes.explode(|node| Some((node, 1_000_000)));
// Define a recursive variable to track surfers.
// We start from `inits` and cycle only `iters`.
let ranks = Variable::new_from(inits, Product::new(Default::default(), 1));
// Match each surfer with the degree, scale numbers down.
let to_push =
degrs.semijoin(&ranks)
.threshold(|(_node, degr), rank| (5 * rank) / (6 * degr))
.map(|(node, _degr)| node);
// Propagate surfers along links, blend in reset surfers.
let mut pushed =
edges.semijoin(&to_push)
.map(|(_node, dest)| dest)
.concat(&reset)
.consolidate();
if iters > 0 {
pushed =
pushed
.inner
.filter(move |(_d,t,_r)| t.inner < iters)
.as_collection();
}
// Bind the recursive variable, return its limit.
ranks.set(&pushed);
pushed.leave()
})
}

View File

@ -0,0 +1,229 @@
//! A demonstration of timely dataflow progress tracking, using differential dataflow operators.
extern crate timely;
extern crate differential_dataflow;
use timely::PartialOrder;
use timely::dataflow::*;
use timely::dataflow::operators::probe::Handle;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::lattice::Lattice;
use timely::progress::{Timestamp, Source, Target, Location};
use timely::progress::timestamp::PathSummary;
fn main() {
timely::execute_from_args(std::env::args(), move |worker| {
let timer = worker.timer();
let mut probe = Handle::new();
let (mut nodes, mut edges, mut times) = worker.dataflow::<usize,_,_>(|scope| {
let (node_input, nodes) = scope.new_collection();
let (edge_input, edges) = scope.new_collection();
let (time_input, times) = scope.new_collection();
// Detect cycles that do not increment timestamps.
find_cycles::<_,usize>(nodes.clone(), edges.clone())
.inspect(move |x| println!("{:?}\tcycles: {:?}", timer.elapsed(), x))
.probe_with(&mut probe);
// Summarize all paths to inputs of operator zero.
summarize::<_,usize>(nodes.clone(), edges.clone())
.inspect(move |x| println!("{:?}\tsummary: {:?}", timer.elapsed(), x))
.probe_with(&mut probe);
// Track the frontier at each dataflow location.
frontier::<_,usize>(nodes, edges, times)
.inspect(move |x| println!("{:?}\tfrontier: {:?}", timer.elapsed(), x))
.probe_with(&mut probe);
(node_input, edge_input, time_input)
});
// A PageRank-like graph, as represented here:
// https://github.com/TimelyDataflow/diagnostics/blob/master/examples/pagerank.png
nodes.insert((Target::new(2, 0), Source::new(2, 0), 1));
nodes.insert((Target::new(3, 0), Source::new(3, 0), 0));
nodes.insert((Target::new(3, 1), Source::new(3, 0), 0));
nodes.insert((Target::new(4, 0), Source::new(4, 0), 0));
edges.insert((Source::new(1, 0), Target::new(3, 0)));
edges.insert((Source::new(3, 0), Target::new(4, 0)));
edges.insert((Source::new(4, 0), Target::new(2, 0)));
edges.insert((Source::new(2, 0), Target::new(3, 1)));
// Initially no capabilities.
nodes.advance_to(1); nodes.flush();
edges.advance_to(1); edges.flush();
times.advance_to(1); times.flush();
while probe.less_than(times.time()) {
worker.step();
}
// Introduce a new input capability at time zero.
times.insert((Location::new_source(1, 0), 0));
nodes.advance_to(2); nodes.flush();
edges.advance_to(2); edges.flush();
times.advance_to(2); times.flush();
while probe.less_than(times.time()) {
worker.step();
}
// Remove input capability and produce a message.
times.remove((Location::new_source(1, 0), 0));
times.insert((Location::new_target(3, 0), 0));
nodes.advance_to(3); nodes.flush();
edges.advance_to(3); edges.flush();
times.advance_to(3); times.flush();
while probe.less_than(times.time()) {
worker.step();
}
// Consume the message, and .. do nothing, I guess.
times.remove((Location::new_target(3, 0), 0));
nodes.advance_to(4); nodes.flush();
edges.advance_to(4); edges.flush();
times.advance_to(4); times.flush();
while probe.less_than(times.time()) {
worker.step();
}
println!("finished; elapsed: {:?}", timer.elapsed());
}).unwrap();
}
/// Propagates times along a timely dataflow graph.
///
/// Timely dataflow graphs are described by nodes with interconnected input and output ports,
/// and edges which connect output ports to input ports of what may be other nodes.
///
/// A set of times at various locations (input or output ports) could traverse nodes and
/// edges to arrive at various other locations. Each location can then track minimal times
/// that can reach them: those times not greater than some other time that can reach it.
///
/// The computation to determine this, and to maintain it as times change, is an iterative
/// computation that propagates times and maintains the minimal elements at each location.
fn frontier<G: Scope, T: Timestamp>(
nodes: Collection<G, (Target, Source, T::Summary)>,
edges: Collection<G, (Source, Target)>,
times: Collection<G, (Location, T)>,
) -> Collection<G, (Location, T)>
where
G::Timestamp: Lattice+Ord,
T::Summary: differential_dataflow::ExchangeData,
{
// Translate node and edge transitions into a common Location to Location edge with an associated Summary.
let nodes = nodes.map(|(target, source, summary)| (Location::from(target), (Location::from(source), summary)));
let edges = edges.map(|(source, target)| (Location::from(source), (Location::from(target), Default::default())));
let transitions: Collection<G, (Location, (Location, T::Summary))> = nodes.concat(&edges);
times
.iterate(|reach| {
transitions
.enter(&reach.scope())
.join_map(&reach, |_from, (dest, summ), time| (dest.clone(), summ.results_in(time)))
.flat_map(|(dest, time)| time.map(move |time| (dest, time)))
.concat(&times.enter(&reach.scope()))
.reduce(|_location, input, output: &mut Vec<(T, isize)>| {
// retain the lower envelope of times.
for (t1, _count1) in input.iter() {
if !input.iter().any(|(t2, _count2)| t2.less_than(t1)) {
output.push(((*t1).clone(), 1));
}
}
})
})
.consolidate()
}
/// Summary paths from locations to operator zero inputs.
fn summarize<G: Scope, T: Timestamp>(
nodes: Collection<G, (Target, Source, T::Summary)>,
edges: Collection<G, (Source, Target)>,
) -> Collection<G, (Location, (Location, T::Summary))>
where
G::Timestamp: Lattice+Ord,
T::Summary: differential_dataflow::ExchangeData+std::hash::Hash,
{
// Start from trivial reachability from each input to itself.
let zero_inputs =
edges
.map(|(_source, target)| Location::from(target))
.filter(|location| location.node == 0)
.map(|location| (location, (location, Default::default())));
// Retain node connections along "default" timestamp summaries.
let nodes = nodes.map(|(target, source, summary)| (Location::from(source), (Location::from(target), summary)));
let edges = edges.map(|(source, target)| (Location::from(target), (Location::from(source), Default::default())));
let transitions: Collection<G, (Location, (Location, T::Summary))> = nodes.concat(&edges);
zero_inputs
.iterate(|summaries| {
transitions
.enter(&summaries.scope())
.join_map(summaries, |_middle, (from, summ1), (to, summ2)| (from.clone(), to.clone(), summ1.followed_by(summ2)))
.flat_map(|(from, to, summ)| summ.map(move |summ| (from, (to, summ))))
.concat(&zero_inputs.enter(&summaries.scope()))
.map(|(from, (to, summary))| ((from, to), summary))
.reduce(|_from_to, input, output| {
for (summary, _count) in input.iter() {
if !input.iter().any(|(sum2, _count2)| sum2.less_than(*summary)) {
output.push(((*summary).clone(), 1));
}
}
})
.map(|((from, to), summary)| (from, (to, summary)))
})
.consolidate()
}
/// Identifies cycles along paths that do not increment timestamps.
fn find_cycles<G: Scope, T: Timestamp>(
nodes: Collection<G, (Target, Source, T::Summary)>,
edges: Collection<G, (Source, Target)>,
) -> Collection<G, (Location, Location)>
where
G::Timestamp: Lattice+Ord,
T::Summary: differential_dataflow::ExchangeData,
{
// Retain node connections along "default" timestamp summaries.
let nodes = nodes.flat_map(|(target, source, summary)| {
if summary != Default::default() {
Some((Location::from(target), Location::from(source)))
}
else {
None
}
});
let edges = edges.map(|(source, target)| (Location::from(source), Location::from(target)));
let transitions: Collection<G, (Location, Location)> = nodes.concat(&edges);
// Repeatedly restrict to locations with an incoming path.
transitions
.iterate(|locations| {
let active =
locations
.map(|(_source, target)| target)
.distinct();
transitions
.enter(&locations.scope())
.semijoin(&active)
})
.consolidate()
}

View File

@ -0,0 +1,177 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
use timely::dataflow::operators::probe::Handle;
use differential_dataflow::input::InputSession;
use differential_dataflow::operators::*;
fn main() {
// define a new computational scope, in which to run BFS
timely::execute_from_args(std::env::args(), move |worker| {
// An input for (x,y,z) placements.
let mut xyzs = InputSession::<_,_,isize>::new();
// Inputs for (x,y) and (x,z) goals.
let mut xy_goal = InputSession::new();
let mut xz_goal = InputSession::new();
let mut probe = Handle::new();
// Dataflow to validate input against goals.
worker.dataflow(|scope| {
// Introduce inputs to the scope.
let xyzs = xyzs.to_collection(scope);
let xy_goal = xy_goal.to_collection(scope);
let xz_goal = xz_goal.to_collection(scope);
// Report unmet XY goals, and met XY non-goals.
let xy_errors =
xyzs.map(|(x,y,_)| (x,y))
.distinct()
.negate()
.concat(&xy_goal)
.consolidate();
// Report unmet XZ goals, and met XZ non-goals.
let xz_errors =
xyzs.map(|(x,_,z)| (x,z))
.distinct()
.negate()
.concat(&xz_goal)
.consolidate();
let xy_total = xy_errors.distinct().map(|_| ());
let xz_total = xz_errors.distinct().map(|_| ());
xy_total
.concat(&xz_total)
.distinct()
.inspect(|x| println!("Not done: {:?}", x))
.probe_with(&mut probe);
});
// Dataflow to produce maximum inputs.
worker.dataflow(|scope| {
// Introduce goals to the scope.
let xy_goal = xy_goal.to_collection(scope);
let xz_goal = xz_goal.to_collection(scope);
let xy_xs = xy_goal.map(|(x,_)| (x,()));
let xz_xs = xz_goal.map(|(x,_)| (x,()));
xy_xs.join(&xz_xs)
.map(|_| ())
.consolidate()
.inspect(|x| println!("Maximum solution size: {}", x.2))
.probe_with(&mut probe);
// // For each x, produce valid pairs of y and z.
// xy_goal
// .join(&xz_goal)
// .map(|(x,(y,z))| (x,y,z))
// .inspect(|x| println!("Maximum solution: {:?}", x))
// .probe_with(&mut probe);
});
// Dataflow to produce minimum inputs.
worker.dataflow(|scope| {
// Introduce goals to the scope.
let xy_goal = xy_goal.to_collection(scope);
let xz_goal = xz_goal.to_collection(scope);
let xy_xs = xy_goal.map(|(x,_)| x).count();
let xz_xs = xz_goal.map(|(x,_)| x).count();
xy_xs.join(&xz_xs)
.explode(|(_,(ys,zs))| Some(((), ::std::cmp::max(ys,zs))))
.consolidate()
.inspect(|x| println!("Minimum solution size: {}", x.2))
.probe_with(&mut probe);
// // Produce pairs (x, ys) and (x, zs).
// let xy_xs = xy_goal.group(|_x,ys,out|
// out.push((ys.iter().map(|(&y,_)| y).collect::<Vec<_>>(), 1))
// );
// let xz_xs = xz_goal.group(|_x,zs,out|
// out.push((zs.iter().map(|(&z,_)| z).collect::<Vec<_>>(), 1))
// );
// xy_xs.join(&xz_xs)
// .flat_map(|(x,(ys, zs))| {
// let max = ::std::cmp::max(ys.len(), zs.len());
// let ys = ys.into_iter().cycle();
// let zs = zs.into_iter().cycle();
// ys.zip(zs).take(max).map(move |(y,z)| (x,y,z))
// })
// .inspect(|x| println!("Minimum solution: {:?}", x))
// .probe_with(&mut probe);
});
// Introduce XY projektion.
xy_goal.insert((0, 0));
xy_goal.insert((0, 1));
xy_goal.insert((0, 3));
xy_goal.insert((0, 4));
xy_goal.insert((1, 1));
xy_goal.insert((1, 3));
xy_goal.insert((2, 1));
xy_goal.insert((2, 2));
xy_goal.insert((3, 2));
xy_goal.insert((3, 3));
xy_goal.insert((3, 4));
xy_goal.insert((4, 0));
xy_goal.insert((4, 1));
xy_goal.insert((4, 2));
// Introduce XZ projektion.
xz_goal.insert((0, 2));
xz_goal.insert((0, 3));
xz_goal.insert((0, 4));
xz_goal.insert((1, 2));
xz_goal.insert((1, 4));
xz_goal.insert((2, 1));
xz_goal.insert((2, 2));
xz_goal.insert((2, 3));
xz_goal.insert((3, 0));
xz_goal.insert((3, 1));
xz_goal.insert((3, 3));
xz_goal.insert((3, 4));
xz_goal.insert((4, 1));
xz_goal.insert((4, 4));
// Advance one round.
xyzs.advance_to(1); xyzs.flush();
xy_goal.advance_to(1); xy_goal.flush();
xz_goal.advance_to(1); xz_goal.flush();
// Introduce candidate solution.
xyzs.insert((0, 0, 2));
xyzs.insert((0, 1, 3));
xyzs.insert((0, 3, 4));
xyzs.insert((0, 4, 4));
xyzs.insert((1, 1, 2));
xyzs.insert((1, 3, 4));
xyzs.insert((2, 1, 1));
xyzs.insert((2, 2, 2));
xyzs.insert((2, 2, 3));
xyzs.insert((3, 2, 0));
xyzs.insert((3, 3, 1));
xyzs.insert((3, 4, 3));
xyzs.insert((3, 4, 4));
xyzs.insert((4, 0, 1));
xyzs.insert((4, 1, 4));
xyzs.insert((4, 2, 4));
// Advance another round.
xyzs.advance_to(2); xyzs.flush();
xy_goal.advance_to(2); xy_goal.flush();
xz_goal.advance_to(2); xz_goal.flush();
}).unwrap();
}

View File

@ -0,0 +1,127 @@
extern crate timely;
extern crate differential_dataflow;
use std::io::{BufRead, BufReader};
use std::fs::File;
use timely::dataflow::*;
use timely::dataflow::operators::probe::Handle;
use differential_dataflow::input::InputSession;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::lattice::Lattice;
type Time = usize;
type Node = u32;
type Edge = (Node, Node);
fn main() {
let mut args = std::env::args().skip(1);
let filename = args.next().expect("must specify a filename");
let root: Node = args.next().expect("must specify root node").parse().expect("root node must be an integer");
let batch: usize = args.next().expect("must specify batching").parse().expect("batch must be an integer");
let compression: Time = args.next().expect("must specify compression").parse().expect("compression must be an integer");
let inspect: bool = args.next().expect("must specify inspect bit").parse().expect("inspect must be boolean");
// define a new computational scope, in which to run BFS
timely::execute_from_args(args, move |worker| {
let timer = ::std::time::Instant::now();
let index = worker.index();
let peers = worker.peers();
// define BFS dataflow; return handles to roots and edges inputs
let mut roots = InputSession::new();
let mut graph = InputSession::new();
let mut probe = Handle::new();
worker.dataflow(|scope| {
let roots = roots.to_collection(scope);
let graph = graph.to_collection(scope);
bfs(&graph, &roots)
.filter(move |_| inspect)
.map(|(_,l)| l)
.consolidate()
.inspect(|x| println!("\t{:?}", x))
.probe_with(&mut probe);
});
// Load the (fraction of the) data!
let mut edges = Vec::new();
let file = BufReader::new(File::open(filename.clone()).unwrap());
for (count, readline) in file.lines().enumerate() {
if count % peers == index {
let line = readline.ok().expect("read error");
if !line.starts_with('#') {
let mut elts = line[..].split_whitespace();
let src: Node = elts.next().expect("line missing src field").parse().expect("malformed src");
let dst: Node = elts.next().expect("line missing dst field").parse().expect("malformed dst");
let sec: Time = elts.next().expect("line missing sec field").parse().expect("malformed sec");
edges.push((compression * ((sec / compression) + 1), src, dst));
}
}
}
// Could leave this open, continually update/advance.
roots.insert(root);
roots.close();
println!("{:?}\tData loaded.", timer.elapsed());
// Sort edges by second.
edges.sort();
println!("{:?}\tData sorted.", timer.elapsed());
// Repeatedly introduce `batch` many records, then await their completion.
let mut slice = &edges[..];
while !slice.is_empty() {
// Catch up to this point.
graph.advance_to(slice[0].0);
graph.flush();
while probe.less_than(graph.time()) {
worker.step();
}
println!("{:?}\tTime {:?} reached", timer.elapsed(), graph.time());
let limit = std::cmp::min(slice.len(), batch);
for &(time, src, dst) in slice[..limit].iter() {
graph.advance_to(time);
graph.insert((src, dst));
}
slice = &slice[limit..];
}
graph.close();
while worker.step() { }
println!("{:?}\tComputation complete.", timer.elapsed());
}).unwrap();
}
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn bfs<G: Scope>(edges: &Collection<G, Edge>, roots: &Collection<G, Node>) -> Collection<G, (Node, u32)>
where G::Timestamp: Lattice+Ord {
// initialize roots as reaching themselves at distance 0
let nodes = roots.map(|x| (x, 0));
// repeatedly update minimal distances each node can be reached from each root
nodes.iterate(|inner| {
let edges = edges.enter(&inner.scope());
let nodes = nodes.enter(&inner.scope());
inner.join_map(&edges, |_k,l,d| (*d, l+1))
.concat(&nodes)
.reduce(|_, s, t| t.push((*s[0].0, 1)))
})
}

View File

@ -0,0 +1,14 @@
[package]
name = "experiments"
version = "0.1.0"
authors = ["Frank McSherry <fmcsherry@me.com>"]
[dependencies]
core_affinity = "0.5.9"
rand="0.3.13"
abomonation = "0.7"
abomonation_derive = "0.5"
#timely = "0.7"
timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" }
differential-dataflow = { path = "../" }
graph_map = { git = "https://github.com/frankmcsherry/graph-map" }

View File

@ -0,0 +1,237 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
extern crate core_affinity;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::operators::{Exchange, Probe};
// use timely::progress::nested::product::Product;
// use timely::progress::timestamp::RootTimestamp;
use differential_dataflow::input::Input;
use differential_dataflow::operators::arrange::ArrangeBySelf;
use differential_dataflow::operators::count::CountTotal;
use differential_dataflow::operators::threshold::ThresholdTotal;
// use differential_dataflow::trace::implementations::ord::OrdKeySpine;
#[derive(Debug)]
enum Comp {
Nothing,
Exchange,
Arrange,
Count,
Distinct,
}
#[derive(Debug)]
enum Mode {
OpenLoop,
ClosedLoop,
}
fn main() {
let keys: usize = std::env::args().nth(1).unwrap().parse().unwrap();
let recs: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let rate: usize = std::env::args().nth(3).unwrap().parse().unwrap();
let work: usize = std::env::args().nth(4).unwrap().parse().unwrap_or(usize::max_value());
let comp: Comp = match std::env::args().nth(5).unwrap().as_str() {
"exchange" => Comp::Exchange,
"arrange" => Comp::Arrange,
"count" => Comp::Count,
"distinct" => Comp::Distinct,
_ => Comp::Nothing,
};
let mode: Mode = if std::env::args().any(|x| x == "open-loop") { Mode::OpenLoop } else { Mode::ClosedLoop };
// define a new computational scope, in which to run BFS
timely::execute_from_args(std::env::args().skip(4), move |worker| {
let index = worker.index();
let core_ids = core_affinity::get_core_ids().unwrap();
core_affinity::set_for_current(core_ids[index]);
// create a degree counting differential dataflow
let (mut input, probe) = worker.dataflow::<u64,_,_>(|scope| {
let (handle, data) = scope.new_collection();
let probe = match comp {
Comp::Nothing => data.probe(),
Comp::Exchange => data.inner.exchange(|&(x,_,_): &((usize,()),_,_)| x.0 as u64).probe(),
Comp::Arrange => data.arrange_by_self().stream.probe(),
Comp::Count => data.arrange_by_self().count_total().probe(),
Comp::Distinct => data.arrange_by_self().distinct_total().probe(),
};
// OrdKeySpine::<usize, Product<RootTimestamp,u64>,isize>::with_effort(work)
(handle, probe)
});
let index = worker.index();
let peers = worker.peers();
let seed: &[_] = &[1, 2, 3, index];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for additions
let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for deletions
let timer = ::std::time::Instant::now();
for _ in 0 .. ((recs as usize) / peers) + if index < ((recs as usize) % peers) { 1 } else { 0 } {
input.insert((rng1.gen_range(0, keys),()));
}
input.advance_to(1u64);
input.flush();
while probe.less_than(input.time()) { worker.step(); }
if index == 0 {
let elapsed1 = timer.elapsed();
let elapsed1_ns = elapsed1.as_secs() * 1_000_000_000 + (elapsed1.subsec_nanos() as u64);
// println!("{:?}\tdata loaded; rate: {:?}", elapsed1, 1000000000.0 * (recs as f64) / (elapsed1_ns as f64));
println!("ARRANGE\tLOADING\t{}\t{:?}", peers, 1000000000.0 * (recs as f64) / (elapsed1_ns as f64));
}
if rate > 0 {
let timer = ::std::time::Instant::now();
// let mut counts = vec![0u64; 64];
let mut counts = vec![[0u64; 16]; 64];
match mode {
// closed-loop latency-throughput test, parameterized by rate size.
Mode::ClosedLoop => {
let mut wave = 1;
let mut elapsed = timer.elapsed();
while elapsed.as_secs() < 25 {
for round in 0 .. rate {
input.advance_to((((wave * rate) + round) * peers + index) as u64);
input.insert((rng1.gen_range(0, keys),()));
input.remove((rng2.gen_range(0, keys),()));
}
wave += 1;
input.advance_to((wave * rate * peers) as u64);
input.flush();
let elapsed1 = elapsed.clone();
let elapsed1_ns = elapsed1.as_secs() * 1_000_000_000 + (elapsed1.subsec_nanos() as u64);
while probe.less_than(input.time()) { worker.step(); }
elapsed = timer.elapsed();
let elapsed2 = elapsed.clone();
let elapsed2_ns = elapsed2.as_secs() * 1_000_000_000 + (elapsed2.subsec_nanos() as u64);
let count_index = (elapsed2_ns - elapsed1_ns).next_power_of_two().trailing_zeros() as usize;
if elapsed.as_secs() > 5 {
let low_bits = ((elapsed2_ns - elapsed1_ns) >> (count_index - 5)) & 0xF;
counts[count_index][low_bits as usize] += 1;
}
}
let elapsed = timer.elapsed();
let seconds = elapsed.as_secs() as f64 + (elapsed.subsec_nanos() as f64) / 1000000000.0;
if index == 0 {
// println!("{:?}, {:?}", seconds / (wave - 1) as f64, 2.0 * ((wave - 1) * rate * peers) as f64 / seconds);
println!("ARRANGE\tTHROUGHPUT\t{}\t{:?}", peers, 2.0 * ((wave - 1) * rate * peers) as f64 / seconds);
}
},
Mode::OpenLoop => {
let requests_per_sec = rate / 2;
let ns_per_request = 1_000_000_000 / requests_per_sec;
let mut request_counter = peers + index; // skip first request for each.
let mut ack_counter = peers + index;
let mut inserted_ns = 1;
let ack_target = 10 * keys;
while ack_counter < ack_target {
// while ((timer.elapsed().as_secs() as usize) * rate) < (10 * keys) {
// Open-loop latency-throughput test, parameterized by offered rate `ns_per_request`.
let elapsed = timer.elapsed();
let elapsed_ns = elapsed.as_secs() * 1_000_000_000 + (elapsed.subsec_nanos() as u64);
// Determine completed ns.
let acknowledged_ns: u64 = probe.with_frontier(|frontier| frontier[0]);
// any un-recorded measurements that are complete should be recorded.
while ((ack_counter * ns_per_request) as u64) < acknowledged_ns && ack_counter < ack_target {
let requested_at = (ack_counter * ns_per_request) as u64;
let count_index = (elapsed_ns - requested_at).next_power_of_two().trailing_zeros() as usize;
if ack_counter > ack_target / 2 {
// counts[count_index] += 1;
let low_bits = ((elapsed_ns - requested_at) >> (count_index - 5)) & 0xF;
counts[count_index][low_bits as usize] += 1;
}
ack_counter += peers;
}
// Now, should we introduce more records before stepping the worker?
//
// Thinking: inserted_ns - acknowledged_ns is some amount of time that
// is currently outstanding in the system, and we needn't advance our
// inputs unless by this order of magnitude.
//
// The more sophisticated plan is: we compute the next power of two
// greater than inserted_ns - acknowledged_ns and look for the last
// multiple of this number in the interval [inserted_ns, elapsed_ns].
// If such a multiple exists, we introduce records to that point and
// advance the input.
// let scale = (inserted_ns - acknowledged_ns).next_power_of_two();
// let target_ns = elapsed_ns & !(scale - 1);
let target_ns = if acknowledged_ns >= inserted_ns { elapsed_ns } else { inserted_ns };
// let target_ns = elapsed_ns & !((1 << 16) - 1);
if inserted_ns < target_ns {
while ((request_counter * ns_per_request) as u64) < target_ns {
input.advance_to((request_counter * ns_per_request) as u64);
input.insert((rng1.gen_range(0, keys),()));
input.remove((rng2.gen_range(0, keys),()));
request_counter += peers;
}
input.advance_to(target_ns);
input.flush();
inserted_ns = target_ns;
}
worker.step();
}
}
}
if index == 0 {
let mut results = Vec::new();
let total = counts.iter().map(|x| x.iter().sum::<u64>()).sum();
let mut sum = 0;
for index in (10 .. counts.len()).rev() {
for sub in (0 .. 16).rev() {
if sum > 0 && sum < total {
let latency = (1 << (index-1)) + (sub << (index-5));
let fraction = (sum as f64) / (total as f64);
results.push((latency, fraction));
}
sum += counts[index][sub];
}
}
for (latency, fraction) in results.drain(..).rev() {
println!("ARRANGE\tLATENCY\t{}\t{}\t{}\t{}\t{}\t{:?}\t{:?}\t{}\t{}", peers, keys, recs, rate, work, comp, mode, latency, fraction);
}
}
}
}).unwrap();
}

View File

@ -0,0 +1,64 @@
extern crate rand;
extern crate timely;
extern crate graph_map;
extern crate differential_dataflow;
use std::time::Instant;
use differential_dataflow::input::Input;
use differential_dataflow::operators::*;
fn main() {
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
timely::execute_from_args(std::env::args().skip(1), move |worker| {
let peers = worker.peers();
let index = worker.index();
let mut input = worker.dataflow::<(),_,_>(|scope| {
let (input, graph) = scope.new_collection();
let organizers = graph.explode(|(x,y)| Some((x, (1,0))).into_iter().chain(Some((y, (0,1))).into_iter()))
.threshold_total(|_,w| if w.1 == 0 { 1 } else { 0 });
organizers
.iterate(|attend| {
graph.enter(&attend.scope())
.semijoin(attend)
.map(|(_,y)| y)
.threshold_total(|_,w| if w >= &3 { 1 } else { 0 })
.concat(&organizers.enter(&attend.scope()))
.consolidate()
})
.map(|_| ())
.consolidate()
.inspect(|x| println!("{:?}", x));
input
});
let timer = Instant::now();
use std::io::{BufReader, BufRead};
use std::fs::File;
let file = BufReader::new(File::open(filename.clone()).unwrap());
for (count, readline) in file.lines().enumerate() {
let line = readline.ok().expect("read error");
if count % peers == index && !line.starts_with('#') {
let mut elts = line[..].split_whitespace();
let src: u32 = elts.next().unwrap().parse().ok().expect("malformed src");
let dst: u32 = elts.next().unwrap().parse().ok().expect("malformed dst");
input.insert((src, dst));
}
}
println!("{:?}\tData ingested", timer.elapsed());
}).unwrap();
}

View File

@ -0,0 +1,291 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
use std::time::Instant;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::*;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::operators::arrange::ArrangeByKey;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::trace::implementations::ord::OrdValSpine as DefaultValTrace;
use differential_dataflow::operators::arrange::TraceAgent;
use differential_dataflow::operators::arrange::Arranged;
type Arrange<G, K, V, R> = Arranged<G, TraceAgent<DefaultValTrace<K, V, <G as ScopeParent>::Timestamp, R>>>;
type Node = u32;
fn main() {
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
let rounds: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let batch: usize = std::env::args().nth(3).unwrap().parse().unwrap();
let workers: usize = std::env::args().nth(4).unwrap().parse().unwrap();
let inspect = std::env::args().any(|x| x == "inspect");
use timely::communication::allocator::zero_copy::allocator_process::ProcessBuilder;
let allocators = ProcessBuilder::new_vector(workers);
timely::execute::execute_from(allocators, Box::new(()), move |worker| {
// timely::execute_from_args(std::env::args().skip(1), move |worker| {
let peers = worker.peers();
let index = worker.index();
let (mut input, mut query1, mut query2, mut query3, probe) = worker.dataflow(|scope| {
let (input, graph) = scope.new_collection();
let (query1_input, query1) = scope.new_collection();
let (query2_input, query2) = scope.new_collection();
let (query3_input, query3) = scope.new_collection();
// each edge should exist in both directions.
let graph = graph.arrange_by_key();
let probe =
interactive(&graph, query1, query2, query3)
.filter(move |_| inspect)
// .map(|_| ())
.consolidate()
.inspect(|x| println!("count: {:?}", x))
.probe();
(input, query1_input, query2_input, query3_input, probe)
});
let timer = Instant::now();
let mut nodes = Vec::new();
use std::io::{BufReader, BufRead};
use std::fs::File;
let file = BufReader::new(File::open(filename.clone()).unwrap());
for (count, readline) in file.lines().enumerate() {
let line = readline.ok().expect("read error");
if count % peers == index && !line.starts_with('#') {
let mut elts = line[..].split_whitespace();
let src: u32 = elts.next().unwrap().parse().ok().expect("malformed src");
let dst: u32 = elts.next().unwrap().parse().ok().expect("malformed dst");
nodes.push(src);
nodes.push(dst);
input.insert((src, dst));
}
}
nodes.sort();
nodes.dedup();
if index == 0 {
println!("{:?}\tData ingested", timer.elapsed());
let seed: &[_] = &[1, 2, 3, index];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for additions
let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for deletions
let mut rng3: StdRng = SeedableRng::from_seed(seed); // rng for additions
let mut rng4: StdRng = SeedableRng::from_seed(seed); // rng for deletions
let mut rng5: StdRng = SeedableRng::from_seed(seed); // rng for additions
let mut rng6: StdRng = SeedableRng::from_seed(seed); // rng for deletions
// Pre-insert first queries, so we can remove them.
let insert = *rng1.choose(&nodes[..]).unwrap();
// println!("initializing {}", insert);
query1.insert(insert);
query2.insert(*rng3.choose(&nodes[..]).unwrap());
query3.insert(*rng5.choose(&nodes[..]).unwrap());
// run until graph is loaded
input.advance_to(1); input.flush();
query1.advance_to(1); query1.flush();
query2.advance_to(1); query2.flush();
query3.advance_to(1); query3.flush();
worker.step_while(|| probe.less_than(input.time()));
println!("{:?}\tData indexed", timer.elapsed());
let worker_batch = batch;
input.advance_to(1 + 1 * rounds); input.flush();
query2.advance_to(1 + 1 * rounds); query2.flush();
query3.advance_to(1 + 1 * rounds); query3.flush();
let mut latencies1 = Vec::with_capacity(rounds);
for round in 1 .. (1 + 1 * rounds) {
let timer = Instant::now();
for _ in 0 .. worker_batch {
let insert = *rng1.choose(&nodes[..]).unwrap();
let remove = *rng2.choose(&nodes[..]).unwrap();
// println!("replacing {} -> {}", remove, insert);
query1.insert(insert);
query1.remove(remove);
}
query1.advance_to(round);
query1.flush();
while probe.less_than(query1.time()) { worker.step(); }
latencies1.push(timer.elapsed().as_nanos());
}
if index == 0 { println!("{:?}\tRound 1 complete", timer.elapsed()); }
input.advance_to(1 + 2 * rounds); input.flush();
query1.advance_to(1 + 2 * rounds); query1.flush();
query3.advance_to(1 + 2 * rounds); query3.flush();
let mut latencies2 = Vec::with_capacity(rounds);
for round in (1 + 1 * rounds) .. (1 + 2 * rounds) {
let timer = Instant::now();
for _ in 0 .. worker_batch {
query2.insert(*rng3.choose(&nodes[..]).unwrap());
query2.remove(*rng4.choose(&nodes[..]).unwrap());
}
query2.advance_to(round);
query2.flush();
while probe.less_than(query2.time()) { worker.step(); }
latencies2.push(timer.elapsed().as_nanos());
// if index == 0 { println!("query2: {:?}", timer.elapsed()); }
}
if index == 0 { println!("{:?}\tRound 2 complete", timer.elapsed()); }
input.advance_to(1 + 3 * rounds); input.flush();
query1.advance_to(1 + 3 * rounds); query1.flush();
query2.advance_to(1 + 3 * rounds); query2.flush();
let mut latencies3 = Vec::with_capacity(rounds);
for round in (1 + 2 * rounds) .. (1 + 3 * rounds) {
let timer = Instant::now();
for _ in 0 .. worker_batch {
query3.insert(*rng5.choose(&nodes[..]).unwrap());
query3.remove(*rng6.choose(&nodes[..]).unwrap());
}
query3.advance_to(round);
query3.flush();
while probe.less_than(query3.time()) { worker.step(); }
latencies3.push(timer.elapsed().as_nanos());
// if index == 0 { println!("query3: {:?}", timer.elapsed()); }
}
if index == 0 { println!("{:?}\tRound 3 complete", timer.elapsed()); }
// query1.close();
// query2.close();
// query3.close();
// // let mut latencies4 = Vec::with_capacity(rounds);
// for round in (1 + 3 * rounds) .. (1 + 4 * rounds) {
// let timer = Instant::now();
// for _ in 0 .. worker_batch {
// input.insert((*rng.choose(&nodes[..]).unwrap(), *rng.choose(&nodes[..]).unwrap()));
// }
// input.advance_to(round);
// input.flush();
// while probe.less_than(input.time()) { worker.step(); }
// if index == 0 { println!("query4: {:?}", timer.elapsed()); }
// }
latencies1.sort();
for x in latencies1 { println!("q1:\t{:?}", x); }
latencies2.sort();
for x in latencies2 { println!("q2:\t{:?}", x); }
latencies3.sort();
for x in latencies3 { println!("q3:\t{:?}", x); }
}
}).unwrap();
}
fn interactive<G: Scope>(
edges: &Arrange<G, Node, Node, isize>,
tc_1: Collection<G, Node>,
tc_2: Collection<G, Node>,
sg_x: Collection<G, Node>
) -> Collection<G, Node>
where G::Timestamp: Lattice{
// descendants of tc_1:
let query1 =
tc_1.map(|x| (x,x))
.iterate(|inner|
edges
.enter(&inner.scope())
.join_map(&inner, |_,&y,&q| (y,q))
.concat(&tc_1.enter(&inner.scope()).map(|x| (x,x)))
.distinct()
)
.map(|(x,q)| (q,x));
// ancestors of tc_2:
let query2 =
tc_2.map(|x| (x,x))
.iterate(|inner|
edges
.as_collection(|&k,&v| (v,k))
.enter(&inner.scope())
.join_map(&inner, |_,&y,&q| (y,q))
.concat(&tc_2.enter(&inner.scope()).map(|x| (x,x)))
.distinct()
)
.map(|(x,q)| (q,x));
// Adapted from: http://ranger.uta.edu/~fegaras/cse6331/spring97/p25.html
// sg(X,X) <- magic(X).
// sg(X,Y) <- magic(X), par(X,Xp), par(Y,Yp), sg(Xp,Yp).
// ancestors of sg_x:
let magic =
sg_x.iterate(|inner|
edges
.as_collection(|&k,&v| (v,k))
.enter(&inner.scope())
.semijoin(&inner)
.map(|(_x,y)| y)
.concat(&sg_x.enter(&inner.scope()))
.distinct()
);
let magic_edges =
edges
.semijoin(&magic)
.map(|(x,y)|(y,x))
.semijoin(&magic)
.map(|(x,y)|(y,x));
let query3 =
magic
.map(|x| (x,x)) // for query q, sg(x,x)
.iterate(|inner| {
let edges = edges.enter(&inner.scope());
let magic = magic.enter(&inner.scope());
let magic_edges = magic_edges.enter(&inner.scope());
let result =
inner
.join_map(&magic_edges, |_x,&y,&cx| (y,cx))
.join_core(&edges, |_y,&cx,&cy| Some((cx,cy)))
.concat(&magic.map(|x| (x,x)))
.distinct();
// result.map(|_| ()).consolidate().inspect(|x| println!("\t{:?}", x));
result
})
.semijoin(&sg_x);
query1.concat(&query2).concat(&query3).map(|(q,_)| q)
}

View File

@ -0,0 +1,143 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
extern crate core_affinity;
use std::time::Instant;
use timely::dataflow::*;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::trace::implementations::ord::{OrdValSpine, OrdKeySpine};
use differential_dataflow::operators::arrange::TraceAgent;
use differential_dataflow::operators::arrange::Arranged;
use differential_dataflow::operators::arrange::Arrange;
use differential_dataflow::operators::iterate::SemigroupVariable;
use differential_dataflow::difference::Present;
type EdgeArranged<G, K, V, R> = Arranged<G, TraceAgent<OrdValSpine<K, V, <G as ScopeParent>::Timestamp, R, Offs>>>;
type Node = u32;
type Edge = (Node, Node);
type Iter = u32;
type Offs = u32;
fn main() {
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
let program = std::env::args().nth(2).unwrap();
timely::execute_from_args(std::env::args().skip(1), move |worker| {
let peers = worker.peers();
let index = worker.index();
let timer = Instant::now();
let core_ids = core_affinity::get_core_ids().unwrap();
core_affinity::set_for_current(core_ids[index % core_ids.len()]);
let inspect = true;
let mut input = worker.dataflow::<(),_,_>(|scope| {
let (input, graph) = scope.new_collection();
// each edge should exist in both directions.
let graph = graph.arrange::<OrdValSpine<_,_,_,_,Offs>>();
match program.as_str() {
"tc" => tc(&graph).filter(move |_| inspect).map(|_| ()).consolidate().inspect(|x| println!("tc count: {:?}", x)).probe(),
"sg" => sg(&graph).filter(move |_| inspect).map(|_| ()).consolidate().inspect(|x| println!("sg count: {:?}", x)).probe(),
_ => panic!("must specify one of 'tc', 'sg'.")
};
input
});
let mut nodes = 0;
use std::io::{BufReader, BufRead};
use std::fs::File;
let file = BufReader::new(File::open(filename.clone()).unwrap());
for (count, readline) in file.lines().enumerate() {
let line = readline.ok().expect("read error");
if count % peers == index && !line.starts_with('#') {
let mut elts = line[..].split_whitespace();
let src: u32 = elts.next().unwrap().parse().ok().expect("malformed src");
let dst: u32 = elts.next().unwrap().parse().ok().expect("malformed dst");
if nodes < src { nodes = src; }
if nodes < dst { nodes = dst; }
input.update((src, dst), Present);
}
}
if index == 0 { println!("{:?}\tData ingested", timer.elapsed()); }
input.close();
while worker.step() { }
if index == 0 { println!("{:?}\tComputation complete", timer.elapsed()); }
}).unwrap();
}
use timely::order::Product;
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn tc<G: Scope<Timestamp=()>>(edges: &EdgeArranged<G, Node, Node, Present>) -> Collection<G, Edge, Present> {
// repeatedly update minimal distances each node can be reached from each root
edges.stream.scope().iterative::<Iter,_,_>(|scope| {
let inner = SemigroupVariable::new(scope, Product::new(Default::default(), 1));
let edges = edges.enter(&inner.scope());
let result =
inner
.map(|(x,y)| (y,x))
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
.join_core(&edges, |_y,&x,&z| Some((x, z)))
.concat(&edges.as_collection(|&k,&v| (k,v)))
.arrange::<OrdKeySpine<_,_,_,Offs>>()
.threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None })
;
inner.set(&result);
result.leave()
}
)
}
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn sg<G: Scope<Timestamp=()>>(edges: &EdgeArranged<G, Node, Node, Present>) -> Collection<G, Edge, Present> {
let peers = edges.join_core(&edges, |_,&x,&y| Some((x,y))).filter(|&(x,y)| x != y);
// repeatedly update minimal distances each node can be reached from each root
peers.scope().iterative::<Iter,_,_>(|scope| {
let inner = SemigroupVariable::new(scope, Product::new(Default::default(), 1));
let edges = edges.enter(&inner.scope());
let peers = peers.enter(&inner.scope());
let result =
inner
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
.join_core(&edges, |_,&x,&z| Some((x, z)))
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
.join_core(&edges, |_,&x,&z| Some((x, z)))
.concat(&peers)
.arrange::<OrdKeySpine<_,_,_,Offs>>()
.threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None })
;
inner.set(&result);
result.leave()
}
)
}

View File

@ -0,0 +1,397 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
extern crate core_affinity;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::*;
use timely::dataflow::operators::probe::Handle;
use timely::order::Product;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::iterate::Variable;
use differential_dataflow::operators::arrange::ArrangeByKey;
use differential_dataflow::operators::arrange::ArrangeBySelf;
type Node = usize;
type Iter = usize;
fn main() {
let nodes: usize = std::env::args().nth(1).unwrap().parse().unwrap();
let edges: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let rate: usize = std::env::args().nth(3).unwrap().parse().unwrap();
let goal: usize = std::env::args().nth(4).unwrap().parse().unwrap();
let queries: usize = std::env::args().nth(5).unwrap().parse().unwrap();
let shared: bool = std::env::args().any(|x| x == "share");
// Our setting involves four read query types, and two updatable base relations.
//
// Q1: Point lookup: reads "state" associated with a node.
// Q2: One-hop lookup: reads "state" associated with neighbors of a node.
// Q3: Two-hop lookup: reads "state" associated with n-of-n's of a node.
// Q4: Shortest path: reports hop count between two query nodes.
//
// R1: "State": a pair of (node, T) for some type T that I don't currently know.
// R2: "Graph": pairs (node, node) indicating linkage between the two nodes.
timely::execute_from_args(std::env::args().skip(3), move |worker| {
let index = worker.index();
let peers = worker.peers();
let timer = ::std::time::Instant::now();
let core_ids = core_affinity::get_core_ids().unwrap();
core_affinity::set_for_current(core_ids[index % core_ids.len()]);
// define BFS dataflow; return handles to roots and edges inputs
let mut probe = Handle::new();
let (mut q1, mut q2, mut q3, mut q4, mut state, mut graph) = worker.dataflow(|scope| {
let (q1_input, q1) = scope.new_collection();
let (q2_input, q2) = scope.new_collection::<usize,isize>();
let (q3_input, q3) = scope.new_collection::<usize,isize>();
let (q4_input, q4) = scope.new_collection();
let (state_input, state) = scope.new_collection();
let (graph_input, graph) = scope.new_collection();
if shared {
let state_indexed = state.arrange_by_key();
let graph_indexed = graph.map(|(src, dst)| (dst, src))
.concat(&graph)
.arrange_by_key();
// Q1: Point lookups on `state`:
q1 .arrange_by_self()
.join_core(&state_indexed, |&query, &(), &state| Some((query, state)))
.probe_with(&mut probe);
// Q2: One-hop lookups on `state`:
q2 .arrange_by_self()
.join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query)))
.join_core(&state_indexed, |_friend, &query, &state| Some((query, state)))
.probe_with(&mut probe);
// Q3: Two-hop lookups on `state`:
q3 .arrange_by_self()
.join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query)))
.join_core(&graph_indexed, |_friend, &query, &friend2| Some((friend2, query)))
.join_core(&state_indexed, |_friend2, &query, &state| Some((query, state)))
.probe_with(&mut probe);
// Q4: Shortest path queries:
three_hop(&graph_indexed, &graph_indexed, &q4)
.probe_with(&mut probe);
connected_components(&graph_indexed)
.probe_with(&mut probe);
}
else {
// let state_indexed = state.arrange_by_key();
let graph = graph.map(|(src, dst)| (dst, src))
.concat(&graph);
// Q1: Point lookups on `state`:
q1 .arrange_by_self()
.join_core(&state.arrange_by_key(), |&query, &(), &state| Some((query, state)))
.probe_with(&mut probe);
// Q2: One-hop lookups on `state`:
q2 .arrange_by_self()
.join_core(&graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query)))
.join_core(&state.arrange_by_key(), |_friend, &query, &state| Some((query, state)))
.probe_with(&mut probe);
// Q3: Two-hop lookups on `state`:
q3 .arrange_by_self()
.join_core(&graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query)))
.join_core(&graph.arrange_by_key(), |_friend, &query, &friend2| Some((friend2, query)))
.join_core(&state.arrange_by_key(), |_friend2, &query, &state| Some((query, state)))
.probe_with(&mut probe);
// Q4: Shortest path queries:
three_hop(&graph.arrange_by_key(), &graph.arrange_by_key(), &q4)
.probe_with(&mut probe);
connected_components(&graph.arrange_by_key())
.probe_with(&mut probe);
}
(q1_input, q2_input, q3_input, q4_input, state_input, graph_input)
});
let seed: &[_] = &[1, 2, 3, index];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for edge deletions
let mut rng3: StdRng = SeedableRng::from_seed(seed); // rng for q1 additions
// let mut rng4: StdRng = SeedableRng::from_seed(seed); // rng for q1 deletions
// let mut rng5: StdRng = SeedableRng::from_seed(seed); // rng for q4 additions
// let mut rng6: StdRng = SeedableRng::from_seed(seed); // rng for q4 deletions
if index == 0 { println!("performing workload on random graph with {} nodes, {} edges:", nodes, edges); }
let worker_edges = edges/peers + if index < (edges % peers) { 1 } else { 0 };
for _ in 0 .. worker_edges {
graph.insert((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)));
}
for node in 0 .. nodes {
if node % peers == index {
state.insert((node, node));
}
}
// let queries = 1000;
let worker_window = queries/peers + if index < (queries % peers) { 1 } else { 0 };
for _ in 0 .. worker_window {
q1.insert(rng3.gen_range(0, nodes));
q2.insert(rng3.gen_range(0, nodes));
q3.insert(rng3.gen_range(0, nodes));
q4.insert((rng3.gen_range(0, nodes), rng3.gen_range(0, nodes)));
}
q1.advance_to(usize::max_value()); q1.flush(); // q1 queries start now.
q2.advance_to(usize::max_value()); q2.flush(); // q2 queries start here.
q3.advance_to(usize::max_value()); q3.flush(); // q3 queries start here.
q4.advance_to(usize::max_value()); q4.flush(); // q4 queries start here.
state.advance_to(usize::max_value()); state.flush();
graph.advance_to(1); graph.flush();
// finish graph loading work.
while probe.less_than(graph.time()) { worker.step(); }
if index == 0 { println!("{:?}\tgraph loaded", timer.elapsed()); }
let requests_per_sec = rate / 2;
let ns_per_request = 1_000_000_000 / requests_per_sec;
let mut request_counter = peers + index; // skip first request for each.
let mut ack_counter = peers + index;
let mut inserted_ns = 1;
let timer = ::std::time::Instant::now();
let mut counts = vec![[0usize; 16]; 64];
let ack_target = goal * rate;
while ack_counter < ack_target {
// Open-loop latency-throughput test, parameterized by offered rate `ns_per_request`.
let elapsed = timer.elapsed();
let elapsed_ns: usize = (elapsed.as_secs() * 1_000_000_000 + (elapsed.subsec_nanos() as u64)) as usize;
// Determine completed ns.
let acknowledged_ns: usize = probe.with_frontier(|frontier| frontier[0]);
// any un-recorded measurements that are complete should be recorded.
while (ack_counter * ns_per_request) < acknowledged_ns && ack_counter < ack_target {
let requested_at = ack_counter * ns_per_request;
let count_index = (elapsed_ns - requested_at).next_power_of_two().trailing_zeros() as usize;
if ack_counter > ack_target / 2 {
let low_bits = ((elapsed_ns - requested_at) >> (count_index - 5)) & 0xF;
counts[count_index][low_bits as usize] += 1;
}
ack_counter += peers;
}
// Now, should we introduce more records before stepping the worker?
//
// Thinking: inserted_ns - acknowledged_ns is some amount of time that
// is currently outstanding in the system, and we needn't advance our
// inputs unless by this order of magnitude.
//
// The more sophisticated plan is: we compute the next power of two
// greater than inserted_ns - acknowledged_ns and look for the last
// multiple of this number in the interval [inserted_ns, elapsed_ns].
// If such a multiple exists, we introduce records to that point and
// advance the input.
// let scale = (inserted_ns - acknowledged_ns).next_power_of_two();
// let target_ns = elapsed_ns & !(scale - 1);
let mut target_ns = elapsed_ns & !((1 << 20) - 1);
// let mut target_ns = if acknowledged_ns >= inserted_ns { elapsed_ns } else { inserted_ns };
if target_ns > inserted_ns + 1_000_000_000 { target_ns = inserted_ns + 1_000_000_000; }
if inserted_ns < target_ns {
while (request_counter * ns_per_request) < target_ns {
graph.advance_to(request_counter * ns_per_request);
graph.insert((rng1.gen_range(0, nodes),rng1.gen_range(0, nodes)));
graph.remove((rng2.gen_range(0, nodes),rng2.gen_range(0, nodes)));
request_counter += peers;
}
graph.advance_to(target_ns); graph.flush();
inserted_ns = target_ns;
}
worker.step();
}
if index == 0 {
let mut results = Vec::new();
let total = counts.iter().map(|x| x.iter().sum::<usize>()).sum();
let mut sum = 0;
for index in (10 .. counts.len()).rev() {
for sub in (0 .. 16).rev() {
if sum > 0 && sum < total {
let latency = (1 << (index-1)) + (sub << (index-5));
let fraction = (sum as f64) / (total as f64);
results.push((latency, fraction));
}
sum += counts[index][sub];
}
}
for (latency, fraction) in results.drain(..).rev() {
println!("{}\t{}", latency, fraction);
}
}
}).unwrap();
}
use differential_dataflow::trace::implementations::ord::OrdValSpine as DefaultValTrace;
use differential_dataflow::operators::arrange::TraceAgent;
use differential_dataflow::operators::arrange::Arranged;
type Arrange<G, K, V, R> = Arranged<G, TraceAgent<DefaultValTrace<K, V, <G as ScopeParent>::Timestamp, R>>>;
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn three_hop<G: Scope>(
forward_graph: &Arrange<G, Node, Node, isize>,
reverse_graph: &Arrange<G, Node, Node, isize>,
goals: &Collection<G, (Node, Node)>) -> Collection<G, ((Node, Node), u32)>
where G::Timestamp: Lattice+Ord {
let sources = goals.map(|(x,_)| x);
let targets = goals.map(|(_,y)| y);
// Q3: Two-hop lookups on `state`:
let forward0 = sources.map(|x| (x, (x,0)));
let forward1 = forward0.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1))));
let forward2 = forward1.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1))));
let reverse0 = targets.map(|x| (x, (x,0)));
let reverse1 = reverse0.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1))));
let reverse2 = reverse1.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1))));
let forward = forward0.concat(&forward1).concat(&forward2);
let reverse = reverse0.concat(&reverse1).concat(&reverse2);
forward
.join_map(&reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2))
.reduce(|_st,input,output| output.push((*input[0].0,1)))
}
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn _bidijkstra<G: Scope>(
forward_graph: &Arrange<G, Node, Node, isize>,
reverse_graph: &Arrange<G, Node, Node, isize>,
goals: &Collection<G, (Node, Node)>) -> Collection<G, ((Node, Node), u32)>
where G::Timestamp: Lattice+Ord {
goals.scope().iterative::<Iter,_,_>(|inner| {
// Our plan is to start evolving distances from both sources and destinations.
// The evolution from a source or destination should continue as long as there
// is a corresponding destination or source that has not yet been reached.
// forward and reverse (node, (root, dist))
let forward = Variable::new_from(goals.map(|(x,_)| (x,(x,0))).enter(inner), Product::new(Default::default(), 1));
let reverse = Variable::new_from(goals.map(|(_,y)| (y,(y,0))).enter(inner), Product::new(Default::default(), 1));
let goals = goals.enter(inner);
let forward_graph = forward_graph.enter(inner);
let reverse_graph = reverse_graph.enter(inner);
// Let's determine which (src, dst) pairs are ready to return.
//
// done(src, dst) := forward(src, med), reverse(dst, med), goal(src, dst).
//
// This is a cyclic join, which should scare us a bunch.
let reached =
forward
.join_map(&reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2))
.reduce(|_key, s, t| t.push((*s[0].0, 1)))
.semijoin(&goals);
let active =
reached
.negate()
.map(|(srcdst,_)| srcdst)
.concat(&goals)
.consolidate();
// Let's expand out forward queries that are active.
let forward_active = active.map(|(x,_y)| x).distinct();
let forward_next =
forward
.map(|(med, (src, dist))| (src, (med, dist)))
.semijoin(&forward_active)
.map(|(src, (med, dist))| (med, (src, dist)))
.join_core(&forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1))))
.concat(&forward)
.map(|(next, (src, dist))| ((next, src), dist))
.reduce(|_key, s, t| t.push((*s[0].0, 1)))
.map(|((next, src), dist)| (next, (src, dist)));
forward.set(&forward_next);
// Let's expand out reverse queries that are active.
let reverse_active = active.map(|(_x,y)| y).distinct();
let reverse_next =
reverse
.map(|(med, (rev, dist))| (rev, (med, dist)))
.semijoin(&reverse_active)
.map(|(rev, (med, dist))| (med, (rev, dist)))
.join_core(&reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1))))
.concat(&reverse)
.map(|(next, (rev, dist))| ((next, rev), dist))
.reduce(|_key, s, t| t.push((*s[0].0, 1)))
.map(|((next,rev), dist)| (next, (rev, dist)));
reverse.set(&reverse_next);
reached.leave()
})
}
fn connected_components<G: Scope>(graph: &Arrange<G, Node, Node, isize>) -> Collection<G, (Node, Node)>
where G::Timestamp: Lattice {
// each edge (x,y) means that we need at least a label for the min of x and y.
let nodes =
graph
.as_collection(|&k,&v| {
let min = std::cmp::min(k,v);
(min, min)
})
.consolidate();
// don't actually use these labels, just grab the type
nodes
.filter(|_| false)
.iterate(|inner| {
let graph = graph.enter(&inner.scope());
let nodes = nodes.enter_at(&inner.scope(), |r| 256 * (64 - r.1.leading_zeros() as u64));
let inner = inner.arrange_by_key();
let prop = inner.join_core(&graph, |_k,l,d| Some((*d,*l)));
nodes
.concat(&prop)
.reduce(|_, s, t| { t.push((*s[0].0, 1)); })
})
}

View File

@ -0,0 +1,339 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
extern crate core_affinity;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::*;
use timely::dataflow::operators::probe::Handle;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::lattice::Lattice;
// use differential_dataflow::operators::iterate::Variable;
use differential_dataflow::operators::arrange::ArrangeByKey;
use differential_dataflow::operators::arrange::ArrangeBySelf;
type Node = usize;
fn main() {
let nodes: usize = std::env::args().nth(1).unwrap().parse().unwrap();
let edges: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let rate: usize = std::env::args().nth(3).unwrap().parse().unwrap();
let goal: usize = std::env::args().nth(4).unwrap().parse().unwrap();
let mode: usize = std::env::args().nth(5).unwrap().parse().unwrap();
// Our setting involves four read query types, and two updatable base relations.
//
// Q1: Point lookup: reads "state" associated with a node.
// Q2: One-hop lookup: reads "state" associated with neighbors of a node.
// Q3: Two-hop lookup: reads "state" associated with n-of-n's of a node.
// Q4: Shortest path: reports hop count between two query nodes.
//
// R1: "State": a pair of (node, T) for some type T that I don't currently know.
// R2: "Graph": pairs (node, node) indicating linkage between the two nodes.
timely::execute_from_args(std::env::args().skip(3), move |worker| {
let index = worker.index();
let peers = worker.peers();
let timer = ::std::time::Instant::now();
let core_ids = core_affinity::get_core_ids().unwrap();
core_affinity::set_for_current(core_ids[index % core_ids.len()]);
// define BFS dataflow; return handles to roots and edges inputs
let mut probe = Handle::new();
let (mut query, mut state, mut graph) = worker.dataflow(|scope| {
let (query_input, query) = scope.new_collection();
let (state_input, state) = scope.new_collection();
let (graph_input, graph) = scope.new_collection();
let state_indexed = state.arrange_by_key();
let graph_indexed = graph.map(|(src, dst)| (dst, src))
.concat(&graph)
.arrange_by_key();
match mode {
1 => {
// Q1: Point lookups on `state`:
query
.map(|(x,_)| x)
.arrange_by_self()
.join_core(&state_indexed, |&query, &(), &state| Some((query, state)))
.probe_with(&mut probe);
},
2 => {
// Q2: One-hop lookups on `state`:
query
.map(|(x,_)| x)
.arrange_by_self()
.join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query)))
.join_core(&state_indexed, |_friend, &query, &state| Some((query, state)))
.probe_with(&mut probe);
},
3 => {
// Q3: Two-hop lookups on `state`:
query
.map(|(x,_)| x)
.arrange_by_self()
.join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query)))
.join_core(&graph_indexed, |_friend, &query, &friend2| Some((friend2, query)))
.join_core(&state_indexed, |_friend2, &query, &state| Some((query, state)))
.probe_with(&mut probe);
},
4 => {
// Q4: Shortest path queries:
three_hop(&graph_indexed, &graph_indexed, &query)
.probe_with(&mut probe);
}
x => { panic!("Unknown mode: {:?}; must be: 1, 2, 3, 4", x); }
}
(query_input, state_input, graph_input)
});
let seed: &[_] = &[1, 2, 3, index];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for edge deletions
let seed: &[_] = &[1, 2, 4, index];
let mut rng3: StdRng = SeedableRng::from_seed(seed); // rng for query additions
let mut rng4: StdRng = SeedableRng::from_seed(seed); // rng for q1 deletions
if index == 0 { println!("performing workload on random graph with {} nodes, {} edges:", nodes, edges); }
let worker_edges = edges/peers + if index < (edges % peers) { 1 } else { 0 };
for _ in 0 .. worker_edges {
graph.insert((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)));
}
for node in 0 .. nodes {
if node % peers == index {
state.insert((node, node));
}
}
let worker_window = 1;
for _ in 0 .. worker_window {
query.insert((rng3.gen_range(0, nodes), rng3.gen_range(0, nodes)));
}
query.advance_to(1); query.flush();
state.advance_to(usize::max_value()); state.flush();
graph.advance_to(1); graph.flush();
// finish graph loading work.
while probe.less_than(graph.time()) { worker.step(); }
if index == 0 { println!("{:?}\tgraph loaded", timer.elapsed()); }
let requests_per_sec = rate / 2;
let ns_per_request = 1_000_000_000 / requests_per_sec;
let mut request_counter = peers + index; // skip first request for each.
let mut ack_counter = peers + index;
let mut inserted_ns = 1;
let timer = ::std::time::Instant::now();
let mut counts = vec![[0usize; 16]; 64];
let ack_target = goal * rate;
while ack_counter < ack_target {
// Open-loop latency-throughput test, parameterized by offered rate `ns_per_request`.
let elapsed = timer.elapsed();
let elapsed_ns: usize = (elapsed.as_secs() * 1_000_000_000 + (elapsed.subsec_nanos() as u64)) as usize;
// Determine completed ns.
let acknowledged_ns: usize = probe.with_frontier(|frontier| frontier[0]);
// any un-recorded measurements that are complete should be recorded.
while (ack_counter * ns_per_request) < acknowledged_ns && ack_counter < ack_target {
let requested_at = ack_counter * ns_per_request;
let count_index = (elapsed_ns - requested_at).next_power_of_two().trailing_zeros() as usize;
if ack_counter > ack_target / 2 {
let low_bits = ((elapsed_ns - requested_at) >> (count_index - 5)) & 0xF;
counts[count_index][low_bits as usize] += 1;
}
ack_counter += peers;
}
// Now, should we introduce more records before stepping the worker?
//
// Thinking: inserted_ns - acknowledged_ns is some amount of time that
// is currently outstanding in the system, and we needn't advance our
// inputs unless by this order of magnitude.
//
// The more sophisticated plan is: we compute the next power of two
// greater than inserted_ns - acknowledged_ns and look for the last
// multiple of this number in the interval [inserted_ns, elapsed_ns].
// If such a multiple exists, we introduce records to that point and
// advance the input.
// let scale = (inserted_ns - acknowledged_ns).next_power_of_two();
// let target_ns = elapsed_ns & !(scale - 1);
let mut target_ns = elapsed_ns & !((1 << 20) - 1);
// let mut target_ns = if acknowledged_ns >= inserted_ns { elapsed_ns } else { inserted_ns };
if target_ns > inserted_ns + 1_000_000_000 { target_ns = inserted_ns + 1_000_000_000; }
if inserted_ns < target_ns {
while (request_counter * ns_per_request) < target_ns {
if (request_counter / peers) % 2 == 0 {
graph.advance_to(request_counter * ns_per_request);
graph.insert((rng1.gen_range(0, nodes),rng1.gen_range(0, nodes)));
graph.remove((rng2.gen_range(0, nodes),rng2.gen_range(0, nodes)));
}
else {
query.advance_to(request_counter * ns_per_request);
query.insert((rng3.gen_range(0, nodes),rng3.gen_range(0, nodes)));
query.remove((rng4.gen_range(0, nodes),rng4.gen_range(0, nodes)));
}
request_counter += peers;
}
graph.advance_to(target_ns); graph.flush();
query.advance_to(target_ns); query.flush();
inserted_ns = target_ns;
}
worker.step();
}
if index == 0 {
let mut results = Vec::new();
let total = counts.iter().map(|x| x.iter().sum::<usize>()).sum();
let mut sum = 0;
for index in (10 .. counts.len()).rev() {
for sub in (0 .. 16).rev() {
if sum > 0 && sum < total {
let latency = (1 << (index-1)) + (sub << (index-5));
let fraction = (sum as f64) / (total as f64);
results.push((latency, fraction));
}
sum += counts[index][sub];
}
}
for (latency, fraction) in results.drain(..).rev() {
println!("{}\t{}", latency, fraction);
}
}
}).unwrap();
}
use differential_dataflow::trace::implementations::ord::OrdValSpine as DefaultValTrace;
use differential_dataflow::operators::arrange::TraceAgent;
use differential_dataflow::operators::arrange::Arranged;
type Arrange<G, K, V, R> = Arranged<G, TraceAgent<DefaultValTrace<K, V, <G as ScopeParent>::Timestamp, R>>>;
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn three_hop<G: Scope>(
forward_graph: &Arrange<G, Node, Node, isize>,
reverse_graph: &Arrange<G, Node, Node, isize>,
goals: &Collection<G, (Node, Node)>) -> Collection<G, ((Node, Node), u32)>
where G::Timestamp: Lattice+Ord {
let sources = goals.map(|(x,_)| x);
let targets = goals.map(|(_,y)| y);
// Q3: Two-hop lookups on `state`:
let forward0 = sources.map(|x| (x, (x,0)));
let forward1 = forward0.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1))));
let forward2 = forward1.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1))));
let reverse0 = targets.map(|x| (x, (x,0)));
let reverse1 = reverse0.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1))));
let reverse2 = reverse1.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1))));
let forward = forward0.concat(&forward1).concat(&forward2);
let reverse = reverse0.concat(&reverse1).concat(&reverse2);
forward
.join_map(&reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2))
.reduce(|_st,input,output| output.push((*input[0].0,1)))
}
// // returns pairs (n, s) indicating node n can be reached from a root in s steps.
// fn bidijkstra<G: Scope>(
// forward_graph: &Arrange<G, Node, Node, isize>,
// reverse_graph: &Arrange<G, Node, Node, isize>,
// goals: &Collection<G, (Node, Node)>,
// bound: u64) -> Collection<G, ((Node, Node), u32)>
// where G::Timestamp: Lattice+Ord {
// goals.scope().scoped(|inner| {
// // Our plan is to start evolving distances from both sources and destinations.
// // The evolution from a source or destination should continue as long as there
// // is a corresponding destination or source that has not yet been reached.
// // forward and reverse (node, (root, dist))
// let forward = Variable::from_args(bound, 1, goals.map(|(x,_)| (x,(x,0))).enter(inner));
// let reverse = Variable::from_args(bound, 1, goals.map(|(_,y)| (y,(y,0))).enter(inner));
// let goals = goals.enter(inner);
// let forward_graph = forward_graph.enter(inner);
// let reverse_graph = reverse_graph.enter(inner);
// // Let's determine which (src, dst) pairs are ready to return.
// //
// // done(src, dst) := forward(src, med), reverse(dst, med), goal(src, dst).
// //
// // This is a cyclic join, which should scare us a bunch.
// let reached =
// forward
// .join_map(&reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2))
// .reduce(|_key, s, t| t.push((*s[0].0, 1)))
// .semijoin(&goals);
// let active =
// reached
// .negate()
// .map(|(srcdst,_)| srcdst)
// .concat(&goals)
// .consolidate();
// // Let's expand out forward queries that are active.
// let forward_active = active.map(|(x,_y)| x).distinct();
// let forward_next =
// forward
// .map(|(med, (src, dist))| (src, (med, dist)))
// .semijoin(&forward_active)
// .map(|(src, (med, dist))| (med, (src, dist)))
// .join_core(&forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1))))
// .concat(&forward)
// .map(|(next, (src, dist))| ((next, src), dist))
// .reduce(|_key, s, t| t.push((*s[0].0, 1)))
// .map(|((next, src), dist)| (next, (src, dist)));
// forward.set(&forward_next);
// // Let's expand out reverse queries that are active.
// let reverse_active = active.map(|(_x,y)| y).distinct();
// let reverse_next =
// reverse
// .map(|(med, (rev, dist))| (rev, (med, dist)))
// .semijoin(&reverse_active)
// .map(|(rev, (med, dist))| (med, (rev, dist)))
// .join_core(&reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1))))
// .concat(&reverse)
// .map(|(next, (rev, dist))| ((next, rev), dist))
// .reduce(|_key, s, t| t.push((*s[0].0, 1)))
// .map(|((next,rev), dist)| (next, (rev, dist)));
// reverse.set(&reverse_next);
// reached.leave()
// })
// }

View File

@ -0,0 +1,400 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
extern crate core_affinity;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::*;
use timely::dataflow::operators::probe::Handle;
use timely::order::Product;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::iterate::Variable;
use differential_dataflow::operators::arrange::ArrangeByKey;
use differential_dataflow::operators::arrange::ArrangeBySelf;
type Node = usize;
type Iter = usize;
fn main() {
let nodes: usize = std::env::args().nth(1).unwrap().parse().unwrap();
let edges: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let rate: usize = std::env::args().nth(3).unwrap().parse().unwrap();
let goal: usize = std::env::args().nth(4).unwrap().parse().unwrap();
let queries: usize = std::env::args().nth(5).unwrap().parse().unwrap();
let shared: bool = std::env::args().any(|x| x == "share");
// Our setting involves four read query types, and two updatable base relations.
//
// Q1: Point lookup: reads "state" associated with a node.
// Q2: One-hop lookup: reads "state" associated with neighbors of a node.
// Q3: Two-hop lookup: reads "state" associated with n-of-n's of a node.
// Q4: Shortest path: reports hop count between two query nodes.
//
// R1: "State": a pair of (node, T) for some type T that I don't currently know.
// R2: "Graph": pairs (node, node) indicating linkage between the two nodes.
timely::execute_from_args(std::env::args().skip(3), move |worker| {
let index = worker.index();
let peers = worker.peers();
let timer = ::std::time::Instant::now();
let core_ids = core_affinity::get_core_ids().unwrap();
core_affinity::set_for_current(core_ids[index % core_ids.len()]);
// define BFS dataflow; return handles to roots and edges inputs
let mut probe = Handle::new();
let (mut q1, mut q2, mut q3, mut q4, mut state, mut graph) = worker.dataflow(|scope| {
let (q1_input, q1) = scope.new_collection();
let (q2_input, q2) = scope.new_collection::<usize,isize>();
let (q3_input, q3) = scope.new_collection::<usize,isize>();
let (q4_input, q4) = scope.new_collection();
let (state_input, state) = scope.new_collection();
let (graph_input, graph) = scope.new_collection();
if shared {
let state_indexed = state.arrange_by_key();
let graph_indexed = graph.map(|(src, dst)| (dst, src))
.concat(&graph)
.arrange_by_key();
// Q1: Point lookups on `state`:
q1 .arrange_by_self()
.join_core(&state_indexed, |&query, &(), &state| Some((query, state)))
.probe_with(&mut probe);
// Q2: One-hop lookups on `state`:
q2 .arrange_by_self()
.join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query)))
.join_core(&state_indexed, |_friend, &query, &state| Some((query, state)))
.probe_with(&mut probe);
// Q3: Two-hop lookups on `state`:
q3 .arrange_by_self()
.join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query)))
.join_core(&graph_indexed, |_friend, &query, &friend2| Some((friend2, query)))
.join_core(&state_indexed, |_friend2, &query, &state| Some((query, state)))
.probe_with(&mut probe);
// Q4: Shortest path queries:
three_hop(&graph_indexed, &graph_indexed, &q4)
.probe_with(&mut probe);
}
else {
// let state_indexed = state.arrange_by_key();
let graph = graph.map(|(src, dst)| (dst, src))
.concat(&graph);
// Q1: Point lookups on `state`:
q1 .arrange_by_self()
.join_core(&state.arrange_by_key(), |&query, &(), &state| Some((query, state)))
.probe_with(&mut probe);
// Q2: One-hop lookups on `state`:
q2 .arrange_by_self()
.join_core(&graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query)))
.join_core(&state.arrange_by_key(), |_friend, &query, &state| Some((query, state)))
.probe_with(&mut probe);
// Q3: Two-hop lookups on `state`:
q3 .arrange_by_self()
.join_core(&graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query)))
.join_core(&graph.arrange_by_key(), |_friend, &query, &friend2| Some((friend2, query)))
.join_core(&state.arrange_by_key(), |_friend2, &query, &state| Some((query, state)))
.probe_with(&mut probe);
// Q4: Shortest path queries:
three_hop(&graph.arrange_by_key(), &graph.arrange_by_key(), &q4)
.probe_with(&mut probe);
}
(q1_input, q2_input, q3_input, q4_input, state_input, graph_input)
});
let seed: &[_] = &[1, 2, 3, index];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for edge deletions
let seed: &[_] = &[1, 2, 4, index];
let mut rng3: StdRng = SeedableRng::from_seed(seed); // rng for query additions
let mut rng4: StdRng = SeedableRng::from_seed(seed); // rng for q1 deletions
let seed: &[_] = &[1, 2, 5, index];
let mut rng5: StdRng = SeedableRng::from_seed(seed); // rng for query additions
let mut rng6: StdRng = SeedableRng::from_seed(seed); // rng for q1 deletions
let seed: &[_] = &[1, 2, 6, index];
let mut rng7: StdRng = SeedableRng::from_seed(seed); // rng for query additions
let mut rng8: StdRng = SeedableRng::from_seed(seed); // rng for q1 deletions
let seed: &[_] = &[1, 2, 7, index];
let mut rng9: StdRng = SeedableRng::from_seed(seed); // rng for query additions
let mut rng0: StdRng = SeedableRng::from_seed(seed); // rng for q1 deletions
if index == 0 { println!("performing workload on random graph with {} nodes, {} edges:", nodes, edges); }
let worker_edges = edges/peers + if index < (edges % peers) { 1 } else { 0 };
for _ in 0 .. worker_edges {
graph.insert((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)));
}
for node in 0 .. nodes {
if node % peers == index {
state.insert((node, node));
}
}
let worker_window = queries/peers + if index < (queries % peers) { 1 } else { 0 };
for _ in 0 .. worker_window {
q1.insert(rng3.gen_range(0, nodes));
q2.insert(rng5.gen_range(0, nodes));
q3.insert(rng7.gen_range(0, nodes));
q4.insert((rng9.gen_range(0, nodes), rng9.gen_range(0, nodes)));
}
q1.advance_to(1); q1.flush(); // q1 queries start now.
q2.advance_to(1); q2.flush(); // q2 queries start here.
q3.advance_to(1); q3.flush(); // q3 queries start here.
q4.advance_to(1); q4.flush(); // q4 queries start here.
state.advance_to(usize::max_value()); state.flush();
graph.advance_to(1); graph.flush();
// finish graph loading work.
while probe.less_than(graph.time()) { worker.step(); }
if index == 0 { println!("{:?}\tgraph loaded", timer.elapsed()); }
let requests_per_sec = rate / 2;
let ns_per_request = 1_000_000_000 / requests_per_sec;
let mut request_counter = peers + index; // skip first request for each.
let mut ack_counter = peers + index;
let mut inserted_ns = 1;
let timer = ::std::time::Instant::now();
let mut counts = vec![[0usize; 16]; 64];
let ack_target = goal * rate;
while ack_counter < ack_target {
// Open-loop latency-throughput test, parameterized by offered rate `ns_per_request`.
let elapsed = timer.elapsed();
let elapsed_ns: usize = (elapsed.as_secs() * 1_000_000_000 + (elapsed.subsec_nanos() as u64)) as usize;
// Determine completed ns.
let acknowledged_ns: usize = probe.with_frontier(|frontier| frontier[0]);
// any un-recorded measurements that are complete should be recorded.
while (ack_counter * ns_per_request) < acknowledged_ns && ack_counter < ack_target {
let requested_at = ack_counter * ns_per_request;
let count_index = (elapsed_ns - requested_at).next_power_of_two().trailing_zeros() as usize;
if ack_counter > ack_target / 2 {
let low_bits = ((elapsed_ns - requested_at) >> (count_index - 5)) & 0xF;
counts[count_index][low_bits as usize] += 1;
}
ack_counter += peers;
}
// Now, should we introduce more records before stepping the worker?
//
// Thinking: inserted_ns - acknowledged_ns is some amount of time that
// is currently outstanding in the system, and we needn't advance our
// inputs unless by this order of magnitude.
//
// The more sophisticated plan is: we compute the next power of two
// greater than inserted_ns - acknowledged_ns and look for the last
// multiple of this number in the interval [inserted_ns, elapsed_ns].
// If such a multiple exists, we introduce records to that point and
// advance the input.
// let scale = (inserted_ns - acknowledged_ns).next_power_of_two();
// let target_ns = elapsed_ns & !(scale - 1);
let mut target_ns = elapsed_ns & !((1 << 20) - 1);
// let mut target_ns = if acknowledged_ns >= inserted_ns { elapsed_ns } else { inserted_ns };
if target_ns > inserted_ns + 1_000_000_000 { target_ns = inserted_ns + 1_000_000_000; }
if inserted_ns < target_ns {
while (request_counter * ns_per_request) < target_ns {
if (request_counter / peers) % 2 == 0 {
graph.advance_to(request_counter * ns_per_request);
graph.insert((rng1.gen_range(0, nodes),rng1.gen_range(0, nodes)));
graph.remove((rng2.gen_range(0, nodes),rng2.gen_range(0, nodes)));
}
else {
match ((request_counter / peers) / 2) % 4 {
0 => {
q1.advance_to(request_counter * ns_per_request);
q1.insert(rng3.gen_range(0, nodes));
q1.remove(rng4.gen_range(0, nodes));
},
1 => {
q2.advance_to(request_counter * ns_per_request);
q2.insert(rng5.gen_range(0, nodes));
q2.remove(rng6.gen_range(0, nodes));
},
2 => {
q3.advance_to(request_counter * ns_per_request);
q3.insert(rng7.gen_range(0, nodes));
q3.remove(rng8.gen_range(0, nodes));
},
3 => {
q4.advance_to(request_counter * ns_per_request);
q4.insert((rng9.gen_range(0, nodes),rng9.gen_range(0, nodes)));
q4.remove((rng0.gen_range(0, nodes),rng0.gen_range(0, nodes)));
},
_ => { unimplemented!() }
}
}
request_counter += peers;
}
graph.advance_to(target_ns); graph.flush();
q1.advance_to(target_ns); q1.flush();
q2.advance_to(target_ns); q2.flush();
q3.advance_to(target_ns); q3.flush();
q4.advance_to(target_ns); q4.flush();
inserted_ns = target_ns;
}
worker.step();
}
if index == 0 {
let mut results = Vec::new();
let total = counts.iter().map(|x| x.iter().sum::<usize>()).sum();
let mut sum = 0;
for index in (10 .. counts.len()).rev() {
for sub in (0 .. 16).rev() {
if sum > 0 && sum < total {
let latency = (1 << (index-1)) + (sub << (index-5));
let fraction = (sum as f64) / (total as f64);
results.push((latency, fraction));
}
sum += counts[index][sub];
}
}
for (latency, fraction) in results.drain(..).rev() {
println!("{}\t{}", latency, fraction);
}
}
}).unwrap();
}
use differential_dataflow::trace::implementations::ord::OrdValSpine as DefaultValTrace;
use differential_dataflow::operators::arrange::TraceAgent;
use differential_dataflow::operators::arrange::Arranged;
type Arrange<G, K, V, R> = Arranged<G, TraceAgent<DefaultValTrace<K, V, <G as ScopeParent>::Timestamp, R>>>;
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn three_hop<G: Scope>(
forward_graph: &Arrange<G, Node, Node, isize>,
reverse_graph: &Arrange<G, Node, Node, isize>,
goals: &Collection<G, (Node, Node)>) -> Collection<G, ((Node, Node), u32)>
where G::Timestamp: Lattice+Ord {
let sources = goals.map(|(x,_)| x);
let targets = goals.map(|(_,y)| y);
// Q3: Two-hop lookups on `state`:
let forward0 = sources.map(|x| (x, (x,0)));
let forward1 = forward0.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1))));
let forward2 = forward1.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1))));
let reverse0 = targets.map(|x| (x, (x,0)));
let reverse1 = reverse0.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1))));
let reverse2 = reverse1.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1))));
let forward = forward0.concat(&forward1).concat(&forward2);
let reverse = reverse0.concat(&reverse1).concat(&reverse2);
forward
.join_map(&reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2))
.reduce(|_st,input,output| output.push((*input[0].0,1)))
}
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn _bidijkstra<G: Scope>(
forward_graph: &Arrange<G, Node, Node, isize>,
reverse_graph: &Arrange<G, Node, Node, isize>,
goals: &Collection<G, (Node, Node)>) -> Collection<G, ((Node, Node), u32)>
where G::Timestamp: Lattice+Ord {
goals.scope().iterative::<Iter,_,_>(|inner| {
// Our plan is to start evolving distances from both sources and destinations.
// The evolution from a source or destination should continue as long as there
// is a corresponding destination or source that has not yet been reached.
// forward and reverse (node, (root, dist))
let forward = Variable::new_from(goals.map(|(x,_)| (x,(x,0))).enter(inner), Product::new(Default::default(), 1));
let reverse = Variable::new_from(goals.map(|(_,y)| (y,(y,0))).enter(inner), Product::new(Default::default(), 1));
let goals = goals.enter(inner);
let forward_graph = forward_graph.enter(inner);
let reverse_graph = reverse_graph.enter(inner);
// Let's determine which (src, dst) pairs are ready to return.
//
// done(src, dst) := forward(src, med), reverse(dst, med), goal(src, dst).
//
// This is a cyclic join, which should scare us a bunch.
let reached =
forward
.join_map(&reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2))
.reduce(|_key, s, t| t.push((*s[0].0, 1)))
.semijoin(&goals);
let active =
reached
.negate()
.map(|(srcdst,_)| srcdst)
.concat(&goals)
.consolidate();
// Let's expand out forward queries that are active.
let forward_active = active.map(|(x,_y)| x).distinct();
let forward_next =
forward
.map(|(med, (src, dist))| (src, (med, dist)))
.semijoin(&forward_active)
.map(|(src, (med, dist))| (med, (src, dist)))
.join_core(&forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1))))
.concat(&forward)
.map(|(next, (src, dist))| ((next, src), dist))
.reduce(|_key, s, t| t.push((*s[0].0, 1)))
.map(|((next, src), dist)| (next, (src, dist)));
forward.set(&forward_next);
// Let's expand out reverse queries that are active.
let reverse_active = active.map(|(_x,y)| y).distinct();
let reverse_next =
reverse
.map(|(med, (rev, dist))| (rev, (med, dist)))
.semijoin(&reverse_active)
.map(|(rev, (med, dist))| (med, (rev, dist)))
.join_core(&reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1))))
.concat(&reverse)
.map(|(next, (rev, dist))| ((next, rev), dist))
.reduce(|_key, s, t| t.push((*s[0].0, 1)))
.map(|((next,rev), dist)| (next, (rev, dist)));
reverse.set(&reverse_next);
reached.leave()
})
}

View File

@ -0,0 +1,307 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
extern crate core_affinity;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::*;
use timely::dataflow::operators::probe::Handle;
use timely::order::Product;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::operators::iterate::Variable;
use differential_dataflow::operators::arrange::ArrangeByKey;
use differential_dataflow::operators::arrange::ArrangeBySelf;
type Node = usize;
type Iter = usize;
fn main() {
let nodes: usize = std::env::args().nth(1).unwrap().parse().unwrap();
let edges: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let batch: usize = std::env::args().nth(3).unwrap().parse().unwrap();
// let inspect: bool = std::env::args().any(|x| x == "inspect");
// Our setting involves four read query types, and two updatable base relations.
//
// Q1: Point lookup: reads "state" associated with a node.
// Q2: One-hop lookup: reads "state" associated with neighbors of a node.
// Q3: Two-hop lookup: reads "state" associated with n-of-n's of a node.
// Q4: Shortest path: reports hop count between two query nodes.
//
// R1: "State": a pair of (node, T) for some type T that I don't currently know.
// R2: "Graph": pairs (node, node) indicating linkage between the two nodes.
timely::execute_from_args(std::env::args().skip(3), move |worker| {
let index = worker.index();
let peers = worker.peers();
let timer = ::std::time::Instant::now();
let core_ids = core_affinity::get_core_ids().unwrap();
core_affinity::set_for_current(core_ids[index]);
// define BFS dataflow; return handles to roots and edges inputs
let mut probe = Handle::new();
let (mut q1, mut q2, mut q3, mut q4, mut state, mut graph) = worker.dataflow(|scope| {
let (q1_input, q1) = scope.new_collection();
let (q2_input, q2) = scope.new_collection();
let (q3_input, q3) = scope.new_collection();
let (q4_input, q4) = scope.new_collection();
let (state_input, state) = scope.new_collection();
let (graph_input, graph) = scope.new_collection();
let state_indexed = state.arrange_by_key();
let graph_indexed = graph.map(|(src, dst)| (dst, src))
.concat(&graph)
.arrange_by_key();
// Q1: Point lookups on `state`:
q1 .arrange_by_self()
.join_core(&state_indexed, |&query, &(), &state| Some((query, state)))
// .filter(move |_| inspect)
// .inspect(|x| println!("Q1: {:?}", x))
.probe_with(&mut probe);
// Q2: One-hop lookups on `state`:
q2 .arrange_by_self()
.join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query)))
.join_core(&state_indexed, |_friend, &query, &state| Some((query, state)))
// .filter(move |_| inspect)
// .inspect(|x| println!("Q2: {:?}", x))
.probe_with(&mut probe);
// Q3: Two-hop lookups on `state`:
q3 .arrange_by_self()
.join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query)))
.join_core(&graph_indexed, |_friend, &query, &friend2| Some((friend2, query)))
.join_core(&state_indexed, |_friend2, &query, &state| Some((query, state)))
// .filter(move |_| inspect)
// .consolidate()
// .inspect(|x| println!("Q3: {:?}", x))
.probe_with(&mut probe);
// Q4: Shortest path queries:
three_hop(&graph_indexed, &graph_indexed, &q4)
// .filter(move |_| inspect)
// .inspect(|x| println!("Q4: {:?}", x))
.probe_with(&mut probe);
(q1_input, q2_input, q3_input, q4_input, state_input, graph_input)
});
let seed1: &[_] = &[1, 2, 3, index];
let mut rng1: StdRng = SeedableRng::from_seed(seed1); // rng for edge additions
// let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for edge deletions
let seed2: &[_] = &[1, 2, 4, index];
let mut rng3: StdRng = SeedableRng::from_seed(seed2); // rng for queries
if index == 0 { println!("performing workload on random graph with {} nodes, {} edges:", nodes, edges); }
let worker_edges = edges/peers + if index < (edges % peers) { 1 } else { 0 };
for _ in 0 .. worker_edges {
graph.insert((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)));
}
for node in 0 .. nodes {
if node % peers == index {
state.insert((node, node));
}
}
q1.advance_to(1); q1.flush(); // q1 queries start now.
q2.advance_to(1001); q2.flush(); // q2 queries start here.
q3.advance_to(2001); q3.flush(); // q3 queries start here.
q4.advance_to(3001); q4.flush(); // q4 queries start here.
state.advance_to(4001); state.flush();
graph.advance_to(4001); graph.flush();
// state.close(); // no changes to state.
// graph.close(); // no changes to graph.
// finish graph loading work.
while probe.less_than(q1.time()) { worker.step(); }
if index == 0 { println!("{:?}\tgraph loaded", timer.elapsed()); }
let worker_batch = batch/peers + if index < batch % peers { 1 } else { 0 };
// Q1 testing:
let mut list = Vec::with_capacity(worker_batch);
let timer_q1 = ::std::time::Instant::now();
for round in 1 .. 1001 {
for _ in 0 .. worker_batch {
list.push(rng3.gen_range(0, nodes));
}
for &thing in list.iter() { q1.insert(thing); }
q1.advance_to(round);
for &thing in list.iter() { q1.remove(thing); }
q1.flush();
while probe.less_than(q1.time()) { worker.step(); }
}
if index == 0 { println!("{:?}\tq1 eval complete; avg: {:?}", timer.elapsed(), timer_q1.elapsed()/1000); }
q1.close();
// Q2 testing:
let mut list = Vec::with_capacity(worker_batch);
let timer_q2 = ::std::time::Instant::now();
for round in 1001 .. 2001 {
for _ in 0 .. worker_batch {
list.push(rng3.gen_range(0, nodes));
}
for &thing in list.iter() { q2.insert(thing); }
q2.advance_to(round);
for &thing in list.iter() { q2.remove(thing); }
q2.flush();
while probe.less_than(q2.time()) { worker.step(); }
}
if index == 0 { println!("{:?}\tq2 eval complete; avg: {:?}", timer.elapsed(), timer_q2.elapsed()/1000); }
q2.close();
// Q3 testing:
let mut list = Vec::with_capacity(worker_batch);
let timer_q3 = ::std::time::Instant::now();
for round in 2001 .. 3001 {
for _ in 0 .. worker_batch {
list.push(rng3.gen_range(0, nodes));
}
for &thing in list.iter() { q3.insert(thing); }
q3.advance_to(round);
for &thing in list.iter() { q3.remove(thing); }
q3.flush();
while probe.less_than(q3.time()) { worker.step(); }
}
if index == 0 { println!("{:?}\tq3 eval complete; avg: {:?}", timer.elapsed(), timer_q3.elapsed()/1000); }
q3.close();
// Q4 testing:
let mut list = Vec::with_capacity(worker_batch);
let timer_q4 = ::std::time::Instant::now();
for round in 3001 .. 4001 {
for _ in 0 .. worker_batch {
list.push((rng3.gen_range(0, nodes), rng3.gen_range(0, nodes)));
}
for &thing in list.iter() { q4.insert(thing); }
q4.advance_to(round);
for &thing in list.iter() { q4.remove(thing); }
q4.flush();
while probe.less_than(q4.time()) { worker.step(); }
}
if index == 0 { println!("{:?}\tq4 eval complete; avg: {:?}", timer.elapsed(), timer_q4.elapsed()/1000); }
q4.close();
}).unwrap();
}
use differential_dataflow::trace::implementations::ord::OrdValSpine as DefaultValTrace;
use differential_dataflow::operators::arrange::TraceAgent;
use differential_dataflow::operators::arrange::Arranged;
type Arrange<G, K, V, R> = Arranged<G, TraceAgent<DefaultValTrace<K, V, <G as ScopeParent>::Timestamp, R>>>;
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn three_hop<G: Scope>(
forward_graph: &Arrange<G, Node, Node, isize>,
reverse_graph: &Arrange<G, Node, Node, isize>,
goals: &Collection<G, (Node, Node)>) -> Collection<G, ((Node, Node), u32)>
where G::Timestamp: Lattice+Ord {
let sources = goals.map(|(x,_)| x);
let targets = goals.map(|(_,y)| y);
// Q3: Two-hop lookups on `state`:
let forward0 = sources.map(|x| (x, (x,0)));
let forward1 = forward0.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1))));
let forward2 = forward1.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1))));
let reverse0 = targets.map(|x| (x, (x,0)));
let reverse1 = reverse0.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1))));
let reverse2 = reverse1.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1))));
let forward = forward0.concat(&forward1).concat(&forward2);
let reverse = reverse0.concat(&reverse1).concat(&reverse2);
forward
.join_map(&reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2))
.reduce(|_st,input,output| output.push((*input[0].0,1)))
}
// returns pairs (n, s) indicating node n can be reached from a root in s steps.
fn _bidijkstra<G: Scope>(
forward_graph: &Arrange<G, Node, Node, isize>,
reverse_graph: &Arrange<G, Node, Node, isize>,
goals: &Collection<G, (Node, Node)>) -> Collection<G, ((Node, Node), u32)>
where G::Timestamp: Lattice+Ord {
goals.scope().iterative::<Iter,_,_>(|inner| {
// Our plan is to start evolving distances from both sources and destinations.
// The evolution from a source or destination should continue as long as there
// is a corresponding destination or source that has not yet been reached.
// forward and reverse (node, (root, dist))
let forward = Variable::new_from(goals.map(|(x,_)| (x,(x,0))).enter(inner), Product::new(Default::default(), 1));
let reverse = Variable::new_from(goals.map(|(_,y)| (y,(y,0))).enter(inner), Product::new(Default::default(), 1));
let goals = goals.enter(inner);
let forward_graph = forward_graph.enter(inner);
let reverse_graph = reverse_graph.enter(inner);
// Let's determine which (src, dst) pairs are ready to return.
//
// done(src, dst) := forward(src, med), reverse(dst, med), goal(src, dst).
//
// This is a cyclic join, which should scare us a bunch.
let reached =
forward
.join_map(&reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2))
.reduce(|_key, s, t| t.push((*s[0].0, 1)))
.semijoin(&goals);
let active =
reached
.negate()
.map(|(srcdst,_)| srcdst)
.concat(&goals)
.consolidate();
// Let's expand out forward queries that are active.
let forward_active = active.map(|(x,_y)| x).distinct();
let forward_next =
forward
.map(|(med, (src, dist))| (src, (med, dist)))
.semijoin(&forward_active)
.map(|(src, (med, dist))| (med, (src, dist)))
.join_core(&forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1))))
.concat(&forward)
.map(|(next, (src, dist))| ((next, src), dist))
.reduce(|_key, s, t| t.push((*s[0].0, 1)))
.map(|((next, src), dist)| (next, (src, dist)));
forward.set(&forward_next);
// Let's expand out reverse queries that are active.
let reverse_active = active.map(|(_x,y)| y).distinct();
let reverse_next =
reverse
.map(|(med, (rev, dist))| (rev, (med, dist)))
.semijoin(&reverse_active)
.map(|(rev, (med, dist))| (med, (rev, dist)))
.join_core(&reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1))))
.concat(&reverse)
.map(|(next, (rev, dist))| ((next, rev), dist))
.reduce(|_key, s, t| t.push((*s[0].0, 1)))
.map(|((next,rev), dist)| (next, (rev, dist)));
reverse.set(&reverse_next);
reached.leave()
})
}

View File

@ -0,0 +1,148 @@
extern crate graph_map;
use graph_map::GraphMMap;
fn main() {
let filename = std::env::args().nth(1).expect("Must supply filename");
let rootnode = std::env::args().nth(2).expect("Must supply root node").parse().expect("Invalid root node");
let graph = GraphMMap::new(&filename);
let timer = ::std::time::Instant::now();
breadth_first(&graph, rootnode);
println!("{:?}\tbreadth_first", timer.elapsed());
let timer = ::std::time::Instant::now();
breadth_first_hash(&graph, rootnode);
println!("{:?}\tbreadth_first_hash", timer.elapsed());
let timer = ::std::time::Instant::now();
union_find(&graph);
println!("{:?}\tunion_find", timer.elapsed());
let timer = ::std::time::Instant::now();
union_find_hash(&graph);
println!("{:?}\tunion_find_hash", timer.elapsed());
}
fn breadth_first(graph: &GraphMMap, root: u32) {
let nodes = graph.nodes() as u32;
let mut reached = vec![false; nodes as usize];
let mut buff1 = Vec::with_capacity(nodes as usize);
let mut buff2 = Vec::with_capacity(nodes as usize);
reached[root as usize] = true;
buff1.push(root);
while !buff1.is_empty() {
buff1.sort_unstable(); // useful here, not for hashset tho.
for node in buff1.drain(..) {
for &edge in graph.edges(node as usize) {
unsafe {
if !*reached.get_unchecked(edge as usize) {
*reached.get_unchecked_mut(edge as usize) = true;
buff2.push(edge);
}
}
}
}
::std::mem::swap(&mut buff1, &mut buff2);
}
}
fn breadth_first_hash(graph: &GraphMMap, root: u32) {
use std::collections::HashSet;
let nodes = graph.nodes() as u32;
let mut reached = HashSet::new();
let mut buff1 = Vec::with_capacity(nodes as usize);
let mut buff2 = Vec::with_capacity(nodes as usize);
reached.insert(root);
buff1.push(root);
while !buff1.is_empty() {
for node in buff1.drain(..) {
for &edge in graph.edges(node as usize) {
if !reached.contains(&edge) {
reached.insert(edge);
buff2.push(edge);
}
}
}
::std::mem::swap(&mut buff1, &mut buff2);
}
}
fn union_find(graph: &GraphMMap) {
let nodes = graph.nodes() as u32;
let mut roots: Vec<u32> = (0..nodes).collect(); // u32 works, and is smaller than uint/u64
let mut ranks: Vec<u8> = vec![0u8; nodes as usize]; // u8 should be large enough (n < 2^256)
for node in 0 .. graph.nodes() {
for &edge in graph.edges(node) {
let mut x = node as u32;
let mut y = edge as u32;
// x = roots[x as usize];
// y = roots[y as usize];
x = unsafe { *roots.get_unchecked(x as usize) };
y = unsafe { *roots.get_unchecked(y as usize) };
// while x != roots[x as usize] { x = roots[x as usize]; }
// while y != roots[y as usize] { y = roots[y as usize]; }
unsafe { while x != *roots.get_unchecked(x as usize) { x = *roots.get_unchecked(x as usize); } }
unsafe { while y != *roots.get_unchecked(y as usize) { y = *roots.get_unchecked(y as usize); } }
if x != y {
unsafe {
match ranks[x as usize].cmp(&ranks[y as usize]) {
std::cmp::Ordering::Less => *roots.get_unchecked_mut(x as usize) = y as u32,
std::cmp::Ordering::Greater => *roots.get_unchecked_mut(y as usize) = x as u32,
std::cmp::Ordering::Equal => { *roots.get_unchecked_mut(y as usize) = x as u32;
*ranks.get_unchecked_mut(x as usize) += 1 },
}
}
}
}
}
}
fn union_find_hash(graph: &GraphMMap) {
use std::collections::HashMap;
let nodes = graph.nodes() as u32;
let mut roots: HashMap<u32,u32> = (0..nodes).map(|x| (x,x)).collect();
let mut ranks: HashMap<u32,u8> = (0..nodes).map(|x| (x,0)).collect();
for node in 0 .. graph.nodes() {
for &edge in graph.edges(node) {
let mut x = node as u32;
let mut y = edge as u32;
x = roots[&x];
y = roots[&y];
while x != roots[&x] { x = roots[&x]; }
while y != roots[&y] { y = roots[&y]; }
if x != y {
match ranks[&x].cmp(&ranks[&y]) {
std::cmp::Ordering::Less => *roots.get_mut(&x).unwrap() = y,
std::cmp::Ordering::Greater => *roots.get_mut(&y).unwrap() = x,
std::cmp::Ordering::Equal => { *roots.get_mut(&y).unwrap() = x; *ranks.get_mut(&x).unwrap() += 1; },
}
}
}
}
}

View File

@ -0,0 +1,211 @@
extern crate timely;
extern crate differential_dataflow;
extern crate graph_map;
extern crate core_affinity;
use std::rc::Rc;
use timely::dataflow::*;
use timely::order::Product;
use timely::dataflow::operators::ToStream;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::operators::arrange::ArrangeByKey;
use differential_dataflow::operators::arrange::ArrangeBySelf;
use differential_dataflow::operators::iterate::SemigroupVariable;
use differential_dataflow::trace::implementations::spine_fueled::Spine;
use differential_dataflow::AsCollection;
use graph_map::GraphMMap;
type Node = u32;
type Iter = u32;
type Diff = i32;
// use differential_dataflow::trace::implementations::graph::GraphBatch;
// type GraphTrace = Spine<Node, Node, (), isize, Rc<GraphBatch<Node>>>;
use differential_dataflow::trace::implementations::ord::OrdValBatch;
type GraphTrace = Spine<Node, Node, (), Diff, Rc<OrdValBatch<Node, Node, (), Diff>>>;
fn main() {
let filename = std::env::args().nth(1).expect("Must supply filename");
let rootnode = std::env::args().nth(2).expect("Must supply root node").parse().expect("Invalid root node");
timely::execute_from_args(std::env::args(), move |worker| {
let index = worker.index();
let peers = worker.peers();
let timer = ::std::time::Instant::now();
let core_ids = core_affinity::get_core_ids().unwrap();
core_affinity::set_for_current(core_ids[index % core_ids.len()]);
// Phase 1: Forward index.
let mut forward = worker.dataflow(|scope| {
let graph = GraphMMap::new(&filename);
let nodes = graph.nodes();
let edges = (0..nodes).filter(move |node| node % peers == index)
.flat_map(move |node| {
let vec = graph.edges(node).to_vec();
vec.into_iter().map(move |edge| ((node as Node, edge as Node), (), 1))
})
.to_stream(scope)
.as_collection();
edges.arrange_by_key().trace
});
while worker.step() { }
if index == 0 { println!("{:?}\tphase 1:\tforward graph indexed", timer.elapsed()); }
let timer = ::std::time::Instant::now();
// Phase 2: Reachability.
let mut roots = worker.dataflow(|scope| {
let (roots_input, roots) = scope.new_collection();
reach(&mut forward, roots);
roots_input
});
if index == 0 { roots.update(rootnode, 1); }
roots.close();
while worker.step() { }
if index == 0 { println!("{:?}\tphase 2:\treach complete", timer.elapsed()); }
let timer = ::std::time::Instant::now();
// Phase 3: Breadth-first distance labeling.
let mut roots = worker.dataflow(|scope| {
let (roots_input, roots) = scope.new_collection();
bfs(&mut forward, roots);
roots_input
});
if index == 0 { roots.update(rootnode, 1); }
roots.close();
while worker.step() { }
if index == 0 { println!("{:?}\tphase 3:\tbfs complete", timer.elapsed()); }
let timer = ::std::time::Instant::now();
// Phase 4: Reverse index.
let mut reverse = worker.dataflow(|scope| {
forward
.import(scope)
.as_collection(|&k,&v| (v,k))
.arrange_by_key()
.trace
});
while worker.step() { }
if index == 0 { println!("{:?}\tphase 4:\treverse graph indexed", timer.elapsed()); }
let timer = ::std::time::Instant::now();
// Phase 5: Undirected connectivity.
worker.dataflow(|scope| { connected_components(scope, &mut forward, &mut reverse); });
while worker.step() { }
if index == 0 { println!("{:?}\tphase 5:\tcc complete", timer.elapsed()); }
}).unwrap();
}
use differential_dataflow::operators::arrange::TraceAgent;
type TraceHandle = TraceAgent<GraphTrace>;
fn reach<G: Scope<Timestamp = ()>> (
graph: &mut TraceHandle,
roots: Collection<G, Node, Diff>
) -> Collection<G, Node, Diff> {
let graph = graph.import(&roots.scope());
roots.scope().iterative::<Iter,_,_>(|scope| {
let graph = graph.enter(scope);
let roots = roots.enter(scope);
let inner = SemigroupVariable::new(scope, Product::new(Default::default(), 1));
let result =
graph.join_core(&inner.arrange_by_self(), |_src,&dst,&()| Some(dst))
.concat(&roots)
.threshold_total(|_,_| 1);
inner.set(&result);
result.leave()
})
}
fn bfs<G: Scope<Timestamp = ()>> (
graph: &mut TraceHandle,
roots: Collection<G, Node, Diff>
) -> Collection<G, (Node, u32), Diff> {
let graph = graph.import(&roots.scope());
let roots = roots.map(|r| (r,0));
roots.scope().iterative::<Iter,_,_>(|scope| {
let graph = graph.enter(scope);
let roots = roots.enter(scope);
let inner = SemigroupVariable::new(scope, Product::new(Default::default(), 1));
let result =
graph.join_map(&inner, |_src,&dest,&dist| (dest, dist+1))
.concat(&roots)
.reduce(|_key, input, output| output.push((*input[0].0,1)));
inner.set(&result);
result.leave()
})
}
fn connected_components<G: Scope<Timestamp = ()>>(
scope: &mut G,
forward: &mut TraceHandle,
reverse: &mut TraceHandle,
) -> Collection<G, (Node, Node), Diff> {
let forward = forward.import(scope);
let reverse = reverse.import(scope);
// each edge (x,y) means that we need at least a label for the min of x and y.
let nodes_f = forward.flat_map_ref(|k,v| if k < v { Some(*k) } else { None });
let nodes_r = reverse.flat_map_ref(|k,v| if k < v { Some(*k) } else { None });
let nodes = nodes_f.concat(&nodes_r).consolidate().map(|x| (x,x));
scope.iterative(|scope| {
// import arrangements, nodes.
let forward = forward.enter(scope);
let reverse = reverse.enter(scope);
let nodes = nodes.enter(scope);
let inner = SemigroupVariable::new(scope, Product::new(Default::default(), 1));
let labels = inner.arrange_by_key();
let f_prop = labels.join_core(&forward, |_k,l,d| Some((*d,*l)));
let r_prop = labels.join_core(&reverse, |_k,l,d| Some((*d,*l)));
use timely::dataflow::operators::{Map, Concat, Delay};
let result =
nodes
.inner
.map_in_place(|dtr| (dtr.1).inner = 256 * ((((::std::mem::size_of::<Node>() * 8) as u32) - (dtr.0).1.leading_zeros())))
.concat(&inner.filter(|_| false).inner)
.delay(|dtr,_| dtr.1.clone())
.as_collection()
.concat(&f_prop)
.concat(&r_prop)
.reduce(|_, s, t| { t.push((*s[0].0, 1)); });
inner.set(&result);
result.leave()
})
}

View File

@ -0,0 +1,171 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
use std::rc::Rc;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::*;
use differential_dataflow::input::Input;
use differential_dataflow::Collection;
use differential_dataflow::operators::*;
use differential_dataflow::trace::Trace;
use differential_dataflow::operators::arrange::ArrangeByKey;
use differential_dataflow::operators::arrange::ArrangeBySelf;
use differential_dataflow::trace::implementations::spine_fueled::Spine;
type Node = usize;
use differential_dataflow::trace::implementations::ord::OrdValBatch;
// use differential_dataflow::trace::implementations::ord::OrdValSpine;
// type GraphTrace<N> = Spine<usize, N, (), isize, Rc<GraphBatch<N>>>;
type GraphTrace = Spine<Node, Node, (), isize, Rc<OrdValBatch<Node, Node, (), isize>>>;
fn main() {
let nodes: usize = std::env::args().nth(1).unwrap().parse().unwrap();
let edges: usize = std::env::args().nth(2).unwrap().parse().unwrap();
// Our setting involves four read query types, and two updatable base relations.
//
// Q1: Point lookup: reads "state" associated with a node.
// Q2: One-hop lookup: reads "state" associated with neighbors of a node.
// Q3: Two-hop lookup: reads "state" associated with n-of-n's of a node.
// Q4: Shortest path: reports hop count between two query nodes.
//
// R1: "State": a pair of (node, T) for some type T that I don't currently know.
// R2: "Graph": pairs (node, node) indicating linkage between the two nodes.
timely::execute_from_args(std::env::args().skip(3), move |worker| {
let index = worker.index();
let peers = worker.peers();
let timer = ::std::time::Instant::now();
let (mut graph, mut trace) = worker.dataflow(|scope| {
let (graph_input, graph) = scope.new_collection();
let graph_indexed = graph.arrange_by_key();
// let graph_indexed = graph.arrange_by_key();
(graph_input, graph_indexed.trace)
});
let seed: &[_] = &[1, 2, 3, index];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
// let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for edge deletions
if index == 0 { println!("performing workload on random graph with {} nodes, {} edges:", nodes, edges); }
let worker_edges = edges/peers + if index < (edges % peers) { 1 } else { 0 };
for _ in 0 .. worker_edges {
graph.insert((rng1.gen_range(0, nodes) as Node, rng1.gen_range(0, nodes) as Node));
}
graph.close();
while worker.step() { }
if index == 0 { println!("{:?}\tgraph loaded", timer.elapsed()); }
// Phase 2: Reachability.
let mut roots = worker.dataflow(|scope| {
let (roots_input, roots) = scope.new_collection();
reach(&mut trace, roots);
roots_input
});
if index == 0 { roots.insert(0); }
roots.close();
while worker.step() { }
if index == 0 { println!("{:?}\treach complete", timer.elapsed()); }
// Phase 3: Breadth-first distance labeling.
let mut roots = worker.dataflow(|scope| {
let (roots_input, roots) = scope.new_collection();
bfs(&mut trace, roots);
roots_input
});
if index == 0 { roots.insert(0); }
roots.close();
while worker.step() { }
if index == 0 { println!("{:?}\tbfs complete", timer.elapsed()); }
}).unwrap();
}
// use differential_dataflow::trace::implementations::ord::OrdValSpine;
use differential_dataflow::operators::arrange::TraceAgent;
type TraceHandle = TraceAgent<GraphTrace>;
fn reach<G: Scope<Timestamp = ()>> (
graph: &mut TraceHandle,
roots: Collection<G, Node>
) -> Collection<G, Node> {
let graph = graph.import(&roots.scope());
roots.iterate(|inner| {
let graph = graph.enter(&inner.scope());
let roots = roots.enter(&inner.scope());
// let reach = inner.concat(&roots).distinct_total().arrange_by_self();
// graph.join_core(&reach, |_src,&dst,&()| Some(dst))
graph.join_core(&inner.arrange_by_self(), |_src,&dst,&()| Some(dst))
.concat(&roots)
.distinct_total()
})
}
fn bfs<G: Scope<Timestamp = ()>> (
graph: &mut TraceHandle,
roots: Collection<G, Node>
) -> Collection<G, (Node, u32)> {
let graph = graph.import(&roots.scope());
let roots = roots.map(|r| (r,0));
roots.iterate(|inner| {
let graph = graph.enter(&inner.scope());
let roots = roots.enter(&inner.scope());
graph.join_map(&inner, |_src,&dest,&dist| (dest, dist+1))
.concat(&roots)
.reduce(|_key, input, output| output.push((*input[0].0,1)))
})
}
// fn connected_components<G: Scope<Timestamp = ()>>(
// graph: &mut TraceHandle<Node>
// ) -> Collection<G, (Node, Node)> {
// // each edge (x,y) means that we need at least a label for the min of x and y.
// let nodes =
// graph
// .as_collection(|&k,&v| {
// let min = std::cmp::min(k,v);
// (min, min)
// })
// .consolidate();
// // each edge should exist in both directions.
// let edges = edges.map_in_place(|x| mem::swap(&mut x.0, &mut x.1))
// .concat(&edges);
// // don't actually use these labels, just grab the type
// nodes.filter(|_| false)
// .iterate(|inner| {
// let edges = edges.enter(&inner.scope());
// let nodes = nodes.enter_at(&inner.scope(), |r| 256 * (64 - r.1.leading_zeros() as u64));
// inner.join_map(&edges, |_k,l,d| (*d,*l))
// .concat(&nodes)
// .group(|_, s, t| { t.push((*s[0].0, 1)); } )
// })
// }

View File

@ -0,0 +1,164 @@
extern crate timely;
extern crate graph_map;
extern crate differential_dataflow;
use std::io::{BufRead, BufReader};
use std::fs::File;
// use timely::progress::nested::product::Product;
// use timely::dataflow::operators::{Accumulate, Inspect};
use differential_dataflow::input::Input;
// use differential_dataflow::trace::Trace;
// use differential_dataflow::trace::implementations::ord::OrdValSpine;
use differential_dataflow::operators::*;
use differential_dataflow::operators::arrange::ArrangeByKey;
fn main() {
timely::execute_from_args(std::env::args(), move |worker| {
let timer = ::std::time::Instant::now();
let peers = worker.peers();
let index = worker.index();
let (mut nodes, mut edges, probe) = worker.dataflow(|scope| {
// let timer = timer.clone();
let (n_handle, nodes) = scope.new_collection();
let (e_handle, edges) = scope.new_collection();
let edges = edges.arrange_by_key();
// a N c <- a N b && b E c
// N(a,c) <- N(a,b), E(b, c)
let probe =
nodes
.filter(|_| false)
.iterate(|inner| {
let nodes = nodes.enter(&inner.scope());
let edges = edges.enter(&inner.scope());
inner
.map(|(a,b)| (b,a))
.join_core(&edges, |_b,&a,&c| Some((a,c)))
.concat(&nodes)
.distinct()
})
.probe();
(n_handle, e_handle, probe)
});
if index == 0 { println!("{:?}:\tDataflow assembled", timer.elapsed()); }
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
let file = BufReader::new(File::open(filename).unwrap());
for readline in file.lines() {
let line = readline.ok().expect("read error");
if !line.starts_with('#') && line.len() > 0 {
let mut elts = line[..].split_whitespace();
let src: u32 = elts.next().unwrap().parse().ok().expect("malformed src");
if (src as usize) % peers == index {
let dst: u32 = elts.next().unwrap().parse().ok().expect("malformed dst");
let typ: &str = elts.next().unwrap();
match typ {
"n" => { nodes.insert((src, dst)); },
"e" => { edges.insert((src, dst)); },
_ => { },
}
}
}
}
if index == 0 {
edges.close();
nodes.advance_to(1);
nodes.flush();
while probe.less_than(&nodes.time()) { worker.step(); }
if index == 0 { println!("{:?}:\tData loaded", timer.elapsed()); }
let mut counts = vec![[0u64; 16]; 64];
let filename = std::env::args().nth(1).unwrap();
let file = BufReader::new(File::open(filename).unwrap());
for readline in file.lines() {
let line = readline.ok().expect("read error");
if !line.starts_with('#') && line.len() > 0 {
let mut elts = line[..].split_whitespace();
let src: u32 = elts.next().unwrap().parse().ok().expect("malformed src");
if (src as usize) % peers == index {
let dst: u32 = elts.next().unwrap().parse().ok().expect("malformed dst");
let typ: &str = elts.next().unwrap();
match typ {
"n" => {
nodes.remove((src, dst));
let round = nodes.time() + 0;
nodes.advance_to(round + 1);
nodes.flush();
let timer = std::time::Instant::now();
while probe.less_than(&nodes.time()) { worker.step(); }
let elapsed = timer.elapsed();
let elapsed_ns = 1_000_000_000 * (elapsed.as_secs() as usize) + (elapsed.subsec_nanos() as usize);
let count_index = elapsed_ns.next_power_of_two().trailing_zeros() as usize;
let low_bits = (elapsed_ns >> (count_index - 5)) & 0xF;
counts[count_index][low_bits as usize] += 1;
// println!("elapsed: {:?}", elapsed);
// println!("round: {:?}", round);
if round == 999 {
let mut results = Vec::new();
let total = counts.iter().map(|x| x.iter().sum::<u64>()).sum();
let mut sum = 0;
for index in (10 .. counts.len()).rev() {
for sub in (0 .. 16).rev() {
if sum > 0 && sum < total {
let latency = (1 << (index-1)) + (sub << (index-5));
let fraction = (sum as f64) / (total as f64);
results.push((latency, fraction));
}
sum += counts[index][sub];
}
}
for (latency, fraction) in results.drain(..).rev() {
println!("{}\t{}", latency, fraction);
}
println!();
}
},
_ => { },
}
}
}
}
let mut results = Vec::new();
let total = counts.iter().map(|x| x.iter().sum::<u64>()).sum();
let mut sum = 0;
for index in (10 .. counts.len()).rev() {
for sub in (0 .. 16).rev() {
if sum > 0 && sum < total {
let latency = (1 << (index-1)) + (sub << (index-5));
let fraction = (sum as f64) / (total as f64);
results.push((latency, fraction));
}
sum += counts[index][sub];
}
}
for (latency, fraction) in results.drain(..).rev() {
println!("{}\t{}", latency, fraction);
}
println!("{:?}:\tComputation complete", timer.elapsed());
}
}).unwrap();
}

View File

@ -0,0 +1,102 @@
extern crate timely;
extern crate graph_map;
extern crate differential_dataflow;
use std::io::{BufRead, BufReader};
use std::fs::File;
use timely::dataflow::Scope;
use timely::order::Product;
use differential_dataflow::difference::Present;
use differential_dataflow::input::Input;
use differential_dataflow::trace::implementations::ord::OrdValSpine;
use differential_dataflow::operators::*;
use differential_dataflow::operators::arrange::Arrange;
use differential_dataflow::operators::iterate::SemigroupVariable;
type Node = u32;
type Time = ();
type Iter = u32;
type Offs = u32;
fn main() {
timely::execute_from_args(std::env::args(), move |worker| {
let timer = ::std::time::Instant::now();
let peers = worker.peers();
let index = worker.index();
let (mut nodes, mut edges) = worker.dataflow::<Time,_,_>(|scope| {
// let timer = timer.clone();
let (n_handle, nodes) = scope.new_collection();
let (e_handle, edges) = scope.new_collection();
let edges = edges.arrange::<OrdValSpine<_,_,_,_,Offs>>();
// a N c <- a N b && b E c
// N(a,c) <- N(a,b), E(b, c)
let reached =
nodes.scope().iterative::<Iter,_,_>(|inner| {
let nodes = nodes.enter(inner).map(|(a,b)| (b,a));
let edges = edges.enter(inner);
let labels = SemigroupVariable::new(inner, Product::new(Default::default(), 1));
let next =
labels.join_core(&edges, |_b, a, c| Some((*c, *a)))
.concat(&nodes)
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
// .distinct_total_core::<Diff>();
.threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None });
labels.set(&next);
next.leave()
});
reached
.map(|_| ())
.consolidate()
.inspect(|x| println!("{:?}", x))
;
(n_handle, e_handle)
});
if index == 0 { println!("{:?}:\tDataflow assembled", timer.elapsed()); }
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
let file = BufReader::new(File::open(filename).unwrap());
for readline in file.lines() {
let line = readline.ok().expect("read error");
if !line.starts_with('#') && line.len() > 0 {
let mut elts = line[..].split_whitespace();
let src: Node = elts.next().unwrap().parse().ok().expect("malformed src");
if (src as usize) % peers == index {
let dst: Node = elts.next().unwrap().parse().ok().expect("malformed dst");
let typ: &str = elts.next().unwrap();
match typ {
"n" => { nodes.update((src, dst), Present); },
"e" => { edges.update((src, dst), Present); },
unk => { panic!("unknown type: {}", unk)},
}
}
}
}
if index == 0 { println!("{:?}:\tData loaded", timer.elapsed()); }
nodes.close();
edges.close();
while worker.step() { }
if index == 0 { println!("{:?}:\tComputation complete", timer.elapsed()); }
}).unwrap();
}

View File

@ -0,0 +1,282 @@
extern crate timely;
extern crate graph_map;
extern crate differential_dataflow;
use std::io::{BufRead, BufReader};
use std::fs::File;
use timely::dataflow::Scope;
use timely::order::Product;
use differential_dataflow::operators::iterate::SemigroupVariable;
use differential_dataflow::Collection;
use differential_dataflow::input::Input;
use differential_dataflow::operators::*;
use differential_dataflow::operators::arrange::Arrange;
use differential_dataflow::trace::implementations::ord::{OrdValSpine, OrdKeySpine};
use differential_dataflow::difference::Present;
type Node = u32;
type Time = ();
type Iter = u32;
type Offs = u32;
fn main() {
if std::env::args().any(|x| x == "optimized") {
optimized();
}
else {
unoptimized();
}
}
fn unoptimized() {
timely::execute_from_args(std::env::args(), move |worker| {
let timer = ::std::time::Instant::now();
let peers = worker.peers();
let index = worker.index();
let (mut a, mut d) = worker.dataflow::<Time,_,_>(|scope| {
// let timer = timer.clone();
let (a_handle, assignment) = scope.new_collection::<_,Present>();
let (d_handle, dereference) = scope.new_collection::<_,Present>();
let nodes =
assignment
.flat_map(|(a,b)| vec![a,b])
.concat(&dereference.flat_map(|(a,b)| vec![a,b]));
let dereference = dereference.arrange::<OrdValSpine<_,_,_,_,Offs>>();
let (value_flow, memory_alias, value_alias) =
scope
.iterative::<Iter,_,_>(|scope| {
let nodes = nodes.enter(scope);
let assignment = assignment.enter(scope);
let dereference = dereference.enter(scope);
let value_flow = SemigroupVariable::new(scope, Product::new(Default::default(), 1));
let memory_alias = SemigroupVariable::new(scope, Product::new(Default::default(), 1));
let value_flow_arranged = value_flow.arrange::<OrdValSpine<_,_,_,_,Offs>>();
let memory_alias_arranged = memory_alias.arrange::<OrdValSpine<_,_,_,_,Offs>>();
// VA(a,b) <- VF(x,a),VF(x,b)
// VA(a,b) <- VF(x,a),MA(x,y),VF(y,b)
let value_alias_next = value_flow_arranged.join_core(&value_flow_arranged, |_,&a,&b| Some((a,b)));
let value_alias_next = value_flow_arranged.join_core(&memory_alias_arranged, |_,&a,&b| Some((b,a)))
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
.join_core(&value_flow_arranged, |_,&a,&b| Some((a,b)))
.concat(&value_alias_next);
// VF(a,a) <-
// VF(a,b) <- A(a,x),VF(x,b)
// VF(a,b) <- A(a,x),MA(x,y),VF(y,b)
let value_flow_next =
assignment
.map(|(a,b)| (b,a))
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
.join_core(&memory_alias_arranged, |_,&a,&b| Some((b,a)))
.concat(&assignment.map(|(a,b)| (b,a)))
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
.join_core(&value_flow_arranged, |_,&a,&b| Some((a,b)))
.concat(&nodes.map(|n| (n,n)));
let value_flow_next =
value_flow_next
.arrange::<OrdKeySpine<_,_,_,Offs>>()
// .distinct_total_core::<Diff>()
.threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None })
;
// MA(a,b) <- D(x,a),VA(x,y),D(y,b)
let memory_alias_next: Collection<_,_,Present> =
value_alias_next
.join_core(&dereference, |_x,&y,&a| Some((y,a)))
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
.join_core(&dereference, |_y,&a,&b| Some((a,b)));
let memory_alias_next: Collection<_,_,Present> =
memory_alias_next
.arrange::<OrdKeySpine<_,_,_,Offs>>()
// .distinct_total_core::<Diff>()
.threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None })
;
value_flow.set(&value_flow_next);
memory_alias.set(&memory_alias_next);
(value_flow_next.leave(), memory_alias_next.leave(), value_alias_next.leave())
});
value_flow.map(|_| ()).consolidate().inspect(|x| println!("VF: {:?}", x));
memory_alias.map(|_| ()).consolidate().inspect(|x| println!("MA: {:?}", x));
value_alias.map(|_| ()).consolidate().inspect(|x| println!("VA: {:?}", x));
(a_handle, d_handle)
});
if index == 0 { println!("{:?}:\tDataflow assembled", timer.elapsed().as_nanos()); }
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
let file = BufReader::new(File::open(filename).unwrap());
for readline in file.lines() {
let line = readline.ok().expect("read error");
if !line.starts_with('#') && line.len() > 0 {
let mut elts = line[..].split_whitespace();
let src: Node = elts.next().unwrap().parse().ok().expect("malformed src");
if (src as usize) % peers == index {
let dst: Node = elts.next().unwrap().parse().ok().expect("malformed dst");
let typ: &str = elts.next().unwrap();
match typ {
"a" => a.update((src,dst), Present),
"d" => d.update((src,dst), Present),
_ => { },
// x => panic!("Unexpected type: {:?}", x),
}
}
}
}
if index == 0 { println!("{:?}:\tData loaded", timer.elapsed().as_nanos()); }
a.close();
d.close();
while worker.step() { }
if index == 0 { println!("{:?}:\tComputation complete", timer.elapsed().as_nanos()); }
}).unwrap();
}
fn optimized() {
timely::execute_from_args(std::env::args(), move |worker| {
let timer = ::std::time::Instant::now();
let peers = worker.peers();
let index = worker.index();
let (mut a, mut d) = worker.dataflow::<(),_,_>(|scope| {
// let timer = timer.clone();
let (a_handle, assignment) = scope.new_collection();
let (d_handle, dereference) = scope.new_collection();
let nodes =
assignment
.flat_map(|(a,b)| vec![a,b])
.concat(&dereference.flat_map(|(a,b)| vec![a,b]));
let dereference = dereference.arrange::<OrdValSpine<_,_,_,_,Offs>>();
let (value_flow, memory_alias) =
scope
.iterative::<Iter,_,_>(|scope| {
let nodes = nodes.enter(scope);
let assignment = assignment.enter(scope);
let dereference = dereference.enter(scope);
let value_flow = SemigroupVariable::new(scope, Product::new(Default::default(), 1));
let memory_alias = SemigroupVariable::new(scope, Product::new(Default::default(), 1));
let value_flow_arranged = value_flow.arrange::<OrdValSpine<_,_,_,_,Offs>>();
let memory_alias_arranged = memory_alias.arrange::<OrdValSpine<_,_,_,_,Offs>>();
// VF(a,a) <-
// VF(a,b) <- A(a,x),VF(x,b)
// VF(a,b) <- A(a,x),MA(x,y),VF(y,b)
let value_flow_next =
assignment
.map(|(a,b)| (b,a))
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
.join_core(&memory_alias_arranged, |_,&a,&b| Some((b,a)))
.concat(&assignment.map(|(a,b)| (b,a)))
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
.join_core(&value_flow_arranged, |_,&a,&b| Some((a,b)))
.concat(&nodes.map(|n| (n,n)))
.arrange::<OrdKeySpine<_,_,_,Offs>>()
// .distinct_total_core::<Diff>()
.threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None })
;
// VFD(a,b) <- VF(a,x),D(x,b)
let value_flow_deref =
value_flow
.map(|(a,b)| (b,a))
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
.join_core(&dereference, |_x,&a,&b| Some((a,b)))
.arrange::<OrdValSpine<_,_,_,_,Offs>>();
// MA(a,b) <- VFD(x,a),VFD(y,b)
// MA(a,b) <- VFD(x,a),MA(x,y),VFD(y,b)
let memory_alias_next =
value_flow_deref
.join_core(&value_flow_deref, |_y,&a,&b| Some((a,b)));
let memory_alias_next =
memory_alias_arranged
.join_core(&value_flow_deref, |_x,&y,&a| Some((y,a)))
.arrange::<OrdValSpine<_,_,_,_,Offs>>()
.join_core(&value_flow_deref, |_y,&a,&b| Some((a,b)))
.concat(&memory_alias_next)
.arrange::<OrdKeySpine<_,_,_,Offs>>()
// .distinct_total_core::<Diff>()
.threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None })
;
value_flow.set(&value_flow_next);
memory_alias.set(&memory_alias_next);
(value_flow_next.leave(), memory_alias_next.leave())
});
value_flow.map(|_| ()).consolidate().inspect(|x| println!("VF: {:?}", x));
memory_alias.map(|_| ()).consolidate().inspect(|x| println!("MA: {:?}", x));
(a_handle, d_handle)
});
if index == 0 { println!("{:?}:\tDataflow assembled", timer.elapsed().as_nanos()); }
// snag a filename to use for the input graph.
let filename = std::env::args().nth(1).unwrap();
let file = BufReader::new(File::open(filename).unwrap());
for readline in file.lines() {
let line = readline.ok().expect("read error");
if !line.starts_with('#') && line.len() > 0 {
let mut elts = line[..].split_whitespace();
let src: Node = elts.next().unwrap().parse().ok().expect("malformed src");
if (src as usize) % peers == index {
let dst: Node = elts.next().unwrap().parse().ok().expect("malformed dst");
let typ: &str = elts.next().unwrap();
match typ {
"a" => { a.update((src, dst), Present); },
"d" => { d.update((src, dst), Present); },
_ => { },
// x => panic!("Unexpected type: {:?}", x),
}
}
}
}
if index == 0 { println!("{:?}:\tData loaded", timer.elapsed().as_nanos()); }
a.close();
d.close();
while worker.step() { }
if index == 0 { println!("{:?}:\tComputation complete", timer.elapsed().as_nanos()); }
}).unwrap();
}

View File

@ -0,0 +1,338 @@
#[macro_use]
extern crate abomonation_derive;
extern crate abomonation;
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
use rand::{Rng, SeedableRng, StdRng};
use timely::dataflow::ProbeHandle;
use timely::dataflow::operators::unordered_input::UnorderedInput;
use differential_dataflow::AsCollection;
use differential_dataflow::operators::*;
use pair::Pair;
fn main() {
let nodes: usize = std::env::args().nth(1).unwrap().parse().unwrap();
let edges: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let batch: usize = std::env::args().nth(3).unwrap().parse().unwrap();
let rounds: usize = std::env::args().nth(4).unwrap().parse().unwrap();
timely::execute_from_args(std::env::args().skip(5), move |worker| {
let index = worker.index();
let peers = worker.peers();
let mut probe = ProbeHandle::new();
let (mut root_input, root_cap, mut edge_input, mut edge_cap) =
worker.dataflow(|scope| {
let ((root_input, root_cap), roots) = scope.new_unordered_input();
let ((edge_input, edge_cap), edges) = scope.new_unordered_input();
let roots = roots.as_collection();
let edges = edges.as_collection()
// .inspect(|x| println!("edge: {:?}", x))
;
roots.iterate(|inner| {
let edges = edges.enter(&inner.scope());
let roots = roots.enter(&inner.scope());
edges
.semijoin(&inner)
.map(|(_s,d)| d)
.concat(&roots)
.distinct()
})
.consolidate()
// .inspect(|x| println!("edge: {:?}", x))
.map(|_| ())
.consolidate()
.inspect(|x| println!("{:?}\tchanges: {:?}", x.1, x.2))
.probe_with(&mut probe);
(root_input, root_cap, edge_input, edge_cap)
});
let seed: &[_] = &[1, 2, 3, index];
let mut rng1: StdRng = SeedableRng::from_seed(seed); // rng for edge additions
let mut rng2: StdRng = SeedableRng::from_seed(seed); // rng for edge deletions
let worker_edges = edges / peers + if index < edges % peers { 1 } else { 0 };
let worker_batch = batch / peers + if index < batch % peers { 1 } else { 0 };
// Times: (revision, event_time)
// load initial root.
root_input
.session(root_cap)
.give((0, Pair::new(0, 0), 1));
// load initial edges
edge_input
.session(edge_cap.clone())
.give_iterator((0 .. worker_edges).map(|_|
((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)),
Pair::new(0, 0), 1)
));
let edge_cap_next = edge_cap.delayed(&Pair::new(1, 0));
// Caps = { (1,0) , (0,1) }
edge_cap.downgrade(&Pair::new(0, 1));
while probe.less_than(edge_cap.time()) {
worker.step();
}
println!("Initial computation complete");
for round in 1 .. rounds {
edge_input
.session(edge_cap.clone())
.give_iterator((0 .. worker_batch).flat_map(|_| {
let insert = ((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)), Pair::new(0, round), 1);
let remove = ((rng2.gen_range(0, nodes), rng2.gen_range(0, nodes)), Pair::new(0, round),-1);
Some(insert).into_iter().chain(Some(remove).into_iter())
}));
edge_cap.downgrade(&Pair::new(0, round+1));
while probe.less_than(edge_cap.time()) {
worker.step();
}
// Caps = { (1,0), (0,round+1) }
println!("Initial round {} complete", round);
}
// Caps = { (1,0) }
let edge_cap0 = edge_cap;
println!("{:?}", edge_cap0.time());
edge_cap = edge_cap_next;
edge_input
.session(edge_cap.clone())
.give_iterator((0 .. worker_batch).map(|_| {
((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)), Pair::new(1, 0), 1)
}));
// Caps = { (2,0) }
edge_cap.downgrade(&Pair::new(2, 0));
while probe.less_equal(&Pair::new(1, rounds-1)) {
worker.step();
}
edge_input
.session(edge_cap.clone())
.give_iterator((0 .. worker_batch).map(|_| {
((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)), Pair::new(2, 3), 1)
}));
// Caps = { (3,0) }
edge_cap.downgrade(&Pair::new(3, 0));
while probe.less_equal(&Pair::new(2, rounds-1)) {
worker.step();
}
edge_input
.session(edge_cap.clone())
.give_iterator((0 .. worker_batch).map(|_| {
((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)), Pair::new(3, 1), 1)
}));
// Caps = { (4,0) }
edge_cap.downgrade(&Pair::new(4, 0));
while probe.less_equal(&Pair::new(3, rounds-1)) {
worker.step();
}
edge_input
.session(edge_cap0)
.give_iterator((0 .. worker_batch).map(|_| {
((rng1.gen_range(0, nodes), rng1.gen_range(0, nodes)), Pair::new(0, 10), 1)
}));
}).unwrap();
}
/// This module contains a definition of a new timestamp time, a "pair" or product.
///
/// This is a minimal self-contained implementation, in that it doesn't borrow anything
/// from the rest of the library other than the traits it needs to implement. With this
/// type and its implementations, you can use it as a timestamp type.
mod pair {
/// A pair of timestamps, partially ordered by the product order.
#[derive(Hash, Default, Clone, Eq, PartialEq, Ord, PartialOrd, Abomonation)]
pub struct Pair<S, T> {
pub first: S,
pub second: T,
}
impl<S, T> Pair<S, T> {
/// Create a new pair.
pub fn new(first: S, second: T) -> Self {
Pair { first, second }
}
}
// Implement timely dataflow's `PartialOrder` trait.
use timely::order::PartialOrder;
impl<S: PartialOrder, T: PartialOrder> PartialOrder for Pair<S, T> {
fn less_equal(&self, other: &Self) -> bool {
self.first.less_equal(&other.first) && self.second.less_equal(&other.second)
}
}
use timely::progress::timestamp::Refines;
impl<S: Timestamp, T: Timestamp> Refines<()> for Pair<S, T> {
fn to_inner(_outer: ()) -> Self { Self::minimum() }
fn to_outer(self) -> () { () }
fn summarize(_summary: <Self>::Summary) -> () { () }
}
// Implement timely dataflow's `PathSummary` trait.
// This is preparation for the `Timestamp` implementation below.
use timely::progress::PathSummary;
impl<S: Timestamp, T: Timestamp> PathSummary<Pair<S,T>> for () {
fn results_in(&self, timestamp: &Pair<S, T>) -> Option<Pair<S,T>> {
Some(timestamp.clone())
}
fn followed_by(&self, other: &Self) -> Option<Self> {
Some(other.clone())
}
}
// Implement timely dataflow's `Timestamp` trait.
use timely::progress::Timestamp;
impl<S: Timestamp, T: Timestamp> Timestamp for Pair<S, T> {
fn minimum() -> Self { Pair { first: S::minimum(), second: T::minimum() }}
type Summary = ();
}
// Implement differential dataflow's `Lattice` trait.
// This extends the `PartialOrder` implementation with additional structure.
use differential_dataflow::lattice::Lattice;
impl<S: Lattice, T: Lattice> Lattice for Pair<S, T> {
fn join(&self, other: &Self) -> Self {
Pair {
first: self.first.join(&other.first),
second: self.second.join(&other.second),
}
}
fn meet(&self, other: &Self) -> Self {
Pair {
first: self.first.meet(&other.first),
second: self.second.meet(&other.second),
}
}
}
use std::fmt::{Formatter, Error, Debug};
/// Debug implementation to avoid seeing fully qualified path names.
impl<TOuter: Debug, TInner: Debug> Debug for Pair<TOuter, TInner> {
fn fmt(&self, f: &mut Formatter) -> Result<(), Error> {
f.write_str(&format!("({:?}, {:?})", self.first, self.second))
}
}
}
/// This module contains a definition of a new timestamp time, a "pair" or product.
///
/// This is a minimal self-contained implementation, in that it doesn't borrow anything
/// from the rest of the library other than the traits it needs to implement. With this
/// type and its implementations, you can use it as a timestamp type.
mod vector {
/// A pair of timestamps, partially ordered by the product order.
#[derive(Hash, Default, Clone, Eq, PartialEq, Ord, PartialOrd, Abomonation, Debug)]
pub struct Vector<T> {
pub vector: Vec<T>,
}
impl<T> Vector<T> {
/// Create a new pair.
pub fn new(vector: Vec<T>) -> Self {
Vector { vector }
}
}
// Implement timely dataflow's `PartialOrder` trait.
use timely::order::PartialOrder;
impl<T: PartialOrder+Timestamp> PartialOrder for Vector<T> {
fn less_equal(&self, other: &Self) -> bool {
self.vector
.iter()
.enumerate()
.all(|(index, time)| time.less_equal(other.vector.get(index).unwrap_or(&T::minimum())))
}
}
use timely::progress::timestamp::Refines;
impl<T: Timestamp> Refines<()> for Vector<T> {
fn to_inner(_outer: ()) -> Self { Self { vector: Vec::new() } }
fn to_outer(self) -> () { () }
fn summarize(_summary: <Self>::Summary) -> () { () }
}
// Implement timely dataflow's `PathSummary` trait.
// This is preparation for the `Timestamp` implementation below.
use timely::progress::PathSummary;
impl<T: Timestamp> PathSummary<Vector<T>> for () {
fn results_in(&self, timestamp: &Vector<T>) -> Option<Vector<T>> {
Some(timestamp.clone())
}
fn followed_by(&self, other: &Self) -> Option<Self> {
Some(other.clone())
}
}
// Implement timely dataflow's `Timestamp` trait.
use timely::progress::Timestamp;
impl<T: Timestamp> Timestamp for Vector<T> {
fn minimum() -> Self { Self { vector: Vec::new() } }
type Summary = ();
}
// Implement differential dataflow's `Lattice` trait.
// This extends the `PartialOrder` implementation with additional structure.
use differential_dataflow::lattice::Lattice;
impl<T: Lattice+Timestamp+Clone> Lattice for Vector<T> {
fn join(&self, other: &Self) -> Self {
let min_len = ::std::cmp::min(self.vector.len(), other.vector.len());
let max_len = ::std::cmp::max(self.vector.len(), other.vector.len());
let mut vector = Vec::with_capacity(max_len);
for index in 0 .. min_len {
vector.push(self.vector[index].join(&other.vector[index]));
}
for time in &self.vector[min_len..] {
vector.push(time.clone());
}
for time in &other.vector[min_len..] {
vector.push(time.clone());
}
Self { vector }
}
fn meet(&self, other: &Self) -> Self {
let min_len = ::std::cmp::min(self.vector.len(), other.vector.len());
let mut vector = Vec::with_capacity(min_len);
for index in 0 .. min_len {
vector.push(self.vector[index].meet(&other.vector[index]));
}
Self { vector }
}
}
}

View File

@ -0,0 +1,146 @@
extern crate rand;
extern crate timely;
extern crate differential_dataflow;
use rand::{Rng, SeedableRng, StdRng};
use differential_dataflow::input::Input;
use differential_dataflow::operators::Join;
use differential_dataflow::operators::Consolidate;
#[derive(Clone)]
pub enum AdType {
Banner,
Modal,
SponsoredSearch,
Mail,
Mobile,
}
#[derive(Clone, Eq, PartialEq)]
pub enum EventType {
View,
Click,
Purchase,
}
#[derive(Clone)]
pub struct View {
pub user_id: usize,
pub page_id: usize,
pub ad_id: usize,
pub ad_type: AdType,
pub event_type: EventType,
pub event_time: usize,
pub ip_address: [u8;4],
}
impl View {
pub fn rand_from<R: Rng>(rng: &mut R, ad_ids: &[usize]) -> Self {
View {
user_id: rng.gen(),
page_id: rng.gen(),
ad_id: *rng.choose(ad_ids).unwrap(),
ad_type: rng.choose(&[
AdType::Banner,
AdType::Modal,
AdType::SponsoredSearch,
AdType::Mail,
AdType::Mobile,]).unwrap().clone(),
event_type: rng.choose(&[
EventType::View,
EventType::Click,
EventType::Purchase]).unwrap().clone(),
event_time: rng.gen(),
ip_address: rng.gen(),
}
}
}
fn main() {
let campaigns: usize = std::env::args().nth(1).unwrap().parse().unwrap();
let ads_per: usize = std::env::args().nth(2).unwrap().parse().unwrap();
let batch: usize = std::env::args().nth(3).unwrap().parse().unwrap();
let inspect: bool = std::env::args().find(|x| x == "inspect").is_some();
timely::execute_from_args(std::env::args().skip(4), move |worker| {
let timer = ::std::time::Instant::now();
let index = worker.index();
let peers = worker.peers();
// create a degree counting differential dataflow
let (mut views, mut links, probe) = worker.dataflow(|scope| {
// create edge input, count a few ways.
let (views_input, views) = scope.new_collection();
let (links_input, links) = scope.new_collection();
let probe =
links
.semijoin(&views)
.map(|(_ad, campaign)| campaign)
.consolidate()
.inspect(move |x| if inspect { println!("{:?}:\t{:?}", x.0, x.2); })
.probe();
(views_input, links_input, probe)
});
let seed_global: &[_] = &[1, 2, 3, peers];
let mut rng_global: StdRng = SeedableRng::from_seed(seed_global);
// generate ad and campaign identifiers.
let mut ad_identifiers = Vec::new();
for _ in 0 .. campaigns {
let campaign_id: usize = rng_global.gen();
for _ in 0 .. ads_per {
let ad_id: usize = rng_global.gen();
ad_identifiers.push(ad_id);
if index == 0 {
links.insert((ad_id, campaign_id));
}
}
}
links.close();
let seed_worker: &[_] = &[1, 2, 3, index];
let mut rng_worker: StdRng = SeedableRng::from_seed(seed_worker);
let mut typed_things = Vec::new();
for _ in 0 .. (1 << 16) {
typed_things.push(View::rand_from(&mut rng_worker, &ad_identifiers[..]));
}
let mut counter = 0;
let mut next = 10;
loop {
let elapsed_s = timer.elapsed().as_secs();
if elapsed_s >= next {
views.advance_to(elapsed_s);
views.flush();
worker.step_while(|| probe.less_than(views.time()));
next = next + 10;
println!("latency: {:?}ns; rate: {:?}/s", timer.elapsed().subsec_nanos(), counter / 10);
counter = 0;
}
for _ in 0 .. batch {
let mut rand_idx: usize = rng_worker.gen();
for _ in 0 .. 4 {
let rand_thing = &typed_things[rand_idx % (1 << 16)];
if rand_thing.event_type == EventType::Purchase {
views.insert(rand_thing.ad_id);
}
rand_idx = rand_idx >> 16;
}
}
worker.step();
counter += 4 * batch;
}
}).unwrap();
}

View File

@ -0,0 +1,7 @@
#[cfg(test)]
mod tests {
#[test]
fn it_works() {
assert_eq!(2 + 2, 4);
}
}

View File

@ -0,0 +1,13 @@
[package]
name = "interactive"
version = "0.1.0"
authors = ["Frank McSherry <fmcsherry@me.com>"]
[dependencies]
bincode = "1"
serde = "1"
serde_derive = "1"
differential-dataflow = { path = "../" }
dogsdogsdogs = { path = "../dogsdogsdogs" }
timely = { git = "https://github.com/TimelyDataflow/timely-dataflow", features = ["bincode"] }
#timely = { path = "../../timely-dataflow/timely", features = ["bincode"] }

View File

@ -0,0 +1,61 @@
extern crate interactive;
use std::time::Duration;
use interactive::{Command, Plan};
use interactive::concrete::{Session, Value};
fn main() {
let socket = std::net::TcpStream::connect("127.0.0.1:8000".to_string()).expect("failed to connect");
let mut session = Session::new(socket);
// Create initially empty set of edges.
session.issue(Command::CreateInput("Edges".to_string(), Vec::new()));
for node in 0 .. 1000 {
let edge = vec![Value::Usize(node), Value::Usize(node+1)];
session.issue(Command::UpdateInput("Edges".to_string(), vec![(edge, Duration::from_secs(0), 1)]));
}
// Create initially empty set of edges.
session.issue(Command::CreateInput("Nodes".to_string(), Vec::new()));
session.issue(
Plan::source("Nodes")
.join(Plan::source("Edges"), vec![(0, 0)])
.project(vec![1])
.inspect("one-hop")
.into_rule("One-hop"));
session.issue(Command::AdvanceTime(Duration::from_secs(1)));
session.issue(Command::UpdateInput("Nodes".to_string(), vec![(vec![Value::Usize(0)], Duration::from_secs(1), 1)]));
session.issue(Command::AdvanceTime(Duration::from_secs(2)));
session.issue(
Plan::source("Nodes")
.join(Plan::source("Edges"), vec![(0, 0)])
.project(vec![1])
.join(Plan::source("Edges"), vec![(0, 0)])
.project(vec![1])
.join(Plan::source("Edges"), vec![(0, 0)])
.project(vec![1])
.join(Plan::source("Edges"), vec![(0, 0)])
.project(vec![1])
.join(Plan::source("Edges"), vec![(0, 0)])
.project(vec![1])
.join(Plan::source("Edges"), vec![(0, 0)])
.project(vec![1])
.join(Plan::source("Edges"), vec![(0, 0)])
.project(vec![1])
.join(Plan::source("Edges"), vec![(0, 0)])
.project(vec![1])
.join(Plan::source("Edges"), vec![(0, 0)])
.project(vec![1])
.join(Plan::source("Edges"), vec![(0, 0)])
.project(vec![1])
.inspect("ten-hop")
.into_rule("Ten-hop"));
session.issue(Command::AdvanceTime(Duration::from_secs(3)));
session.issue(Command::Shutdown);
}

View File

@ -0,0 +1,46 @@
extern crate interactive;
use interactive::{Command, Plan};
use interactive::concrete::Session;
fn main() {
let socket = std::net::TcpStream::connect("127.0.0.1:8000".to_string()).expect("failed to connect");
let mut session = Session::new(socket);
session.issue(
Command::SourceLogging(
"127.0.0.1:9000".to_string(), // port the server should listen on.
"timely".to_string(), // flavor of logging (of "timely", "differential").
1, // number of worker connections to await.
1_000_000_000, // maximum granularity in nanoseconds.
"remote".to_string() // name to use for publication.
));
session.issue(
Plan::source("logs/remote/timely/operates")
.inspect("operates")
.into_rule("operates"));
session.issue(
Plan::source("logs/remote/timely/shutdown")
.inspect("shutdown")
.into_rule("shutdown"));
// session.issue(
// Plan::source("logs/remote/timely/channels")
// .inspect("channels")
// .into_rule("channels"));
// session.issue(
// Plan::source("logs/remote/timely/schedule")
// .inspect("schedule")
// .into_rule("schedule"));
// session.issue(
// Plan::source("logs/remote/timely/messages")
// .inspect("messages")
// .into_rule("messages"));
session.issue(Command::Shutdown);
}

View File

@ -0,0 +1,78 @@
extern crate interactive;
use std::time::Duration;
use interactive::{Command, Plan};
use interactive::concrete::{Session, Value};
fn main() {
let socket = std::net::TcpStream::connect("127.0.0.1:8000".to_string()).expect("failed to connect");
let mut session = Session::new(socket);
// Create initially empty set of edges.
session.issue(Command::CreateInput("Edges".to_string(), Vec::new()));
let nodes = 5;
for node_0 in 0 .. (nodes / 2) {
println!("Inserting node: {}", node_0);
let updates =
(0 .. nodes)
.map(|x| vec![Value::Usize(node_0), Value::Usize(x)])
.map(|e| (e, Duration::from_secs(node_0 as u64), 1))
.collect::<Vec<_>>();
session.issue(Command::UpdateInput("Edges".to_string(), updates));
session.issue(Command::AdvanceTime(Duration::from_secs(node_0 as u64 + 1)));
}
session.issue(
Plan::multiway_join(
vec![Plan::source("Edges"), Plan::source("Edges"), Plan::source("Edges")],
vec![
vec![(0,1), (1,0)], // b == b
vec![(0,0), (0,2)], // a == a
vec![(1,1), (1,2)], // c == c
],
vec![(0,0), (1,0), (1,1)],
)
.project(vec![])
.consolidate()
.inspect("triangles")
.into_rule("triangles"));
for node_0 in (nodes / 2) .. nodes {
let updates =
(0 .. nodes)
.map(|x| vec![Value::Usize(node_0), Value::Usize(x)])
.map(|e| (e, Duration::from_secs(node_0 as u64), 1))
.collect::<Vec<_>>();
session.issue(Command::UpdateInput("Edges".to_string(), updates));
session.issue(Command::AdvanceTime(Duration::from_secs(node_0 as u64 + 1)));
}
session.issue(
Plan::multiway_join(
vec![
Plan::source("Edges"), // R0(a,b)
Plan::source("Edges"), // R1(a,c)
Plan::source("Edges"), // R2(a,d)
Plan::source("Edges"), // R3(b,c)
Plan::source("Edges"), // R4(b,d)
Plan::source("Edges"), // R5(c,d)
],
vec![
vec![(0,0), (0,1), (0,2)], // a
vec![(1,0), (0,3), (0,4)], // b
vec![(1,1), (1,3), (0,5)], // c
vec![(1,2), (1,4), (1,5)], // d
],
vec![(0,0), (1,0), (1,1), (1,2)],
)
.project(vec![])
.consolidate()
.inspect("4cliques")
.into_rule("4cliques"));
session.issue(Command::Shutdown);
}

View File

@ -0,0 +1,119 @@
extern crate interactive;
use std::time::Duration;
use interactive::{Command, Plan};
use interactive::concrete::{Session, Value};
fn main() {
let socket = std::net::TcpStream::connect("127.0.0.1:8000".to_string()).expect("failed to connect");
let mut session = Session::new(socket);
session.issue(Command::CreateInput("XYZ".to_string(), Vec::new()));
session.issue(Command::CreateInput("XYGoal".to_string(), Vec::new()));
session.issue(Command::CreateInput("XZGoal".to_string(), Vec::new()));
// Determine errors in the xy plane.
session.issue(
Plan::source("XYZ")
.project(vec![0,1])
.distinct()
.negate()
.concat(Plan::source("XYGoal"))
.consolidate()
// .inspect("xy error")
.into_rule("XYErrors"));
// Determine errors in the xy plane.
session.issue(
Plan::source("XYZ")
.project(vec![0,2])
.distinct()
.negate()
.concat(Plan::source("XZGoal"))
.consolidate()
// .inspect("xz error")
.into_rule("XZErrors"));
session.issue(Command::AdvanceTime(Duration::from_secs(1)));
session.issue(
Command::UpdateInput(
"XYGoal".to_string(),
vec![
(vec![Value::Usize(0), Value::Usize(0)], Duration::from_secs(1), 1),
(vec![Value::Usize(0), Value::Usize(1)], Duration::from_secs(1), 1),
(vec![Value::Usize(0), Value::Usize(3)], Duration::from_secs(1), 1),
(vec![Value::Usize(0), Value::Usize(4)], Duration::from_secs(1), 1),
(vec![Value::Usize(1), Value::Usize(1)], Duration::from_secs(1), 1),
(vec![Value::Usize(1), Value::Usize(3)], Duration::from_secs(1), 1),
(vec![Value::Usize(2), Value::Usize(1)], Duration::from_secs(1), 1),
(vec![Value::Usize(2), Value::Usize(2)], Duration::from_secs(1), 1),
(vec![Value::Usize(3), Value::Usize(2)], Duration::from_secs(1), 1),
(vec![Value::Usize(3), Value::Usize(3)], Duration::from_secs(1), 1),
(vec![Value::Usize(3), Value::Usize(4)], Duration::from_secs(1), 1),
(vec![Value::Usize(4), Value::Usize(0)], Duration::from_secs(1), 1),
(vec![Value::Usize(4), Value::Usize(1)], Duration::from_secs(1), 1),
(vec![Value::Usize(4), Value::Usize(2)], Duration::from_secs(1), 1),
],
));
session.issue(
Command::UpdateInput(
"XZGoal".to_string(),
vec![
(vec![Value::Usize(0), Value::Usize(2)], Duration::from_secs(1), 1),
(vec![Value::Usize(0), Value::Usize(3)], Duration::from_secs(1), 1),
(vec![Value::Usize(0), Value::Usize(4)], Duration::from_secs(1), 1),
(vec![Value::Usize(1), Value::Usize(2)], Duration::from_secs(1), 1),
(vec![Value::Usize(1), Value::Usize(4)], Duration::from_secs(1), 1),
(vec![Value::Usize(2), Value::Usize(1)], Duration::from_secs(1), 1),
(vec![Value::Usize(2), Value::Usize(2)], Duration::from_secs(1), 1),
(vec![Value::Usize(2), Value::Usize(3)], Duration::from_secs(1), 1),
(vec![Value::Usize(3), Value::Usize(0)], Duration::from_secs(1), 1),
(vec![Value::Usize(3), Value::Usize(1)], Duration::from_secs(1), 1),
(vec![Value::Usize(3), Value::Usize(3)], Duration::from_secs(1), 1),
(vec![Value::Usize(3), Value::Usize(4)], Duration::from_secs(1), 1),
(vec![Value::Usize(4), Value::Usize(1)], Duration::from_secs(1), 1),
(vec![Value::Usize(4), Value::Usize(4)], Duration::from_secs(1), 1),
],
));
// Determine errors in the xy plane.
session.issue(
Plan::source("XYErrors")
.distinct()
.project(vec![])
.concat(Plan::source("XZErrors").distinct().project(vec![]))
.consolidate()
.inspect("error")
.into_rule("Errors"));
session.issue(Command::AdvanceTime(Duration::from_secs(2)));
session.issue(
Command::UpdateInput(
"XYZ".to_string(),
vec![
(vec![Value::Usize(0), Value::Usize(0), Value::Usize(2)], Duration::from_secs(2), 1),
(vec![Value::Usize(0), Value::Usize(1), Value::Usize(3)], Duration::from_secs(2), 1),
(vec![Value::Usize(0), Value::Usize(3), Value::Usize(4)], Duration::from_secs(2), 1),
(vec![Value::Usize(0), Value::Usize(4), Value::Usize(4)], Duration::from_secs(2), 1),
(vec![Value::Usize(1), Value::Usize(1), Value::Usize(2)], Duration::from_secs(2), 1),
(vec![Value::Usize(1), Value::Usize(3), Value::Usize(4)], Duration::from_secs(2), 1),
(vec![Value::Usize(2), Value::Usize(1), Value::Usize(1)], Duration::from_secs(2), 1),
(vec![Value::Usize(2), Value::Usize(2), Value::Usize(2)], Duration::from_secs(2), 1),
(vec![Value::Usize(2), Value::Usize(2), Value::Usize(3)], Duration::from_secs(2), 1),
(vec![Value::Usize(3), Value::Usize(2), Value::Usize(0)], Duration::from_secs(2), 1),
(vec![Value::Usize(3), Value::Usize(3), Value::Usize(1)], Duration::from_secs(2), 1),
(vec![Value::Usize(3), Value::Usize(4), Value::Usize(3)], Duration::from_secs(2), 1),
(vec![Value::Usize(3), Value::Usize(4), Value::Usize(4)], Duration::from_secs(2), 1),
(vec![Value::Usize(4), Value::Usize(0), Value::Usize(1)], Duration::from_secs(2), 1),
(vec![Value::Usize(4), Value::Usize(1), Value::Usize(4)], Duration::from_secs(2), 1),
(vec![Value::Usize(4), Value::Usize(2), Value::Usize(4)], Duration::from_secs(2), 1),
],
));
session.issue(Command::AdvanceTime(Duration::from_secs(2)));
session.issue(Command::Shutdown);
}

View File

@ -0,0 +1,89 @@
extern crate timely;
extern crate differential_dataflow;
extern crate interactive;
use std::sync::{Arc, Mutex};
use std::sync::mpsc::Sender;
use std::thread::Thread;
use timely::synchronization::Sequencer;
use interactive::{Command, Manager};
use interactive::concrete::Value;
fn main() {
let mut args = std::env::args();
args.next();
let (root_send, root_recv) = std::sync::mpsc::channel::<(Sender<Command<Value>>, Thread)>();
let root_send = Arc::new(Mutex::new(root_send));
std::thread::Builder::new()
.name("Listener".to_string())
.spawn(move || {
let (send, thread) = root_recv.recv().expect("Did not receive channel to worker");
use std::net::TcpListener;
let listener = TcpListener::bind("127.0.0.1:8000".to_string()).expect("failed to bind listener");
for mut stream in listener.incoming() {
let mut stream = stream.expect("listener error");
let send = send.clone();
let thread = thread.clone();
std::thread::Builder::new()
.name("Client".to_string())
.spawn(move || {
while let Ok(command) = bincode::deserialize_from::<_,Command<Value>>(&mut stream) {
send.send(command).expect("command send failed");
thread.unpark();
}
})
.expect("failed to create thread");
}
})
.expect("Failed to spawn listen thread");
// Initiate timely computation.
timely::execute_from_args(args, move |worker| {
// Send an endpoint and thread handle to root.
let (send, recv) = std::sync::mpsc::channel();
root_send
.lock()
.expect("lock poisoned")
.send((send, std::thread::current()))
.expect("send failed");
let timer = ::std::time::Instant::now();
let mut manager = Manager::<Value>::new();
let mut sequencer: Option<Sequencer<Command<Value>>> = Some(Sequencer::new(worker, timer));
while sequencer.is_some() {
// Check out channel status.
while let Ok(command) = recv.try_recv() {
sequencer
.as_mut()
.map(|s| s.push(command));
}
// Dequeue and act on commands.
// Once per iteration, so that Shutdown works "immediately".
if let Some(command) = sequencer.as_mut().and_then(|s| s.next()) {
if command == Command::Shutdown {
sequencer = None;
}
command.execute(&mut manager, worker);
worker.step();
}
else {
// Only consider parking if the sequencer is empty too.
worker.step_or_park(None);
}
}
println!("Shutting down");
}).expect("Timely computation did not initialize cleanly");
}

View File

@ -0,0 +1,197 @@
//! Commands accepted by the system.
use std::hash::Hash;
use std::io::Write;
use timely::communication::Allocate;
use timely::worker::Worker;
use timely::logging::TimelyEvent;
use differential_dataflow::logging::DifferentialEvent;
use differential_dataflow::ExchangeData;
use super::{Query, Rule, Plan, Time, Diff, Manager, Datum};
use crate::logging::LoggingValue;
/// Commands accepted by the system.
#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)]
pub enum Command<V: Datum> {
/// Installs the query and publishes public rules.
Query(Query<V>),
/// Advances all inputs and traces to `time`, and advances computation.
AdvanceTime(Time),
/// Creates a new named input, with initial input.
CreateInput(String, Vec<Vec<V>>),
/// Introduces updates to a specified input.
UpdateInput(String, Vec<(Vec<V>, Time, Diff)>),
/// Closes a specified input.
CloseInput(String),
/// Attaches a logging source. (address, flavor, number, granularity, name_as)
SourceLogging(String, String, usize, u64, String),
/// Terminates the system.
Shutdown,
}
impl<V: Datum> From<Query<V>> for Command<V> {
fn from(query: Query<V>) -> Self { Command::Query(query) }
}
impl<V: Datum> From<Rule<V>> for Command<V> {
fn from(rule: Rule<V>) -> Self { Command::Query(Query::new().add_rule(rule)) }
}
impl<V: Datum> Command<V>
where
V: ExchangeData+Hash+LoggingValue,
{
/// Executes a command.
pub fn execute<A: Allocate>(self, manager: &mut Manager<V>, worker: &mut Worker<A>) {
match self {
Command::Query(query) => {
// Query construction requires a bit of guff to allow us to
// re-use as much stuff as possible. It *seems* we need to
// be able to cache and re-use:
//
// 1. Collections.
// 2. Arrangements.
// 3. External traces.
//
// Although (2.) and (3.) look pretty similar, arrangements
// provide better progress tracking information than imported
// traces, and the types present in imported traces are not
// the same as those in arrangements.
worker.dataflow(|scope| {
use timely::dataflow::operators::Probe;
use differential_dataflow::operators::arrange::ArrangeBySelf;
use plan::Render;
let mut collections = std::collections::HashMap::new();
// let mut arrangements = std::collections::HashMap::new();
for Rule { name, plan } in query.rules.into_iter() {
let collection =
plan.render(scope, &mut collections, &mut manager.traces)
.arrange_by_self();
collection.stream.probe_with(&mut manager.probe);
let trace = collection.trace;
// Can bind the trace to both the plan and the name.
manager.traces.set_unkeyed(&plan, &trace);
manager.traces.set_unkeyed(&Plan::Source(name), &trace);
}
});
},
Command::AdvanceTime(time) => {
manager.advance_time(&time);
while manager.probe.less_than(&time) {
worker.step();
}
},
Command::CreateInput(name, updates) => {
use differential_dataflow::input::Input;
use differential_dataflow::operators::arrange::ArrangeBySelf;
let (input, trace) = worker.dataflow(|scope| {
let (input, collection) = scope.new_collection_from(updates.into_iter());
let trace = collection.arrange_by_self().trace;
(input, trace)
});
manager.insert_input(name, input, trace);
},
Command::UpdateInput(name, updates) => {
if let Some(input) = manager.inputs.sessions.get_mut(&name) {
for (data, time, diff) in updates.into_iter() {
input.update_at(data, time, diff);
}
}
else {
println!("Input not found: {:?}", name);
}
},
Command::CloseInput(name) => {
manager.inputs.sessions.remove(&name);
},
Command::SourceLogging(address, flavor, number, granularity, name_as) => {
match flavor.as_str() {
"timely" => {
let mut streams = Vec::new();
// Only one worker can bind to listen.
if worker.index() == 0 {
use std::time::Duration;
use std::net::TcpListener;
use timely::dataflow::operators::capture::EventReader;
println!("Awaiting timely logging connections ({})", number);
// e.g. "127.0.0.1:8000"
let listener = TcpListener::bind(address).unwrap();
for index in 0 .. number {
println!("\tTimely logging connection {} of {}", index, number);
let socket = listener.incoming().next().unwrap().unwrap();
socket.set_nonblocking(true).expect("failed to set nonblocking");
streams.push(EventReader::<Duration, (Duration, usize, TimelyEvent),_>::new(socket));
}
println!("\tAll logging connections established");
}
crate::logging::publish_timely_logging(manager, worker, granularity, &name_as, streams);
},
"differential" => {
let mut streams = Vec::new();
// Only one worker can bind to listen.
if worker.index() == 0 {
use std::time::Duration;
use std::net::TcpListener;
use timely::dataflow::operators::capture::EventReader;
// "127.0.0.1:8000"
let listener = TcpListener::bind(address).unwrap();
for _ in 0 .. number {
let socket = listener.incoming().next().unwrap().unwrap();
socket.set_nonblocking(true).expect("failed to set nonblocking");
streams.push(EventReader::<Duration, (Duration, usize, DifferentialEvent),_>::new(socket));
}
}
crate::logging::publish_differential_logging(manager, worker, granularity, &name_as, streams);
},
_ => { println!("{}", format!("Unknown logging flavor: {}", flavor)); }
}
}
Command::Shutdown => {
println!("Shutdown received");
manager.shutdown(worker);
}
}
}
/// Serialize the command at a writer.
pub fn serialize_into<W: Write>(&self, writer: W) {
bincode::serialize_into(writer, self).expect("bincode: serialization failed");
}
}

View File

@ -0,0 +1,128 @@
//! An example value type.
use std::time::Duration;
use super::{Datum, VectorFrom, Command};
/// A session.
pub struct Session<W: std::io::Write> {
write: W,
}
impl<W: std::io::Write> Session<W> {
/// Create a new session.
pub fn new(write: W) -> Self { Self { write } }
/// Issue a command.
pub fn issue<C: Into<Command<Value>>>(&mut self, command: C) {
let command: Command<Value> = command.into();
bincode::serialize_into(&mut self.write, &command)
.expect("bincode: serialization failed");
}
}
/// An example value type
#[derive(Serialize, Deserialize, Debug, Clone, Hash, Eq, PartialEq, Ord, PartialOrd)]
pub enum Value {
/// boolean
Bool(bool),
/// integer
Usize(usize),
/// string
String(String),
/// vector
Vector(Vec<Value>),
/// duration
Duration(Duration),
}
impl Datum for Value {
type Expression = usize;
fn subject_to(data: &[Self], expr: &Self::Expression) -> Self { data[*expr].clone() }
fn projection(index: usize) -> Self::Expression { index }
}
impl From<usize> for Value { fn from(x: usize) -> Self { Value::Usize(x) } }
impl From<bool> for Value { fn from(x: bool) -> Self { Value::Bool(x) } }
impl From<String> for Value { fn from(x: String) -> Self { Value::String(x) } }
impl From<Duration> for Value { fn from(x: Duration) -> Self { Value::Duration(x) } }
impl<V> From<Vec<V>> for Value where Value: From<V> {
fn from(x: Vec<V>) -> Self { Value::Vector(x.into_iter().map(|y| y.into()).collect()) }
}
use timely::logging::TimelyEvent;
impl VectorFrom<TimelyEvent> for Value {
fn vector_from(item: TimelyEvent) -> Vec<Value> {
match item {
TimelyEvent::Operates(x) => {
vec![
x.id.into(),
x.addr.into(),
x.name.into(),
]
},
TimelyEvent::Channels(x) => {
vec![
x.id.into(),
x.scope_addr.into(),
x.source.0.into(),
x.source.1.into(),
x.target.0.into(),
x.target.1.into(),
]
},
TimelyEvent::Schedule(x) => {
vec![
x.id.into(),
(x.start_stop == ::timely::logging::StartStop::Start).into(),
]
},
TimelyEvent::Messages(x) => {
vec![
x.channel.into(),
x.is_send.into(),
x.source.into(),
x.target.into(),
x.seq_no.into(),
x.length.into(),
]
},
TimelyEvent::Shutdown(x) => { vec![x.id.into()] },
// TimelyEvent::Park(x) => {
// match x {
// timely::logging::ParkEvent::ParkUnpark::Park(x) => { vec![true.into(), x.into()] },
// timely::logging::ParkEvent::ParkUnpark::Unpark => { vec![false.into(), 0.into()] },
// }
// },
TimelyEvent::Text(x) => { vec![Value::String(x)] }
_ => { vec![] },
}
}
}
use differential_dataflow::logging::DifferentialEvent;
impl VectorFrom<DifferentialEvent> for Value {
fn vector_from(item: DifferentialEvent) -> Vec<Value> {
match item {
DifferentialEvent::Batch(x) => {
vec![
x.operator.into(),
x.length.into(),
]
},
DifferentialEvent::Merge(x) => {
vec![
x.operator.into(),
x.scale.into(),
x.length1.into(),
x.length2.into(),
x.complete.unwrap_or(0).into(),
x.complete.is_some().into(),
]
},
_ => { vec![] },
}
}
}

View File

@ -0,0 +1,95 @@
//! Interactive differential dataflow
//!
//! This crate provides a demonstration of an interactive differential
//! dataflow system, which accepts query plans as data and then directly
//! implements them without compilation.
#![forbid(missing_docs)]
extern crate bincode;
extern crate timely;
extern crate differential_dataflow;
extern crate dogsdogsdogs;
extern crate serde;
#[macro_use]
extern crate serde_derive;
pub mod plan;
pub use plan::Plan;
pub mod manager;
pub use manager::{Manager, TraceManager, InputManager};
pub mod command;
pub use command::Command;
pub mod logging;
pub mod concrete;
/// System-wide notion of time.
pub type Time = ::std::time::Duration;
/// System-wide update type.
pub type Diff = isize;
use std::hash::Hash;
use std::fmt::Debug;
use serde::{Serialize, Deserialize};
/// Types capable of use as data in interactive.
pub trait Datum : Hash+Sized+Debug {
/// A type that can act on slices of data.
type Expression : Clone+Debug+Eq+Ord+Hash+Serialize+for<'a>Deserialize<'a>;
/// Applies an expression to a slice of data.
fn subject_to(data: &[Self], expr: &Self::Expression) -> Self;
/// Creates a expression that implements projection.
fn projection(index: usize) -> Self::Expression;
}
/// A type that can be converted to a vector of another type.
pub trait VectorFrom<T> : Sized {
/// Converts `T` to a vector of `Self`.
fn vector_from(item: T) -> Vec<Self>;
}
/// Multiple related collection definitions.
#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)]
pub struct Query<V: Datum> {
/// A list of bindings of names to plans.
pub rules: Vec<Rule<V>>,
}
impl<V: Datum> Query<V> {
/// Creates a new, empty query.
pub fn new() -> Self {
Query { rules: Vec::new() }
}
/// Adds a rule to an existing query.
pub fn add_rule(mut self, rule: Rule<V>) -> Self {
self.rules.push(rule);
self
}
}
impl<V: Datum> Query<V> {
/// Converts the query into a command.
pub fn into_command(self) -> Command<V> {
Command::Query(self)
}
}
/// Definition of a single collection.
#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)]
pub struct Rule<V: Datum> {
/// Name of the rule.
pub name: String,
/// Plan describing contents of the rule.
pub plan: Plan<V>,
}
impl<V: Datum> Rule<V> {
/// Converts the rule into a singleton query.
pub fn into_query(self) -> Query<V> {
Query::new().add_rule(self)
}
}

View File

@ -0,0 +1,283 @@
//! Methods for publishing logging arrangements.
use std::hash::Hash;
use std::time::Duration;
use timely::communication::Allocate;
use timely::worker::Worker;
use timely::logging::TimelyEvent;
use timely::dataflow::operators::capture::event::EventIterator;
use differential_dataflow::ExchangeData;
use differential_dataflow::logging::DifferentialEvent;
use crate::{Plan, VectorFrom, Datum};
use crate::manager::Manager;
/// A composite trait for values accommodating logging types.
pub trait LoggingValue : VectorFrom<TimelyEvent>+VectorFrom<DifferentialEvent> { }
impl<V: VectorFrom<TimelyEvent>+VectorFrom<DifferentialEvent>> LoggingValue for V { }
/// Timely logging capture and arrangement.
pub fn publish_timely_logging<V, A, I>(
manager: &mut Manager<V>,
worker: &mut Worker<A>,
granularity_ns: u64,
name: &str,
events: I
)
where
V: ExchangeData+Hash+LoggingValue+Datum,
A: Allocate,
I : IntoIterator,
<I as IntoIterator>::Item: EventIterator<Duration, (Duration, usize, TimelyEvent)>+'static
{
let (operates, channels, schedule, messages, shutdown, park, text) =
worker.dataflow(move |scope| {
// use timely::dataflow::operators::Map;
// use timely::dataflow::operators::Operator;
use timely::dataflow::operators::capture::Replay;
use timely::dataflow::operators::generic::builder_rc::OperatorBuilder;
let input_stream = events.replay_into(scope);
let mut demux = OperatorBuilder::new("Timely Logging Demux".to_string(), scope.clone());
use timely::dataflow::channels::pact::Pipeline;
let mut input = demux.new_input(&input_stream, Pipeline);
let (mut operates_out, operates) = demux.new_output();
let (mut channels_out, channels) = demux.new_output();
let (mut schedule_out, schedule) = demux.new_output();
let (mut messages_out, messages) = demux.new_output();
let (mut shutdown_out, shutdown) = demux.new_output();
let (mut park_out, park) = demux.new_output();
let (mut text_out, text) = demux.new_output();
let mut demux_buffer = Vec::new();
demux.build(move |_capability| {
move |_frontiers| {
let mut operates = operates_out.activate();
let mut channels = channels_out.activate();
let mut schedule = schedule_out.activate();
let mut messages = messages_out.activate();
let mut shutdown = shutdown_out.activate();
let mut park = park_out.activate();
let mut text = text_out.activate();
input.for_each(|time, data| {
data.swap(&mut demux_buffer);
let mut operates_session = operates.session(&time);
let mut channels_session = channels.session(&time);
let mut schedule_session = schedule.session(&time);
let mut messages_session = messages.session(&time);
let mut shutdown_session = shutdown.session(&time);
let mut park_session = park.session(&time);
let mut text_session = text.session(&time);
for (time, _worker, datum) in demux_buffer.drain(..) {
// Round time up to next multiple of `granularity_ns`.
let time_ns = (((time.as_nanos() as u64) / granularity_ns) + 1) * granularity_ns;
let time = Duration::from_nanos(time_ns);
match datum {
TimelyEvent::Operates(_) => {
operates_session.give((V::vector_from(datum), time, 1));
},
TimelyEvent::Channels(_) => {
channels_session.give((V::vector_from(datum), time, 1));
},
TimelyEvent::Schedule(_) => {
schedule_session.give((V::vector_from(datum), time, 1));
},
TimelyEvent::Messages(_) => {
messages_session.give((V::vector_from(datum), time, 1));
},
TimelyEvent::Shutdown(_) => {
shutdown_session.give((V::vector_from(datum), time, 1));
},
TimelyEvent::Park(_) => {
park_session.give((V::vector_from(datum), time, 1));
}
TimelyEvent::Text(_) => {
text_session.give((V::vector_from(datum), time, 1));
}
_ => { },
}
}
});
}
});
// // Pair up start and stop events, to capture scheduling durations.
// let duration =
// input_stream
// .flat_map(move |(ts, worker, x)|
// if let TimelyEvent::Schedule(event) = x {
// Some((ts, worker, event))
// } else { None }
// )
// .unary(timely::dataflow::channels::pact::Pipeline, "Schedules", |_,_| {
// let mut map = std::collections::HashMap::new();
// let mut vec = Vec::new();
// move |input, output| {
// input.for_each(|time, data| {
// data.swap(&mut vec);
// let mut session = output.session(&time);
// for (ts, worker, event) in vec.drain(..) {
// let key = (worker, event.id);
// match event.start_stop {
// timely::logging::StartStop::Start => {
// assert!(!map.contains_key(&key));
// map.insert(key, ts);
// },
// timely::logging::StartStop::Stop => {
// assert!(map.contains_key(&key));
// let start = map.remove(&key).unwrap();
// let elapsed = ts - start;
// let elapsed_ns = (elapsed.as_secs() as isize) * 1_000_000_000 + (elapsed.subsec_nanos() as isize);
// session.give((key.1, ts, elapsed_ns));
// }
// }
// }
// });
// }
// });
use differential_dataflow::collection::AsCollection;
use differential_dataflow::operators::arrange::ArrangeBySelf;
let operates = operates.as_collection().arrange_by_self().trace;
let channels = channels.as_collection().arrange_by_self().trace;
let schedule = schedule.as_collection().arrange_by_self().trace;
let messages = messages.as_collection().arrange_by_self().trace;
let shutdown = shutdown.as_collection().arrange_by_self().trace;
let park = park.as_collection().arrange_by_self().trace;
let text = text.as_collection().arrange_by_self().trace;
// let elapsed =
// duration
// .map(|(k,t,r)| (vec![Value::Usize(k)], t, r))
// .map(move |(k,time,r)| {
// // Round time up to next multiple of `granularity_ns`.
// let time_ns = (((time.as_nanos() as u64) / granularity_ns) + 1) * granularity_ns;
// let time = Duration::from_nanos(time_ns);
// (k,time,r)
// })
// .as_collection()
// .arrange_by_self()
// .trace;
// let histogram =
// duration
// .map(|(k,t,r)| (vec![Value::Usize(k), Value::Usize((r as usize).next_power_of_two())], t, 1))
// .map(move |(k,time,r)| {
// // Round time up to next multiple of `granularity_ns`.
// let time_ns = (((time.as_nanos() as u64) / granularity_ns) + 1) * granularity_ns;
// let time = Duration::from_nanos(time_ns);
// (k,time,r)
// })
// .as_collection()
// .arrange_by_self()
// .trace;
(operates, channels, schedule, messages, shutdown, park, text)
});
manager.traces.set_unkeyed(&Plan::Source(format!("logs/{}/timely/operates", name)), &operates);
manager.traces.set_unkeyed(&Plan::Source(format!("logs/{}/timely/channels", name)), &channels);
manager.traces.set_unkeyed(&Plan::Source(format!("logs/{}/timely/schedule", name)), &schedule);
manager.traces.set_unkeyed(&Plan::Source(format!("logs/{}/timely/messages", name)), &messages);
manager.traces.set_unkeyed(&Plan::Source(format!("logs/{}/timely/shutdown", name)), &shutdown);
manager.traces.set_unkeyed(&Plan::Source(format!("logs/{}/timely/park", name)), &park);
manager.traces.set_unkeyed(&Plan::Source(format!("logs/{}/timely/text", name)), &text);
// manager.traces.set_unkeyed(&Plan::Source(format!("logs/{}/timely/schedule/elapsed", name)), &elapsed);
// manager.traces.set_unkeyed(&Plan::Source(format!("logs/{}/timely/schedule/histogram", name)), &histogram);
}
/// Timely logging capture and arrangement.
pub fn publish_differential_logging<V, A, I>(
manager: &mut Manager<V>,
worker: &mut Worker<A>,
granularity_ns: u64,
name: &str,
events: I
)
where
V: ExchangeData+Hash+LoggingValue+Datum,
A: Allocate,
I : IntoIterator,
<I as IntoIterator>::Item: EventIterator<Duration, (Duration, usize, DifferentialEvent)>+'static
{
let (merge,batch) =
worker.dataflow(move |scope| {
use timely::dataflow::operators::capture::Replay;
use timely::dataflow::operators::generic::builder_rc::OperatorBuilder;
let input = events.replay_into(scope);
let mut demux = OperatorBuilder::new("Differential Logging Demux".to_string(), scope.clone());
use timely::dataflow::channels::pact::Pipeline;
let mut input = demux.new_input(&input, Pipeline);
let (mut batch_out, batch) = demux.new_output();
let (mut merge_out, merge) = demux.new_output();
let mut demux_buffer = Vec::new();
demux.build(move |_capability| {
move |_frontiers| {
let mut batch = batch_out.activate();
let mut merge = merge_out.activate();
input.for_each(|time, data| {
data.swap(&mut demux_buffer);
let mut batch_session = batch.session(&time);
let mut merge_session = merge.session(&time);
for (time, _worker, datum) in demux_buffer.drain(..) {
// Round time up to next multiple of `granularity_ns`.
let time_ns = (((time.as_nanos() as u64) / granularity_ns) + 1) * granularity_ns;
let time = Duration::from_nanos(time_ns);
match datum {
DifferentialEvent::Batch(_) => {
batch_session.give((V::vector_from(datum), time, 1));
},
DifferentialEvent::Merge(_) => {
merge_session.give((V::vector_from(datum), time, 1));
},
_ => { },
}
}
});
}
});
use differential_dataflow::collection::AsCollection;
use differential_dataflow::operators::arrange::ArrangeBySelf;
let batch = batch.as_collection().arrange_by_self().trace;
let merge = merge.as_collection().arrange_by_self().trace;
(merge,batch)
});
manager.traces.set_unkeyed(&Plan::Source(format!("logs/{}/differential/arrange/batch", name)), &batch);
manager.traces.set_unkeyed(&Plan::Source(format!("logs/{}/differential/arrange/merge", name)), &merge);
}

View File

@ -0,0 +1,229 @@
//! Management of inputs and traces.
use std::collections::HashMap;
use std::hash::Hash;
// use std::time::Duration;
use timely::dataflow::ProbeHandle;
use timely::communication::Allocate;
use timely::worker::Worker;
use timely::logging::TimelyEvent;
// use timely::dataflow::operators::capture::event::EventIterator;
use differential_dataflow::ExchangeData;
use differential_dataflow::trace::implementations::ord::{OrdKeySpine, OrdValSpine};
use differential_dataflow::operators::arrange::TraceAgent;
use differential_dataflow::input::InputSession;
use differential_dataflow::logging::DifferentialEvent;
use crate::{Time, Diff, Plan, Datum};
/// A trace handle for key-only data.
pub type TraceKeyHandle<K, T, R> = TraceAgent<OrdKeySpine<K, T, R>>;
/// A trace handle for key-value data.
pub type TraceValHandle<K, V, T, R> = TraceAgent<OrdValSpine<K, V, T, R>>;
/// A key-only trace handle binding `Time` and `Diff` using `Vec<V>` as data.
pub type KeysOnlyHandle<V> = TraceKeyHandle<Vec<V>, Time, Diff>;
/// A key-value trace handle binding `Time` and `Diff` using `Vec<V>` as data.
pub type KeysValsHandle<V> = TraceValHandle<Vec<V>, Vec<V>, Time, Diff>;
/// Manages inputs and traces.
pub struct Manager<V: ExchangeData+Datum> {
/// Manages input sessions.
pub inputs: InputManager<V>,
/// Manages maintained traces.
pub traces: TraceManager<V>,
/// Probes all computations.
pub probe: ProbeHandle<Time>,
}
impl<V: ExchangeData+Datum> Manager<V>
// where
// V: ExchangeData+Hash+LoggingValue,
{
/// Creates a new empty manager.
pub fn new() -> Self {
Manager {
inputs: InputManager::new(),
traces: TraceManager::new(),
probe: ProbeHandle::new(),
}
}
// /// Enables logging of timely and differential events.
// pub fn enable_logging<A: Allocate>(&mut self, worker: &mut Worker<A>) {
// use std::rc::Rc;
// use timely::dataflow::operators::capture::event::link::EventLink;
// use timely::logging::BatchLogger;
// let timely_events = Rc::new(EventLink::new());
// let differential_events = Rc::new(EventLink::new());
// self.publish_timely_logging(worker, Some(timely_events.clone()));
// self.publish_differential_logging(worker, Some(differential_events.clone()));
// let mut timely_logger = BatchLogger::new(timely_events.clone());
// worker
// .log_register()
// .insert::<TimelyEvent,_>("timely", move |time, data| timely_logger.publish_batch(time, data));
// let mut differential_logger = BatchLogger::new(differential_events.clone());
// worker
// .log_register()
// .insert::<DifferentialEvent,_>("differential/arrange", move |time, data| differential_logger.publish_batch(time, data));
// }
/// Clear the managed inputs and traces.
pub fn shutdown<A: Allocate>(&mut self, worker: &mut Worker<A>) {
self.inputs.sessions.clear();
self.traces.inputs.clear();
self.traces.arrangements.clear();
// Deregister loggers, so that the logging dataflows can shut down.
worker
.log_register()
.insert::<TimelyEvent,_>("timely", move |_time, _data| { });
worker
.log_register()
.insert::<DifferentialEvent,_>("differential/arrange", move |_time, _data| { });
}
/// Inserts a new input session by name.
pub fn insert_input(
&mut self,
name: String,
input: InputSession<Time, Vec<V>, Diff>,
trace: KeysOnlyHandle<V>)
{
self.inputs.sessions.insert(name.clone(), input);
self.traces.set_unkeyed(&Plan::Source(name), &trace);
}
/// Advances inputs and traces to `time`.
pub fn advance_time(&mut self, time: &Time) {
self.inputs.advance_time(time);
self.traces.advance_time(time);
}
// /// Timely logging capture and arrangement.
// pub fn publish_timely_logging<A, I>(&mut self, worker: &mut Worker<A>, events: I)
// where
// A: Allocate,
// I : IntoIterator,
// <I as IntoIterator>::Item: EventIterator<Duration, (Duration, usize, TimelyEvent)>+'static
// {
// crate::logging::publish_timely_logging(self, worker, 1, "interactive", events)
// }
// /// Timely logging capture and arrangement.
// pub fn publish_differential_logging<A, I>(&mut self, worker: &mut Worker<A>, events: I)
// where
// A: Allocate,
// I : IntoIterator,
// <I as IntoIterator>::Item: EventIterator<Duration, (Duration, usize, DifferentialEvent)>+'static
// {
// crate::logging::publish_differential_logging(self, worker, 1, "interactive", events)
// }
}
/// Manages input sessions.
pub struct InputManager<V: ExchangeData> {
/// Input sessions by name.
pub sessions: HashMap<String, InputSession<Time, Vec<V>, Diff>>,
}
impl<V: ExchangeData> InputManager<V> {
/// Creates a new empty input manager.
pub fn new() -> Self { Self { sessions: HashMap::new() } }
/// Advances the times of all managed inputs.
pub fn advance_time(&mut self, time: &Time) {
for session in self.sessions.values_mut() {
session.advance_to(time.clone());
session.flush();
}
}
}
/// Root handles to maintained collections.
///
/// Manages a map from plan (describing a collection)
/// to various arranged forms of that collection.
pub struct TraceManager<V: ExchangeData+Datum> {
/// Arrangements where the record itself is they key.
///
/// This contains both input collections, which are here cached so that
/// they can be re-used, intermediate collections that are cached, and
/// any collections that are explicitly published.
inputs: HashMap<Plan<V>, KeysOnlyHandle<V>>,
/// Arrangements of collections by key.
arrangements: HashMap<Plan<V>, HashMap<Vec<usize>, KeysValsHandle<V>>>,
}
impl<V: ExchangeData+Hash+Datum> TraceManager<V> {
/// Creates a new empty trace manager.
pub fn new() -> Self {
Self {
inputs: HashMap::new(),
arrangements: HashMap::new()
}
}
/// Advances the frontier of each maintained trace.
pub fn advance_time(&mut self, time: &Time) {
use differential_dataflow::trace::TraceReader;
use timely::progress::frontier::Antichain;
let frontier = Antichain::from_elem(time.clone());
for trace in self.inputs.values_mut() {
trace.set_logical_compaction(frontier.borrow());
trace.set_physical_compaction(frontier.borrow());
}
for map in self.arrangements.values_mut() {
for trace in map.values_mut() {
trace.set_logical_compaction(frontier.borrow());
trace.set_physical_compaction(frontier.borrow());
}
}
}
/// Recover an arrangement by plan and keys, if it is cached.
pub fn get_unkeyed(&self, plan: &Plan<V>) -> Option<KeysOnlyHandle<V>> {
self.inputs
.get(plan)
.map(|x| x.clone())
}
/// Installs an unkeyed arrangement for a specified plan.
pub fn set_unkeyed(&mut self, plan: &Plan<V>, handle: &KeysOnlyHandle<V>) {
self.inputs
.insert(plan.clone(), handle.clone());
}
/// Recover an arrangement by plan and keys, if it is cached.
pub fn get_keyed(&self, plan: &Plan<V>, keys: &[usize]) -> Option<KeysValsHandle<V>> {
self.arrangements
.get(plan)
.and_then(|map| map.get(keys).map(|x| x.clone()))
}
/// Installs a keyed arrangement for a specified plan and sequence of keys.
pub fn set_keyed(&mut self, plan: &Plan<V>, keys: &[usize], handle: &KeysValsHandle<V>) {
self.arrangements
.entry(plan.clone())
.or_insert(HashMap::new())
.insert(keys.to_vec(), handle.clone());
}
}

View File

@ -0,0 +1,40 @@
//! Concat expression plan.
use std::hash::Hash;
use timely::dataflow::Scope;
use differential_dataflow::{Collection, ExchangeData};
use plan::{Plan, Render};
use {TraceManager, Time, Diff, Datum};
/// Merges the source collections.
#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)]
pub struct Concat<V: Datum> {
/// Plan for the data source.
pub plans: Vec<Plan<V>>,
}
impl<V: ExchangeData+Hash+Datum> Render for Concat<V> {
type Value = V;
fn render<S: Scope<Timestamp = Time>>(
&self,
scope: &mut S,
arrangements: &mut TraceManager<V>) -> Collection<S, Vec<Self::Value>, Diff>
{
use timely::dataflow::operators::Concatenate;
use differential_dataflow::AsCollection;
let collections =
self.plans
.iter()
.map(|plan| plan.render(scope, arrangements).inner)
.collect::<Vec<_>>();
scope
.concatenate(collections)
.as_collection()
}
}

View File

@ -0,0 +1,99 @@
//! Predicate expression plan.
use std::hash::Hash;
use timely::dataflow::Scope;
use differential_dataflow::{Collection, ExchangeData};
use plan::{Plan, Render};
use {TraceManager, Time, Diff, Datum};
/// What to compare against.
///
/// A second argument is either a constant or the index of another value.
#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)]
pub enum SecondArgument<Value> {
/// A constant value.
Constant(Value),
/// An index of another value.
Position(usize),
}
impl<Value> SecondArgument<Value> {
/// Produces the indicated value.
pub fn value<'a>(&'a self, values: &'a [Value]) -> &'a Value {
match self {
SecondArgument::Constant(value) => value,
SecondArgument::Position(index) => &values[*index],
}
}
}
/// Possible predicates to apply.
#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)]
pub enum Predicate<Value> {
/// Strictly less than.
LessThan(usize, SecondArgument<Value>),
/// Less than or equal.
LessEqual(usize, SecondArgument<Value>),
/// Strictly greater than.
GreaterThan(usize, SecondArgument<Value>),
/// Greater than or equal.
GreaterEqual(usize, SecondArgument<Value>),
/// Equal.
Equal(usize, SecondArgument<Value>),
/// Not equal.
NotEqual(usize, SecondArgument<Value>),
/// Any of a list of predicates.
Any(Vec<Predicate<Value>>),
/// All of a list of predicates.
All(Vec<Predicate<Value>>),
/// The complement of a predicate.
Not(Box<Predicate<Value>>),
}
impl<Value: Ord> Predicate<Value> {
/// Indicates if the predicate is satisfied.
pub fn satisfied(&self, values: &[Value]) -> bool {
match self {
Predicate::LessThan(index, other) => values[*index].lt(other.value(values)),
Predicate::LessEqual(index, other) => values[*index].le(other.value(values)),
Predicate::GreaterThan(index, other) => values[*index].gt(other.value(values)),
Predicate::GreaterEqual(index, other) => values[*index].ge(other.value(values)),
Predicate::Equal(index, other) => values[*index].eq(other.value(values)),
Predicate::NotEqual(index, other) => values[*index].ne(other.value(values)),
Predicate::Any(predicates) => predicates.iter().any(|p| p.satisfied(values)),
Predicate::All(predicates) => predicates.iter().all(|p| p.satisfied(values)),
Predicate::Not(predicate) => !predicate.satisfied(values),
}
}
}
/// A plan stage filtering source tuples by the specified
/// predicate. Frontends are responsible for ensuring that the source
/// binds the argument symbols.
#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)]
pub struct Filter<V: Datum> {
/// Logical predicate to apply.
pub predicate: Predicate<V>,
/// Plan for the data source.
pub plan: Box<Plan<V>>,
}
impl<V: ExchangeData+Hash+Datum> Render for Filter<V> {
type Value = V;
fn render<S: Scope<Timestamp = Time>>(
&self,
scope: &mut S,
collections: &mut std::collections::HashMap<Plan<Self::Value>, Collection<S, Vec<Self::Value>, Diff>>,
arrangements: &mut TraceManager<Self::Value>,
) -> Collection<S, Vec<Self::Value>, Diff>
{
let predicate = self.predicate.clone();
self.plan
.render(scope, collections, arrangements)
.filter(move |tuple| predicate.satisfied(tuple))
}
}

View File

@ -0,0 +1,110 @@
//! Equijoin expression plan.
use std::hash::Hash;
use timely::dataflow::Scope;
use differential_dataflow::operators::JoinCore;
use differential_dataflow::{Collection, ExchangeData};
use plan::{Plan, Render};
use {TraceManager, Time, Diff, Datum};
/// A plan stage joining two source relations on the specified
/// symbols. Throws if any of the join symbols isn't bound by both
/// sources.
#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)]
pub struct Join<Value: Datum> {
/// Pairs of indices whose values must be equal.
pub keys: Vec<(usize, usize)>,
/// Plan for the left input.
pub plan1: Box<Plan<Value>>,
/// Plan for the right input.
pub plan2: Box<Plan<Value>>,
}
impl<V: ExchangeData+Hash+Datum> Render for Join<V> {
type Value = V;
fn render<S: Scope<Timestamp = Time>>(
&self,
scope: &mut S,
collections: &mut std::collections::HashMap<Plan<Self::Value>, Collection<S, Vec<Self::Value>, Diff>>,
arrangements: &mut TraceManager<Self::Value>,
) -> Collection<S, Vec<Self::Value>, Diff>
{
use differential_dataflow::operators::arrange::ArrangeByKey;
// acquire arrangements for each input.
let keys1 = self.keys.iter().map(|key| key.0).collect::<Vec<_>>();
let mut trace1 =
if let Some(arrangement) = arrangements.get_keyed(&self.plan1, &keys1[..]) {
arrangement
}
else {
let keys = keys1.clone();
let arrangement =
self.plan1
.render(scope, collections, arrangements)
.map(move |tuple|
(
// TODO: Re-use `tuple` for values.
keys.iter().map(|index| tuple[*index].clone()).collect::<Vec<_>>(),
tuple
.into_iter()
.enumerate()
.filter(|(index,_value)| !keys.contains(index))
.map(|(_index,value)| value)
.collect::<Vec<_>>(),
)
)
.arrange_by_key();
arrangements.set_keyed(&self.plan1, &keys1[..], &arrangement.trace);
arrangement.trace
};
// extract relevant fields for each index.
let keys2 = self.keys.iter().map(|key| key.1).collect::<Vec<_>>();
let mut trace2 =
if let Some(arrangement) = arrangements.get_keyed(&self.plan2, &keys2[..]) {
arrangement
}
else {
let keys = keys2.clone();
let arrangement =
self.plan2
.render(scope, collections, arrangements)
.map(move |tuple|
(
// TODO: Re-use `tuple` for values.
keys.iter().map(|index| tuple[*index].clone()).collect::<Vec<_>>(),
tuple
.into_iter()
.enumerate()
.filter(|(index,_value)| !keys.contains(index))
.map(|(_index,value)| value)
.collect::<Vec<_>>(),
)
)
.arrange_by_key();
arrangements.set_keyed(&self.plan2, &keys2[..], &arrangement.trace);
arrangement.trace
};
let arrange1 = trace1.import(scope);
let arrange2 = trace2.import(scope);
arrange1
.join_core(&arrange2, |keys, vals1, vals2| {
Some(
keys.iter().cloned()
.chain(vals1.iter().cloned())
.chain(vals2.iter().cloned())
.collect()
)
})
}
}

View File

@ -0,0 +1,46 @@
//! Projection expression plan.
use std::hash::Hash;
use timely::dataflow::Scope;
use differential_dataflow::{Collection, ExchangeData};
use plan::{Plan, Render};
use {TraceManager, Time, Diff, Datum};
/// A plan which retains values at specified locations.
///
/// The plan does not ascribe meaning to specific locations (e.g. bindings)
/// to variable names, and simply selects out the indicated sequence of values,
/// panicking if some input record is insufficiently long.
#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)]
pub struct Map<V: Datum> {
/// Sequence (and order) of indices to be retained.
pub expressions: Vec<V::Expression>,
/// Plan for the data source.
pub plan: Box<Plan<V>>,
}
impl<V: ExchangeData+Hash+Datum> Render for Map<V> {
type Value = V;
fn render<S: Scope<Timestamp = Time>>(
&self,
scope: &mut S,
collections: &mut std::collections::HashMap<Plan<Self::Value>, Collection<S, Vec<Self::Value>, Diff>>,
arrangements: &mut TraceManager<Self::Value>,
) -> Collection<S, Vec<Self::Value>, Diff>
{
let expressions = self.expressions.clone();
// TODO: re-use `tuple` allocation.
self.plan
.render(scope, collections, arrangements)
.map(move |tuple|
expressions
.iter()
.map(|expr| V::subject_to(&tuple[..], expr))
.collect()
)
}
}

View File

@ -0,0 +1,228 @@
//! Types and traits for implementing query plans.
use std::hash::Hash;
use timely::dataflow::Scope;
use differential_dataflow::{Collection, ExchangeData};
use {TraceManager, Time, Diff};
// pub mod count;
pub mod filter;
pub mod join;
pub mod map;
pub mod sfw;
use crate::Datum;
// pub use self::count::Count;
pub use self::filter::{Filter, Predicate};
pub use self::join::Join;
pub use self::sfw::MultiwayJoin;
pub use self::map::Map;
/// A type that can be rendered as a collection.
pub trait Render : Sized {
/// Value type produced.
type Value: ExchangeData+Datum;
/// Renders the instance as a collection in the supplied scope.
///
/// This method has access to arranged data, and may rely on and update the set
/// of arrangements based on the needs and offerings of the rendering process.
fn render<S: Scope<Timestamp = Time>>(
&self,
scope: &mut S,
collections: &mut std::collections::HashMap<Plan<Self::Value>, Collection<S, Vec<Self::Value>, Diff>>,
arrangements: &mut TraceManager<Self::Value>,
) -> Collection<S, Vec<Self::Value>, Diff>;
}
/// Possible query plan types.
#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)]
pub enum Plan<V: Datum> {
/// Map
Map(Map<V>),
/// Distinct
Distinct(Box<Plan<V>>),
/// Concat
Concat(Vec<Plan<V>>),
/// Consolidate
Consolidate(Box<Plan<V>>),
/// Equijoin
Join(Join<V>),
/// MultiwayJoin
MultiwayJoin(MultiwayJoin<V>),
/// Negation
Negate(Box<Plan<V>>),
/// Filters bindings by one of the built-in predicates
Filter(Filter<V>),
/// Sources data from another relation.
Source(String),
/// Prints resulting updates.
Inspect(String, Box<Plan<V>>),
}
impl<V: ExchangeData+Hash+Datum> Plan<V> {
/// Retains only the values at the indicated indices.
pub fn project(self, indices: Vec<usize>) -> Self {
Plan::Map(Map {
expressions: indices.into_iter().map(|i| V::projection(i)).collect(),
plan: Box::new(self),
})
}
/// Reduces a collection to distinct tuples.
pub fn distinct(self) -> Self {
Plan::Distinct(Box::new(self))
}
/// Merges two collections.
pub fn concat(self, other: Self) -> Self {
Plan::Concat(vec![self, other])
}
/// Merges multiple collections.
pub fn concatenate(plans: Vec<Self>) -> Self {
Plan::Concat(plans)
}
/// Merges multiple collections.
pub fn consolidate(self) -> Self {
Plan::Consolidate(Box::new(self))
}
/// Equi-joins two collections using the specified pairs of keys.
pub fn join(self, other: Plan<V>, keys: Vec<(usize, usize)>) -> Self {
Plan::Join(Join {
keys,
plan1: Box::new(self),
plan2: Box::new(other),
})
}
/// Equi-joins multiple collections using lists of equality constraints.
///
/// The list `equalities` should contain equivalence classes of pairs of
/// attribute index and source index, and the `multiway_join` method will
/// ensure that each equivalence class has equal values in each attribute.
pub fn multiway_join(
sources: Vec<Self>,
equalities: Vec<Vec<(usize, usize)>>,
results: Vec<(usize, usize)>
) -> Self {
Plan::MultiwayJoin(MultiwayJoin {
results,
sources,
equalities,
})
}
/// Negates a collection (negating multiplicities).
pub fn negate(self) -> Self {
Plan::Negate(Box::new(self))
}
/// Restricts collection to tuples satisfying the predicate.
pub fn filter(self, predicate: Predicate<V>) -> Self {
Plan::Filter(Filter { predicate, plan: Box::new(self) } )
}
/// Loads a source of data by name.
pub fn source(name: &str) -> Self {
Plan::Source(name.to_string())
}
/// Prints each tuple prefixed by `text`.
pub fn inspect(self, text: &str) -> Self {
Plan::Inspect(text.to_string(), Box::new(self))
}
/// Convert the plan into a named rule.
pub fn into_rule(self, name: &str) -> crate::Rule<V> {
crate::Rule {
name: name.to_string(),
plan: self,
}
}
}
impl<V: ExchangeData+Hash+Datum> Render for Plan<V> {
type Value = V;
fn render<S: Scope<Timestamp = Time>>(
&self,
scope: &mut S,
collections: &mut std::collections::HashMap<Plan<Self::Value>, Collection<S, Vec<Self::Value>, Diff>>,
arrangements: &mut TraceManager<Self::Value>,
) -> Collection<S, Vec<Self::Value>, Diff>
{
if collections.get(self).is_none() {
let collection =
match self {
// Plan::Project(projection) => projection.render(scope, collections, arrangements),
Plan::Map(expressions) => expressions.render(scope, collections, arrangements),
Plan::Distinct(distinct) => {
use differential_dataflow::operators::reduce::ReduceCore;
use differential_dataflow::operators::arrange::ArrangeBySelf;
use differential_dataflow::trace::implementations::ord::OrdKeySpine;
let input =
if let Some(mut trace) = arrangements.get_unkeyed(&self) {
trace.import(scope)
}
else {
let input_arrangement = distinct.render(scope, collections, arrangements).arrange_by_self();
arrangements.set_unkeyed(&distinct, &input_arrangement.trace);
input_arrangement
};
let output = input.reduce_abelian::<_,OrdKeySpine<_,_,_>>("Distinct", move |_,_,t| t.push(((), 1)));
arrangements.set_unkeyed(&self, &output.trace);
output.as_collection(|k,&()| k.clone())
},
Plan::Concat(concat) => {
use timely::dataflow::operators::Concatenate;
use differential_dataflow::AsCollection;
let plans =
concat
.iter()
.map(|plan| plan.render(scope, collections, arrangements).inner)
.collect::<Vec<_>>();
scope
.concatenate(plans)
.as_collection()
}
Plan::Consolidate(consolidate) => {
if let Some(mut trace) = arrangements.get_unkeyed(&self) {
trace.import(scope).as_collection(|k,&()| k.clone())
}
else {
use differential_dataflow::operators::Consolidate;
consolidate.render(scope, collections, arrangements).consolidate()
}
},
Plan::Join(join) => join.render(scope, collections, arrangements),
Plan::MultiwayJoin(join) => join.render(scope, collections, arrangements),
Plan::Negate(negate) => {
negate.render(scope, collections, arrangements).negate()
},
Plan::Filter(filter) => filter.render(scope, collections, arrangements),
Plan::Source(source) => {
arrangements
.get_unkeyed(self)
.expect(&format!("Failed to find source collection: {:?}", source))
.import(scope)
.as_collection(|k,()| k.to_vec())
},
Plan::Inspect(text, plan) => {
let text = text.clone();
plan.render(scope, collections, arrangements)
.inspect(move |x| println!("{}\t{:?}", text, x))
},
};
collections.insert(self.clone(), collection);
}
collections.get(self).expect("We just installed this").clone()
}
}

View File

@ -0,0 +1,363 @@
//! Multi-way equijoin expression plan.
//!
//! This plan provides us the opportunity to map out a non-trivial differential
//! implementation for a complex join query. In particular, we are able to invoke
//! delta-query and worst-case optimal join plans, which avoid any intermediate
//! materialization.
//!
//! Each `MultiwayJoin` indicates several source collections, equality constraints
//! among their attributes, and then the set of attributes to produce as results.
//!
//! One naive implementation would take each input collection in order, and develop
//! the join restricted to the prefix of relations so far. Ideally the order would
//! be such that joined collections have equality constraints and prevent Cartesian
//! explosion. At each step, a new collection picks out some of the attributes and
//! instantiates a primitive binary join between the accumulated collection and the
//! next collection.
//!
//! A more sophisticated implementation establishes delta queries for each input
//! collection, which responds to changes in that input collection against the
//! current other input collections. For each input collection we may choose very
//! different join orders, as the order must follow equality constraints.
//!
//! A further implementation could develop the results attribute-by-attribute, as
//! opposed to collection-by-collection, which gives us the ability to use column
//! indices rather than whole-collection indices.
use std::hash::Hash;
use timely::dataflow::Scope;
use differential_dataflow::operators::Consolidate;
use differential_dataflow::operators::arrange::{ArrangeBySelf, ArrangeByKey};
use differential_dataflow::{Collection, ExchangeData};
use plan::{Plan, Render};
use {TraceManager, Time, Diff, Datum};
/// A multiway join of muliple relations.
///
/// By expressing multiple relations and required equivalances between their attributes,
/// we can more efficiently design incremental update strategies without materializing
/// and indexing intermediate relations.
#[derive(Serialize, Deserialize, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)]
pub struct MultiwayJoin<V: Datum> {
/// A list of (attribute index, input) pairs to extract.
pub results: Vec<(usize, usize)>,
/// A list of source collections.
pub sources: Vec<Plan<V>>,
/// Equality constraints.
///
/// Equality constraints are presented as lists of `(attr, input)` equivalence classes.
/// This means that each `(attr, input)` pair can exist in at most one list; if it would
/// appear in more than one list, those two lists should be merged.
pub equalities: Vec<Vec<(usize, usize)>>,
}
// TODO: This logic fails to perform restrictions in cases where a join does not
// occur. One example could be:
//
// result(a,b,c) := R1(a,b), R2(b,c,c)
//
// In this case, the requirement that the 2nd and 3rd columns of R2 be equal
// is not surfaced in any join, and is instead a filter that should be applied
// directly to R2 (before or after the join with R1; either could be best).
impl<V: ExchangeData+Hash+Datum> Render for MultiwayJoin<V> {
type Value = V;
fn render<S: Scope<Timestamp = Time>>(
&self,
scope: &mut S,
collections: &mut std::collections::HashMap<Plan<Self::Value>, Collection<S, Vec<Self::Value>, Diff>>,
arrangements: &mut TraceManager<Self::Value>,
) -> Collection<S, Vec<Self::Value>, Diff>
{
// The idea here is the following:
//
// For each stream, we will determine a streaming delta query, in which changes
// are joined against indexed forms of the other relations using `dogsdogsdogs`
// stateless `propose` operators.
//
// For a query Q(x,y,z) := A(x,y), B(y,z), C(x,z) we might write dataflows like:
//
// dQdA := dA(x,y), B(y,z), C(x,z)
// dQdB := dB(y,z), A(x,y), C(x,z)
// dQdC := dC(x,y), A(x,y), B(y,z)
//
// where each line is read from left to right as a sequence of `propose` joins,
// which respond to timestamped delta changes by joining with the maintained
// relation at that corresponding time.
//
// We take some care to make sure that when these joins are performed, a delta
// interacts with relations as if we updated A, then B, then C, as if in sequence.
// That is, when a dB delta joins A it observes all updates to A at times less or
// equal to the delta's timestamp, but when a dB delta joins C it observes only
// updates to C at times strictly less than the delta's timestamp.
//
// This is done to avoid double counting updates; any concurrent changes will be
// accounted for by the last relation for which there is a concurrent update.
// println!("{:?}", self);
// Attributes we may need from any and all relations.
let mut relevant_attributes = Vec::new();
relevant_attributes.extend(self.results.iter().cloned());
relevant_attributes.extend(self.equalities.iter().flat_map(|list| list.iter().cloned()));
relevant_attributes.sort();
relevant_attributes.dedup();
// println!("Relevant attributes: {:?}", relevant_attributes);
// Into which we accumulate change streams.
let mut accumulated_changes = Vec::new();
// For each participating relation, we build a delta query dataflow.
for (index, plan) in self.sources.iter().enumerate() {
// println!("building dataflow for relation {}", index);
// Restrict down to relevant attributes.
let mut attributes: Vec<(usize, usize)> =
relevant_attributes
.iter()
.filter(|(_attr, input)| input == &index)
.cloned()
.collect::<Vec<_>>();
let attributes_init = attributes.clone();
// println!("\tinitial attributes: {:?}", attributes);
// Ensure the plan is rendered and cached.
if arrangements.get_unkeyed(&plan).is_none() {
// println!("\tbuilding/caching source plan");
let collection = plan.render(scope, collections, arrangements);
arrangements.set_unkeyed(plan, &collection.arrange_by_self().trace);
}
else {
// println!("\tsource plan found");
}
let changes =
arrangements
.get_unkeyed(&plan)
.expect("Surely we just ensured this")
.import(scope)
.as_collection(|val,&()| val.clone())
.map(move |tuple| attributes_init.iter().map(|&(attr,_)|
tuple[attr].clone()).collect::<Vec<_>>()
);
// Before constructing the dataflow, which takes a borrow on `scope`,
// we'll want to ensure that we have all of the necessary data assets
// in place. This requires a bit of planning first, then the building.
// Acquire a sane sequence in which to join the relations:
//
// This is a sequence of relation identifiers, starting with `index`,
// such that each has at least one attribute in common with a prior
// relation, and so can be effectively joined.
let join_order = plan_join_order(index, &self.equalities);
let mut join_plan = Vec::new();
// println!("\tjoin order: {:?}", join_order);
// Skipping `index`, join in each relation in sequence.
for join_idx in join_order.into_iter().skip(1) {
// To join a relation, we need to determine any constraints on
// attributes in common with prior relations. Any other values
// should be appended to tuples in `changes` with care taken to
// update `attributes`.
let (keys, priors) = determine_keys_priors(join_idx, &self.equalities, &attributes[..]);
// The fields in `sources[join_idx]` that should be values are those
// that are required output or participate in an equality constraint,
// but *WHICH ARE NOT* in `keys`.
let vals =
relevant_attributes
.iter()
.filter(|&(attr,index)| index == &join_idx && !keys.contains(&attr))
.cloned()
.collect::<Vec<_>>();
// println!("\tkeys: {:?}, priors: {:?}, vals: {:?}", keys, priors, vals);
let mut projection = Vec::new();
for &attr in keys.iter() {
projection.push(attr);
}
for &(attr, _index) in vals.iter() {
projection.push(attr);
}
// TODO: Sort, to improve chances of re-use opportunities.
// Requires understanding how attributes move to get the right
// key selectors out though.
// projection.sort();
// projection.dedup(); // Should already be deduplicated, probably?
// Get a plan for the projection on to these few attributes.
let plan = self.sources[join_idx].clone().project(projection);
if arrangements.get_keyed(&plan, &keys[..]).is_none() {
// println!("\tbuilding key: {:?}, plan: {:?}", keys, plan);
let keys_clone = keys.clone();
let arrangement =
plan.render(scope, collections, arrangements)
.map(move |tuple| (keys_clone.iter().map(|&i| tuple[i].clone()).collect::<Vec<_>>(), tuple))
.arrange_by_key();
arrangements.set_keyed(&plan, &keys[..], &arrangement.trace);
}
else {
// println!("\tplan found: {:?}, {:?}", keys, plan);
}
let arrangement =
arrangements
.get_keyed(&plan, &keys[..])
.expect("Surely we just ensured this");
let key_selector = move |change: &Vec<V>|
priors.iter().map(|&p| change[p].clone()).collect::<Vec<_>>()
;
join_plan.push((join_idx, key_selector, arrangement));
attributes.extend(keys.into_iter().map(|x| (x, join_idx)));
attributes.extend(vals.into_iter());
// println!("\tattributes: {:?}", attributes);
}
// Build the dataflow.
use dogsdogsdogs::altneu::AltNeu;
let scope_name = format!("DeltaRule: {}/{}", index, self.sources.len());
let changes = scope.clone().scoped::<AltNeu<_>,_,_>(&scope_name, |inner| {
// This should default to an `AltNeu::Alt` timestamp.
let mut changes =
changes
.enter(inner)
;
for (join_idx, key_selector, mut trace) in join_plan.into_iter() {
// Use alt or neu timestamps based on relative indices.
// Must have an `if` statement here as the two arrangement have different
// types, and we would to determine `alt` v `neu` once, rather than per
// tuple in the cursor.
changes =
if join_idx < index {
let arrangement = trace.import(scope).enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), unimplemented!());
dogsdogsdogs::operators::propose(&changes, arrangement, key_selector)
}
else {
let arrangement = trace.import(scope).enter_at(inner, |_,_,t| AltNeu::neu(t.clone()), unimplemented!());
dogsdogsdogs::operators::propose(&changes, arrangement, key_selector)
}
.map(|(mut prefix, extensions)| { prefix.extend(extensions.into_iter()); prefix })
;
// TODO: Equality constraints strictly within a relation have the effect
// of "filtering" data, but they are ignored at the moment. We should
// check for these and do something about it.
}
// Extract `self.results` in order, using `attributes`.
//
// The specific attribute requested in `self.results` may not be present in
// `attributes` when it is equal to another present attribute. So, we should
// look around in `self.equalities` also.
let mut extract_map = Vec::new();
for result in self.results.iter() {
if let Some(position) = attributes.iter().position(|i| i == result) {
extract_map.push(position);
}
else {
for constraint in self.equalities.iter() {
if constraint.contains(result) {
if let Some(position) = constraint.iter().flat_map(|x| attributes.iter().position(|i| i == x)).next() {
extract_map.push(position);
}
else {
println!("WTF NOTHING FOUND NOOOOO!!!");
}
}
}
}
}
changes
.map(move |tuple| extract_map.iter().map(|&i| tuple[i].clone()).collect::<Vec<_>>())
.leave()
});
accumulated_changes.push(changes);
}
differential_dataflow::collection::concatenate(scope, accumulated_changes.into_iter())
.consolidate()
}
}
/// Sequences relations in `constraints`.
///
/// Relations become available for sequencing as soon as they share a constraint with
/// either `source` or another sequenced relation.
fn plan_join_order(source: usize, constraints: &[Vec<(usize, usize)>]) -> Vec<usize> {
let mut result = vec![source];
let mut active = true;
while active {
active = false;
for constraint in constraints.iter() {
// Check to see if the constraint contains a sequenced relation.
if constraint.iter().any(|(_,index)| result.contains(index)) {
// If so, sequence any unsequenced relations.
for (_, index) in constraint.iter() {
if !result.contains(index) {
result.push(*index);
active = true;
}
}
}
}
}
result
}
/// Identifies keys and values for a join.
///
/// The result is a sequence, for each
fn determine_keys_priors(
relation: usize,
constraints: &[Vec<(usize, usize)>],
current_attributes: &[(usize, usize)],
)
-> (Vec<usize>, Vec<usize>)
{
// The fields in `sources[join_idx]` that should be keys are those
// that share an equality constraint with an element of `attributes`.
// For each key, we should capture the associated `attributes` entry
// so that we can easily prepare the keys of the `delta` stream.
let mut keys = Vec::new();
let mut priors = Vec::new();
for constraint in constraints.iter() {
// If there is an intersection between `constraint` and `current_attributes`,
// we should capture the position in `current_attributes` and emit all of the
// attributes for `relation`.
if let Some(prior) = current_attributes.iter().position(|x| constraint.contains(x)) {
for &(attr, index) in constraint.iter() {
if index == relation {
keys.push(attr);
priors.push(prior);
}
}
}
}
(keys, priors)
}

View File

@ -0,0 +1,44 @@
# Summary
[Differential Dataflow](./introduction.md)
- [Motivation](./chapter_0/chapter_0.md)
- [Getting started](./chapter_0/chapter_0_0.md)
- [Step 1: Write a program](./chapter_0/chapter_0_1.md)
- [Step 2: Change its input](./chapter_0/chapter_0_2.md)
- [Increase all the things](./chapter_a/chapter_a.md)
- [Increase the scale](./chapter_a/chapter_a_1.md)
- [Increase the parallelism](./chapter_a/chapter_a_2.md)
- [Increase the interactivity](./chapter_a/chapter_a_3.md)
- [Differential Operators](./chapter_2/chapter_2.md)
- [Map](./chapter_2/chapter_2_1.md)
- [Filter](./chapter_2/chapter_2_2.md)
- [Concat](./chapter_2/chapter_2_3.md)
- [Consolidate](./chapter_2/chapter_2_4.md)
- [Join](./chapter_2/chapter_2_5.md)
- [Reduce](./chapter_2/chapter_2_6.md)
- [Iterate](./chapter_2/chapter_2_7.md)
- [Arrange](./chapter_2/chapter_2_8.md)
- [Differential Interactions](./chapter_3/chapter_3.md)
- [Creating inputs](./chapter_3/chapter_3_1.md)
- [Making changes](./chapter_3/chapter_3_3.md)
- [Advancing time](./chapter_3/chapter_3_4.md)
- [Observing probes](./chapter_3/chapter_3_2.md)
- [Performing work](./chapter_3/chapter_3_5.md)
- [Example Applications](./chapter_4/chapter_4.md)
- [Graph Computation](./chapter_4/chapter_4_1.md)
- [Interactive Queries](./chapter_4/chapter_4_2.md)
- [Real-time Streaming Input](./chapter_4/chapter_4_3.md)
- [Arrangements](./chapter_5/chapter_5.md)
- [An arrangement example](./chapter_5/chapter_5_1.md)
- [Different arrangements](./chapter_5/chapter_5_2.md)
- [Sharing across dataflows](./chapter_5/chapter_5_3.md)
- [Trace wrappers](./chapter_5/chapter_5_4.md)
- [Windows Enough and Time](./chapter_6/chapter_6.md)

View File

@ -0,0 +1,10 @@
# Motivation
Differential dataflow programs are structured as two easy steps:
1. Write a program.
2. Change its input.
We will work through an example program, and then interact with it by changing its inputs. Our goal is foremost to show you what a program looks like, and to give you a sense for what interactions look like.
Once we've done this, in the next chapter we will jazz things up a bit with an increased scale of data, computation, and interaction!

View File

@ -0,0 +1,37 @@
## Getting started
The first thing you will need to do, if you want to follow along with the examples, is to acquire a copy of [Rust](https://www.rust-lang.org/). This is the programming language that differential dataflow uses, and it is in charge of building our projects.
With Rust in hand, crack open a shell and make a new project using Rust build manager `cargo`.
Echidnatron% cargo new my_project
This should create a new folder called `my_project`, and you can wander in there and type
Echidnatron% cargo run
This will do something reassuring but pointless, like print `Hello, world!`, because we haven't gotten differential dataflow involved yet. I mean, it's Rust and you could learn that, but you probably want to read a different web page in that case.
Instead, edit your `Cargo.toml` file, which tells Rust about your dependencies, to look like this:
Echidnatron% cat Cargo.toml
[package]
name = "my_project"
version = "0.1.0"
authors = ["Your Name <your_name@you.ch>"]
[dependencies]
timely = "0.11.1"
differential-dataflow = "0.11.0"
Echidnatron%
You should only need to add those last two lines there, which bring in dependencies on both [timely dataflow](https://github.com/TimelyDataflow/timely-dataflow) and [differential dataflow](https://github.com/TimelyDataflow/differential-dataflow). We will be using both of those.
If you would like to point at the most current code release, hosted on github, you can replace the dependencies with:
[dependencies]
timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" }
differential-dataflow = { git = "https://github.com/TimelyDataflow/differential-dataflow" }
You should now be ready to go. Code examples should mostly work, and you should complain (or [file an issue](https://github.com/TimelyDataflow/differential-dataflow/issues)) if they do not!

View File

@ -0,0 +1,80 @@
## Step 1: Write a program.
You write differential dataflow programs against apparently static input collections, with operations that look a bit like database (SQL) or big data (MapReduce) idioms. This is actually a bit of a trick, because you will have the ability to change the input data, but we'll pretend we don't know that yet.
Let's write a program with one input: a collection `manages` of pairs `(manager, person)` describing people and their direct reports. Our program will determine for each person their manager's manager (where the boss manages the boss's own self). If you are familiar with SQL, this is an "equijoin", and we will write exactly that in differential dataflow.
If you are following along at home, put this in your `src/main.rs` file.
```rust,no_run
extern crate timely;
extern crate differential_dataflow;
use differential_dataflow::input::InputSession;
use differential_dataflow::operators::Join;
fn main() {
// define a new timely dataflow computation.
timely::execute_from_args(std::env::args(), move |worker| {
// create an input collection of data.
let mut input = InputSession::new();
// define a new computation.
worker.dataflow(|scope| {
// create a new collection from our input.
let manages = input.to_collection(scope);
// if (m2, m1) and (m1, p), then output (m1, (m2, p))
manages
.map(|(m2, m1)| (m1, m2))
.join(&manages)
.inspect(|x| println!("{:?}", x));
});
// Read a size for our organization from the arguments.
let size = std::env::args().nth(1).unwrap().parse().unwrap();
// Load input (a binary tree).
input.advance_to(0);
for person in 0 .. size {
input.insert((person/2, person));
}
}).expect("Computation terminated abnormally");
}
```
This program has a bit of boilerplate, but at its heart it defines a new input `manages` and then joins it with itself, once the fields have been re-ordered. The intent is as stated in the comment:
```rust,no_run
// if (m2, m1) and (m1, p), then output (m1, (m2, p))
```
We want to report each pair `(m2, p)`, and we happen to also produce as evidence the `m1` connecting them.
When we execute this program we get to see the skip-level reports for the small binary tree we loaded as input:
Echidnatron% cargo run -- 10
Running `target/debug/my_project`
((0, (0, 0)), 0, 1)
((0, (0, 1)), 0, 1)
((1, (0, 2)), 0, 1)
((1, (0, 3)), 0, 1)
((2, (1, 4)), 0, 1)
((2, (1, 5)), 0, 1)
((3, (1, 6)), 0, 1)
((3, (1, 7)), 0, 1)
((4, (2, 8)), 0, 1)
((4, (2, 9)), 0, 1)
Echidnatron%
This is a bit crazy, but what we are seeing is many triples of the form
(data, time, diff)
describing how the data have *changed*. That's right; our input is actually a *change* from the initially empty input. The output is showing us that at time `(Root, 0)` several tuples have had their frequency incremented by one. That is a fancy way of saying they are the output.
This may make more sense in just a moment, when we want to *change* the input.

View File

@ -0,0 +1,100 @@
## Step 2: Change its input.
We've written a program, one that produces skip-level reports from some `manages` relation. Let's see how we can *change* its input, and what the corresponding output changes will be.
Our organization has gone from one where each manager has at most two reports, to one where each manager has three reports. Of course, this doesn't happen overnight; each day one of the employees will switch from their old manager to their new manager. Of course, the boss gets to stay the boss, because that is what boss means.
The only change we'll make is to add the following just after we load up our initial org chart:
```rust,no_run
for person in 1 .. size {
input.advance_to(person);
input.remove((person/2, person));
input.insert((person/3, person));
}
```
This moves us through new times, indicated by the line
```rust,no_run
input.advance_to(person);
```
which advances the state of the `input` collection up to a timestamp `person`, which just happens to be integers that are conveniently just larger than the time `0` we used to load the data.
Once we've advanced the time, we make some changes.
```rust,no_run
input.remove((person/2, person));
input.insert((person/3, person));
```
This removes the prior management relation, and introduces a new one where the person reports to their newer, more over-worked manager.
We do this for each of the non-boss employees and get to see a bunch of outputs.
Echidnatron% cargo run -- 10
Running `target/debug/my_project`
((0, (0, 0)), 0, 1)
((0, (0, 1)), 0, 1)
((0, (0, 2)), 2, 1)
((1, (0, 2)), 0, 1)
((1, (0, 2)), 2, -1)
((1, (0, 3)), 0, 1)
((1, (0, 4)), 4, 1)
((1, (0, 5)), 5, 1)
((2, (0, 4)), 2, 1)
((2, (0, 4)), 4, -1)
((2, (0, 5)), 2, 1)
((2, (0, 5)), 5, -1)
((2, (0, 6)), 6, 1)
((2, (0, 7)), 7, 1)
((2, (0, 8)), 8, 1)
((2, (1, 4)), 0, 1)
((2, (1, 4)), 2, -1)
((2, (1, 5)), 0, 1)
((2, (1, 5)), 2, -1)
((3, (1, 6)), 0, 1)
((3, (1, 6)), 6, -1)
((3, (1, 7)), 0, 1)
((3, (1, 7)), 7, -1)
((3, (1, 9)), 9, 1)
((4, (1, 8)), 4, 1)
((4, (1, 8)), 8, -1)
((4, (1, 9)), 4, 1)
((4, (1, 9)), 9, -1)
((4, (2, 8)), 0, 1)
((4, (2, 8)), 4, -1)
((4, (2, 9)), 0, 1)
((4, (2, 9)), 4, -1)
Echidnatron%
Gaaaaaaah! What in the !#$!?
It turns out our input changes result in output changes. Let's try and break this down and make some sense. If we group the columns by time, the second element of the tuples, we see a bit more structure.
1. The entries with time `0` are exactly the same as for our prior computation, where we just loaded the data.
2. There aren't any entries at time `1` (go check). That is because the input didn't change in our first step, because 1/2 == 1/3 == 0. Since the input didn't change, the output doesn't change.
3. The other times are more complicated.
Let's look at the entries for time `4`.
((1, (0, 4)), 4, 1)
((2, (0, 4)), 4, -1)
((4, (1, 8)), 4, 1)
((4, (1, 9)), 4, 1)
((4, (2, 8)), 4, -1)
((4, (2, 9)), 4, -1)
There is a bit going on here. Four's manager changed from two to one, and while their skip-level manager remained zero the explanation changed. The first two lines record this change. The next four lines record the change in the skip-level manager of four's reports, eight and nine.
At the end, time `9`, things are a bit simpler because we have reached the employees with no reports, and so the only changes are their skip-level manager, without any implications for other people.
((3, (1, 9)), 9, 1)
((4, (1, 9)), 9, -1)
Oof. Well, we probably *could* have figured these things out by hand, right?
Let's check out some ways this gets more interesting.

View File

@ -0,0 +1,124 @@
## Step 3: Make things more exciting.
We are going to make our example program a bit more exciting, in a few different ways.
### Increase the scale.
Ten people is a pretty small organization. Let's do ten million instead.
We are going to have to turn off the output printing here, so comment out the `inspect()` line (but keep the semicolon). Also, we'll need to add the `--release` flag to our command line, so that we optimize our binary and don't try running debug code for millions of steps.
We'll break down the results of our modified computation two ways, just loading up the initial computation, and then doing that plus all of the changes to the reporting structure. We haven't learned how to interactively load all of the input and await results yet (in just a moment), so we will only see elapsed times measuring the throughput, not the latency.
First, if we just produce the collection of skip-level management (with the step two code from before):
Echidnatron% time cargo run --release -- 10000000
cargo run --release --example hello 10000000 -w1 2.74s user 1.00s system 98% cpu 3.786 total
Echidnatron%
Four seconds. We have no clue if this is a good or bad time.
Second, if we produce the skip-level management and then modify it 10 million times (including the step two code from before):
Echidnatron% time cargo run --release -- 10000000
cargo run --release --example hello 10000000 10.64s user 2.22s system 99% cpu 12.939 total
Echidnatron%
About thirteen seconds now. Just over a microsecond per modification, though these are throughput rather than latency numbers.
### Increase the parallelism.
Differential dataflow works great using multiple threads. Produces the same output as with one thread and everything.
For this to work out, we'll want to ask each worker to load up a fraction of the input. Each timely dataflow worker has methods `index()` and `peers()`, which indicate the workers number and out of how many total workers. We can load our input up like so:
```rust,no_run
let mut person = worker.index();
while person < people {
input.insert((person/2, person));
person += worker.peers();
}
```
We can also make the same changes to the code that supplies the change, where each worker is responsible for those people whose number equals `worker.index()` modulo `worker.peers()`.
I'm on a laptop with two cores. Let's load the data again, without modifying it, but let's use two worker threads (with the `-w2` argument)
Echidnatron% time cargo run --release -- 10000000 -w2
cargo run --release --example hello 10000000 -w2 3.34s user 1.27s system 191% cpu 2.402 total
Echidnatron%
Now let's try loading and doing ten million modifications, but with two worker threads.
Echidnatron% time cargo run --release -- 10000000 -w2
cargo run --release --example hello 10000000 -w2 13.06s user 3.14s system 196% cpu 8.261 total
Echidnatron%
Each of these improve on the single-threaded execution (they do more total work, because). Perhaps amazingly, they even improve the case where we need to do ten million *sequential* modifications. We get exactly the same answer, too.
### Increase the interaction.
Instead of loading all of our changes and only waiting for the result, we can load each change and await its results before supplying the next change. This requires a bit of timely dataflow magic, where we add a probe to the end of our dataflow:
```rust,no_run
// create a manager
let probe = worker.dataflow(|scope| {
// create a new collection from an input session.
let manages = input.to_collection(scope);
// if (m2, m1) and (m1, p), then output (m1, (m2, p))
manages
.map(|(m2, m1)| (m1, m2))
.join(&manages)
// .inspect(|x| println!("{:?}", x))
.probe()
});
```
We can then use this probe to limit the introduction of new data, by waiting for it to catch up with our input before we insert new data:
```rust,no_run
// wait for data loading.
input.advance_to(1); input.flush();
while probe.less_than(&input.time()) { worker.step(); }
println!("{:?}\tdata loaded", timer.elapsed());
// make changes, but await completion.
let mut person = 1 + index;
while person < people {
input.remove((person/2, person));
input.insert((person/3, person));
input.advance_to(person); input.flush();
person += peers;
while probe.less_than(&input.time()) { worker.step(); }
println!("{:?}\tstep {} complete", timer.elapsed(), person);
}
```
This starts to print out a mess of data, indicating not only how long it takes to start up the computation, but also how long each individual round of updates takes.
Echidnatron% cargo run --release --example hello 10000000
Finished release [optimized + debuginfo] target(s) in 0.06s
Running `target/release/examples/hello 10000000`
4.092895186s data loaded
4.092975626s step 2 complete
4.093021676s step 3 complete
4.093041130s step 4 complete
4.093110803s step 5 complete
4.093144075s step 6 complete
4.093187645s step 7 complete
4.093208245s step 8 complete
4.093236460s step 9 complete
4.093281793s step 10 complete
which continues for quite a while.
21.689493445s step 397525 complete
21.689522815s step 397526 complete
21.689553410s step 397527 complete
21.689593500s step 397528 complete
21.689643055s step 397529 complete
You can see that this is pretty prompt; the latencies are in the tens of microseconds, but also that the whole computation is clearly going to take a bit longer. This is because we've forced some work to finish before we start the next work, which we haven't done before.

View File

@ -0,0 +1,5 @@
# When not to use Differential Dataflow
Differential dataflow is not magic. As you change your inputs, differential dataflow re-traces the computation to the best of its ability, and it will do work proportional to how much your *computation* has changed. Even if you feel that the results are qualitatively similar, the path required to reach these results may have changed substantially, and there is nothing differential dataflow will do other than re-play the computation wherever the inputs have changed.
To maintain its computations, differential dataflow tracks how your computation evolved. In many cases this is fine, but it is not hard to produce computations whose history is much larger than the amount of state at any one point in time. In these cases, differential dataflow may have a surprisingly large memory footprint.

View File

@ -0,0 +1,17 @@
# When to use Differential Dataflow
Differential dataflow is an "opinionated" programming framework, meaning that it aims to accommodate certain tasks and is unapologetic about its relatively uselessness for other tasks. To become delighted with differential dataflow it is important to set expectations appropriately, and to do that we need to understand what it tries to do well.
Differential dataflow is aimed at combinatorial algorithms over collections, a typical example being large scale graph computation, perhaps computing and maintaining the set of connected components in a graph. As we will see, this covers a fair bit of ground, from standard big data computations like SQL and MapReduce, up through deductive reasoning systems and some forms of structured machine learning.
* **Functional programming**: Differential dataflow operators are *functional*, in that they transform their inputs into outputs without modifying the inputs themselves. Functional operators (and programs) allow us to more easily understand the changes a program undergoes when its input changes. While functional programming is restrictive, the benefits we get in exchange (efficient distributed, iterative, and incremental execution) are powerful and potentially worth the cognitive shift.
* **Data Parallelism**: Differential dataflow operators are largely "data-parallel", meaning that they can operate on disjoint parts of the input independently. This allows many big data frameworks to distribute work across multiple workers, which we do as well, but perhaps more importantly it allows us to constrain the flow of updates and only perform *re*-computation for values that have changed.
* **Iteration**: The most significant technical departure from prior work is the ability to compute and maintain iterative computations. This is unlike most database and big data processors. The ability to perform efficient iteration allows us to support computations with non-trivial control flow.
* **Incremental updates**: Differential dataflow maintains computations as their inputs change. This maintenance work is often substantially lower than the cost to fully re-evaluate a computation from scratch. Differential dataflow is specifically engineered to provide both high throughput and low latency, at the same time.
These features come together to provide a platform with functionality not yet found in other solutions. If your problem requires these features, or would benefit from them at all, differential dataflow is worth investigating.
The scope of problems well-accommodated by differential dataflow is only growing with time, as we learn more about problems where these needs align, and implement more algorithms and operators in differential dataflow. Ideally this document will give you some ideas about how to find uses in your areas of interest.

View File

@ -0,0 +1,4 @@
# Chapter 1: Building Differential Dataflows
Unlike timely dataflow, differential dataflow is meant to be something you can start using without understanding much about how it works. This is great news for us, because under the covers it is way more complicated than you might hope. We'll talk about that too, later, but for now let's just talk about how to create differential dataflow computations.

View File

@ -0,0 +1,29 @@
# Timely Skeleton
Differential dataflow computations are really just [timely dataflow](https://github.com/frankmcsherry/timely-dataflow) computations where we supply a sweet set of operators and idioms for you. As such, when you build a new differential dataflow computation it will need to have a timely dataflow skeleton built first. For example:
```rust,ignore
extern crate timely;
extern crate differential_dataflow;
fn main() {
// prepare a timely dataflow execution environment.
timely::execute_from_args(std::env::args(), |worker| {
// create a differential dataflow.
let mut input = worker.dataflow::<usize,_,_>(|scope| {
// create inputs, build dataflow, return input handle.
});
// drive the input around here.
}).unwrap();
}
```
This is a pretty standard skeleton, where our program immediately starts up a timely dataflow instance by defining what each independent worker should do. A standard pattern, seen above, is to have each worker construct a dataflow and then drive the inputs around somehow.
We'll get more specific in just a moment.

View File

@ -0,0 +1,70 @@
# Input Collections
Differential dataflow computations all start from user-controlled input "collections".
For now, let's think of an input collection as a multiset (or a "bag"): a collection of typed elements which may contain duplicates. It can be a bit more complicated than this, for neat and interesting reasons, but life is easier for now if you think of it as a multiset.
## An example
Let's take our skeleton from the previous subsection and add an input collection.
```rust,no_run
extern crate timely;
extern crate differential_dataflow;
use differential_dataflow::input::Input;
fn main() {
// define a new computational scope, in which to run BFS
timely::execute_from_args(std::env::args(), |worker| {
// create a counting differential dataflow.
let mut input = worker.dataflow::<usize,_,_>(|scope| {
// create inputs, build dataflow, return stuff.
let (input, words) = scope.new_collection();
words.inspect(|x| println!("seen: {:?}", x));
input
});
// drive the input around here.
}).unwrap();
}
```
Here we've created a new input collection in `scope`, which returns a pair `(input, words)`. The first of these, `input`, is a handle that our program can use to change the collection; this is how we insert and remove records. The second, `words`, is a differential dataflow collection on which we can base further differential dataflow computation. These two are linked so that changes our program makes to `input` result in changes to `words` which then result in changes to derived collections.
This isn't a wildly interesting program yet, because we haven't actually changed `input`. Let's do that now, where the code currently says
// drive the input around here.
Differential dataflow inputs are similar to timely dataflow inputs, if you are familiar with those, but with a few important tweaks. Each input has a "time" it is currently set to. You can `insert(item)` and `remove(item)` to your hearts content, and these changes will take effect at the time currently associated with the input.
For example, we could write:
```rust,no_run
// drive the input around here.
input.insert("hello".to_string());
input.insert("world".to_string());
input.advance_to(1);
input.remove("hello".to_string());
input.insert("goodbye".to_string());
input.advance_to(2);
```
At this point we have described some insertions at time `0` (the initial time for each input) and some changes at time `1`.
## General updates
While the `insert` and `remove` methods are generally useful, there is a more general method `input.update(data, diff)` which allows you to specify a difference directly. This can be useful if your data are natively described as differences, if your differences are large in magnitude, or in exotic cases where you are not using multisets (?!?).
## Advancing time
The calls to `input.advance_to()` are surprisingly important. Not only do they indicate that you would like your next changes to happen at a particular new time, these are the calls that signal to the system that you have finished changing the collection at times prior to your new time. It is not until you advance the time of your inputs that the system can determine the correct answer, because until then you could show up with arbitrary changes.
It is crucial to call `input.advance_to(time)` if you want to see the output changes for `time`.
## Flushing
The calls to `insert()`, `remove()`, and `advance_to()` are buffered in the interest of efficiency, and you may need to call `input.flush()` to ensure that every change you've applied to the input is visible to the system. This is unlike timely dataflow, which does not buffer its `advance_to` calls.

View File

@ -0,0 +1,97 @@
# Operators
Differential dataflow operators are how we build more interesting programs. There are relatively few operators, and the game is to figure out how to use them to do what you need. Let's now go through a few of the operators!
## Map
The `map` operator is like the map operator in Rust's `Iterator` trait, and the map method in timely dataflow: it takes a closure that can be applied to each record to transform it to some new output. For example, we might write:
```rust,ignore
collection
.map(|word| word.len())
.count()
```
which would transform each of our words into their length, and then count the number of times each length appears. This would give us a shorter synopses of our data, if we don't actually care about the word itself.
The map operator is also used for "projection" in the database sense, where we have an input records containing many fields and we just want to select out some of them for further computation. This is handy to do as soon as possible, so that the next bit of computation only needs to manage the data relevant to its interests.
## Filter
The `filter` operator is also like that of Rust's `Iterator` trait and the method in timely dataflow: it takes a closure that reports "true" or "false" for each input record, and passes only those records that result in true. If we only wanted to keep relatively short strings, we might write
```rust,ignore
collection
.filter(|word| word.len() < 8)
.count()
```
Perhaps we want to count all words but only report those words whose count is at least 100.
```rust,ignore
collection
.count()
.filter(|(word, count)| count >= 100)
.map(|(word, _count)| word);
```
This computation has the potential to *change* much less frequently than its input changes. Even though counts may go up and down, it is only when a word's frequency crosses the 100 boundary that we report a change.
## Concat
The `concat` operator merges two collections together, essentially adding the occurrence frequencies of each record. For example, we might have words coming in from two sources, that perhaps change independently:
```rust,ignore
collection1
.concat(&collection2)
.count()
```
## Join
The `join` operator is one of the first exotic operations that isn't just a record-by-record operation. The join method only acts on two collections, each of which must have data of the form `(key, val1)` and `(key, val2)`. That is, their data must be pairs, and the type of the first element must match.
The join operator results in a collection whose data have type `(key, val1, val2)` where there is an output triple for each pair of records in the two inputs with matching key. More specifically, the number of times the record `(key, val1, val2)` appears in the output is the product of the number of times `(key, val1)` and `(key, val2)` appear in their respective inputs.
Join has a lot of uses, but one common example is to "look up" data. If we have a collection containing pairs `(person, address)`, we can use joins against this relation to recover the address of a person (perhaps we are trying to deliver a package to them):
```rust,ignore
let deliver_to =
ordered_by
.join(&person_address)
.map(|(person, package, address)| (package, address));
```
Alternately, we can use the same relation to find people living at a given address (perhaps because we want to allow any of them to sign for the package we want to deliver).
```rust,ignore
let can_sign_for =
deliver_to
.map(|(package, address)| (address, package))
.join(&person_address.map(|(p,a)| (a,p)))
.map(|(address, package, person)| (package, person));
```
As the underlying `ordered_by` and `person_address` collections change, the derived `deliver_to` and `can_sign_for` collections will change as well, maintaining correct and consistent results corresponding to the inputs.
## Reduce
The `reduce` operator applies to one input collection whose records have the form `(key, val)`, and it allows you produce output as an arbitrary function of the key and the list of values. The following example starts from the list of all orders, and produces any duplicate packages in the ordering system (those with count two or greater).
```rust,ignore
ordered_by
.map(|(package, person)| (person, package))
.reduce(|person, packages, duplicates| {
for (package, count) in packages.iter() {
if count > 1 {
duplicates.push((package.clone(), count));
}
}
});
```
There are some subtle details here, ones that will likely trip you up (as they trip up me):
The second and third arguments (the input and output, here `packages` and `duplicates`) contain pairs `(val, count)`. This is great when we want to count things that occur many times (in that `("word", 1000000)` is more succinct than one million copies of `"word"), but in casual use we need to remember that even when we expect the numbers to be mostly one, we need to use them.
In actual fact the input (`packages`) contains pairs of type `(&Val, Count)`, which in Rust-isms mean that you only get to view the associated value, you do not get to take ownership of it. This means that if we want to reproduce it in the output we need to do something like `.clone()` to get a new copy. If it were a string, or had other allocated data behind it, our read-only access to that data means we need to spend the time to create new copies for the output.

View File

@ -0,0 +1,107 @@
# Iteration
One of the most interesting parts of differential dataflow is its support for iterative computation.
Iteration is accomplished in differential dataflow with the `iterate` operator, a method that takes one collection as an argument, and logic describing how you would transform that collection as one step of iteration. The operator then applies this logic to the input collection indefinitely, producing as output the "fixed point" of your logic starting from your input collection.
That might sound a bit complicated, so let's look at an example.
Perhaps you have a collections of `(Manager, Employee)` pairs indicating who manages whom. Perhaps you are also really excited to produce (and maintain) the number of employees in total under each manager.
Let's describe an iterative computation that starts from manager-employee relation, and repeatedly expands it to include transitive management relationships.
```rust,ignore
manager_employee
.iterate(|manages| {
// if x manages y, and y manages z, then x manages z (transitively).
manages
.map(|(x, y)| (y, x))
.join(&manages)
.map(|(y, x, z)| (x, z))
});
```
This is a simplistic use of `iterate`, and your needs may become more complicated.
## General Iteration
More generally, you may want to describe iterative computations with i. multiple inputs, ii. multiple loop variables, and iii. multiple outputs. Differential dataflow provides a way to manually construct iterative contexts, where you can do all of these things.
Timely dataflow provides "scopes", which are nested dataflows in which you can augment timestamps with further information, for example a "round of iteration" for a loop. Our first step to construct a general iterative computation is to create a new scope within the current scope.
```rust,ignore
// if you don't otherwise have the scope ..
let scope = manager_employee.scope();
scope.scoped(|subscope| {
// More stuff will go here
});
```
Each timely dataflow stream, and differential dataflow collection, are associated with a scope. To use a collection that is outside our subscope, we will need to bring it in to the subscope. This is done with the `enter` operator.
```rust,ignore
// if you don't otherwise have the scope ..
let scope = manager_employee.scope();
scope.scoped(|subscope| {
// we can now use m_e in this scope.
let m_e = manager_employee.enter(subscope);
});
```
To create an iterative computation, we now need to define some variables that can be updated in each round of iteration. Differential dataflow provides a [Variable](https://github.com/frankmcsherry/differential-dataflow/blob/master/src/operators/iterate.rs#L132-L137) struct that does exactly this. We create a variable by specifying its initial value (a collection), and then `set` the definition of the collection which will instruct it how to update.
```rust,ignore
// if you don't otherwise have the scope ..
let scope = manager_employee.scope();
scope.scoped(|subscope| {
// we can now use m_e in this scope.
let m_e = manager_employee.enter(subscope);
let variable = Variable::from(m_e);
let step =
variable
.map(|(x, y)| (y, x))
.join(&variable)
.map(|(y, x, z)| (x, z));
variable.set(step);
});
```
Finally, we probably want to return the final value of the variable, what it converges to (assuming it does so). There is a `leave` operator that matches the `enter` operator we used to bring data into the scope; it produces the final value of the collection it is called on:
```rust,ignore
// if you don't otherwise have the scope ..
let scope = manager_employee.scope();
let result =
scope.scoped(|subscope| {
// we can now use m_e in this scope.
let m_e = manager_employee.enter(subscope);
let variable = Variable::from(m_e);
let step =
variable
.map(|(x, y)| (y, x))
.join(&variable)
.map(|(y, x, z)| (x, z));
variable
.set(step)
.leave()
});
```
Although a bit more verbose, this is (or should be) the same computation we described up above with the `iterate` method. However, if you found you needed to use more inputs, outputs, or variables, it will be here for you.

Some files were not shown because too many files have changed in this diff Show More