├── .gitignore ├── README.md ├── differential-dataflow ├── Cargo.lock ├── Cargo.toml ├── README.md ├── run.sh ├── shell.nix └── src │ └── main.rs ├── flink-datastream ├── README.md ├── pom.xml ├── run.sh ├── shell.nix └── src │ └── main │ └── java │ └── net │ └── scattered_thoughts │ └── streaming_consistency │ ├── Demo.java │ ├── JsonDeserializationSchema.java │ └── TransactionsSource.java ├── flink-table ├── README.md ├── pom.xml ├── run.sh ├── shell.nix └── src │ └── main │ └── java │ └── Demo.java ├── graph.nix ├── graph.r ├── graph.sh ├── kafka-streams ├── README.md ├── pom.xml ├── run.sh ├── shell.nix └── src │ └── main │ └── java │ ├── Demo.java │ └── JsonTimestampExtractor.java ├── ksqldb ├── README.md ├── docker-compose.yml ├── run.sh ├── shell.nix ├── views1.sql └── views2.sql ├── materialize ├── README.md ├── run.sh ├── shell.nix └── views.sql ├── original-transactions.py ├── pkgs.nix ├── simplified-transactions.py └── spark-structured-streaming ├── README.md ├── pom.xml ├── run.sh ├── shell.nix └── src └── main └── java └── Demo.java /.gitignore: -------------------------------------------------------------------------------- 1 | tmp 2 | target 3 | dependency-reduced-pom.xml 4 | 5 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | Demonstrations of (in)consistency in various streaming systems, to accompany the article [Internal consistency in streaming systems](https://scattered-thoughts.net/writing/internal-consistency-in-streaming-systems/) 2 | -------------------------------------------------------------------------------- /differential-dataflow/Cargo.lock: -------------------------------------------------------------------------------- 1 | # This file is automatically @generated by Cargo. 2 | # It is not intended for manual editing. 3 | [[package]] 4 | name = "abomonation" 5 | version = "0.7.3" 6 | source = "registry+https://github.com/rust-lang/crates.io-index" 7 | checksum = "56e72913c99b1f927aa7bd59a41518fdd9995f63ffc8760f211609e0241c4fb2" 8 | 9 | [[package]] 10 | name = "abomonation_derive" 11 | version = "0.5.0" 12 | source = "registry+https://github.com/rust-lang/crates.io-index" 13 | checksum = "e50e2a046af56a864c62d97b7153fda72c596e646be1b0c7963736821f6e1efa" 14 | dependencies = [ 15 | "proc-macro2", 16 | "quote", 17 | "synstructure", 18 | ] 19 | 20 | [[package]] 21 | name = "autocfg" 22 | version = "1.0.1" 23 | source = "registry+https://github.com/rust-lang/crates.io-index" 24 | checksum = "cdb031dd78e28731d87d56cc8ffef4a8f36ca26c38fe2de700543e627f8a464a" 25 | 26 | [[package]] 27 | name = "bincode" 28 | version = "1.3.2" 29 | source = "registry+https://github.com/rust-lang/crates.io-index" 30 | checksum = "d175dfa69e619905c4c3cdb7c3c203fa3bdd5d51184e3afdb2742c0280493772" 31 | dependencies = [ 32 | "byteorder", 33 | "serde", 34 | ] 35 | 36 | [[package]] 37 | name = "byteorder" 38 | version = "1.3.4" 39 | source = "registry+https://github.com/rust-lang/crates.io-index" 40 | checksum = "08c48aae112d48ed9f069b33538ea9e3e90aa263cfa3d1c24309612b1f7472de" 41 | 42 | [[package]] 43 | name = "cfg-if" 44 | version = "1.0.0" 45 | source = "registry+https://github.com/rust-lang/crates.io-index" 46 | checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" 47 | 48 | [[package]] 49 | name = "chrono" 50 | version = "0.4.19" 51 | source = "registry+https://github.com/rust-lang/crates.io-index" 52 | checksum = "670ad68c9088c2a963aaa298cb369688cf3f9465ce5e2d4ca10e6e0098a1ce73" 53 | dependencies = [ 54 | "libc", 55 | "num-integer", 56 | "num-traits", 57 | "time", 58 | "winapi", 59 | ] 60 | 61 | [[package]] 62 | name = "crossbeam-channel" 63 | version = "0.5.0" 64 | source = "registry+https://github.com/rust-lang/crates.io-index" 65 | checksum = "dca26ee1f8d361640700bde38b2c37d8c22b3ce2d360e1fc1c74ea4b0aa7d775" 66 | dependencies = [ 67 | "cfg-if", 68 | "crossbeam-utils", 69 | ] 70 | 71 | [[package]] 72 | name = "crossbeam-utils" 73 | version = "0.8.3" 74 | source = "registry+https://github.com/rust-lang/crates.io-index" 75 | checksum = "e7e9d99fa91428effe99c5c6d4634cdeba32b8cf784fc428a2a687f61a952c49" 76 | dependencies = [ 77 | "autocfg", 78 | "cfg-if", 79 | "lazy_static", 80 | ] 81 | 82 | [[package]] 83 | name = "differential-dataflow" 84 | version = "0.12.0" 85 | source = "registry+https://github.com/rust-lang/crates.io-index" 86 | checksum = "cecb0345111032cfd995a1e9c1b79387a0e6bf6690be5d8dd12a58f4861bc6d9" 87 | dependencies = [ 88 | "abomonation", 89 | "abomonation_derive", 90 | "fnv", 91 | "serde", 92 | "serde_derive", 93 | "timely", 94 | ] 95 | 96 | [[package]] 97 | name = "differential-dataflow-demo" 98 | version = "0.1.0" 99 | dependencies = [ 100 | "chrono", 101 | "differential-dataflow", 102 | "serde", 103 | "serde_json", 104 | "timely", 105 | ] 106 | 107 | [[package]] 108 | name = "fnv" 109 | version = "1.0.7" 110 | source = "registry+https://github.com/rust-lang/crates.io-index" 111 | checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" 112 | 113 | [[package]] 114 | name = "futures-core" 115 | version = "0.3.13" 116 | source = "registry+https://github.com/rust-lang/crates.io-index" 117 | checksum = "15496a72fabf0e62bdc3df11a59a3787429221dd0710ba8ef163d6f7a9112c94" 118 | 119 | [[package]] 120 | name = "futures-macro" 121 | version = "0.3.13" 122 | source = "registry+https://github.com/rust-lang/crates.io-index" 123 | checksum = "ea405816a5139fb39af82c2beb921d52143f556038378d6db21183a5c37fbfb7" 124 | dependencies = [ 125 | "proc-macro-hack", 126 | "proc-macro2", 127 | "quote", 128 | "syn", 129 | ] 130 | 131 | [[package]] 132 | name = "futures-task" 133 | version = "0.3.13" 134 | source = "registry+https://github.com/rust-lang/crates.io-index" 135 | checksum = "fa189ef211c15ee602667a6fcfe1c1fd9e07d42250d2156382820fba33c9df80" 136 | 137 | [[package]] 138 | name = "futures-util" 139 | version = "0.3.13" 140 | source = "registry+https://github.com/rust-lang/crates.io-index" 141 | checksum = "1812c7ab8aedf8d6f2701a43e1243acdbcc2b36ab26e2ad421eb99ac963d96d1" 142 | dependencies = [ 143 | "futures-core", 144 | "futures-macro", 145 | "futures-task", 146 | "pin-project-lite", 147 | "pin-utils", 148 | "proc-macro-hack", 149 | "proc-macro-nested", 150 | "slab", 151 | ] 152 | 153 | [[package]] 154 | name = "getopts" 155 | version = "0.2.21" 156 | source = "registry+https://github.com/rust-lang/crates.io-index" 157 | checksum = "14dbbfd5c71d70241ecf9e6f13737f7b5ce823821063188d7e46c41d371eebd5" 158 | dependencies = [ 159 | "unicode-width", 160 | ] 161 | 162 | [[package]] 163 | name = "itoa" 164 | version = "0.4.7" 165 | source = "registry+https://github.com/rust-lang/crates.io-index" 166 | checksum = "dd25036021b0de88a0aff6b850051563c6516d0bf53f8638938edbb9de732736" 167 | 168 | [[package]] 169 | name = "lazy_static" 170 | version = "1.4.0" 171 | source = "registry+https://github.com/rust-lang/crates.io-index" 172 | checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" 173 | 174 | [[package]] 175 | name = "libc" 176 | version = "0.2.93" 177 | source = "registry+https://github.com/rust-lang/crates.io-index" 178 | checksum = "9385f66bf6105b241aa65a61cb923ef20efc665cb9f9bb50ac2f0c4b7f378d41" 179 | 180 | [[package]] 181 | name = "num-integer" 182 | version = "0.1.44" 183 | source = "registry+https://github.com/rust-lang/crates.io-index" 184 | checksum = "d2cc698a63b549a70bc047073d2949cce27cd1c7b0a4a862d08a8031bc2801db" 185 | dependencies = [ 186 | "autocfg", 187 | "num-traits", 188 | ] 189 | 190 | [[package]] 191 | name = "num-traits" 192 | version = "0.2.14" 193 | source = "registry+https://github.com/rust-lang/crates.io-index" 194 | checksum = "9a64b1ec5cda2586e284722486d802acf1f7dbdc623e2bfc57e65ca1cd099290" 195 | dependencies = [ 196 | "autocfg", 197 | ] 198 | 199 | [[package]] 200 | name = "pin-project-lite" 201 | version = "0.2.6" 202 | source = "registry+https://github.com/rust-lang/crates.io-index" 203 | checksum = "dc0e1f259c92177c30a4c9d177246edd0a3568b25756a977d0632cf8fa37e905" 204 | 205 | [[package]] 206 | name = "pin-utils" 207 | version = "0.1.0" 208 | source = "registry+https://github.com/rust-lang/crates.io-index" 209 | checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" 210 | 211 | [[package]] 212 | name = "proc-macro-hack" 213 | version = "0.5.19" 214 | source = "registry+https://github.com/rust-lang/crates.io-index" 215 | checksum = "dbf0c48bc1d91375ae5c3cd81e3722dff1abcf81a30960240640d223f59fe0e5" 216 | 217 | [[package]] 218 | name = "proc-macro-nested" 219 | version = "0.1.7" 220 | source = "registry+https://github.com/rust-lang/crates.io-index" 221 | checksum = "bc881b2c22681370c6a780e47af9840ef841837bc98118431d4e1868bd0c1086" 222 | 223 | [[package]] 224 | name = "proc-macro2" 225 | version = "1.0.26" 226 | source = "registry+https://github.com/rust-lang/crates.io-index" 227 | checksum = "a152013215dca273577e18d2bf00fa862b89b24169fb78c4c95aeb07992c9cec" 228 | dependencies = [ 229 | "unicode-xid", 230 | ] 231 | 232 | [[package]] 233 | name = "quote" 234 | version = "1.0.9" 235 | source = "registry+https://github.com/rust-lang/crates.io-index" 236 | checksum = "c3d0b9745dc2debf507c8422de05d7226cc1f0644216dfdfead988f9b1ab32a7" 237 | dependencies = [ 238 | "proc-macro2", 239 | ] 240 | 241 | [[package]] 242 | name = "ryu" 243 | version = "1.0.5" 244 | source = "registry+https://github.com/rust-lang/crates.io-index" 245 | checksum = "71d301d4193d031abdd79ff7e3dd721168a9572ef3fe51a1517aba235bd8f86e" 246 | 247 | [[package]] 248 | name = "serde" 249 | version = "1.0.125" 250 | source = "registry+https://github.com/rust-lang/crates.io-index" 251 | checksum = "558dc50e1a5a5fa7112ca2ce4effcb321b0300c0d4ccf0776a9f60cd89031171" 252 | dependencies = [ 253 | "serde_derive", 254 | ] 255 | 256 | [[package]] 257 | name = "serde_derive" 258 | version = "1.0.125" 259 | source = "registry+https://github.com/rust-lang/crates.io-index" 260 | checksum = "b093b7a2bb58203b5da3056c05b4ec1fed827dcfdb37347a8841695263b3d06d" 261 | dependencies = [ 262 | "proc-macro2", 263 | "quote", 264 | "syn", 265 | ] 266 | 267 | [[package]] 268 | name = "serde_json" 269 | version = "1.0.64" 270 | source = "registry+https://github.com/rust-lang/crates.io-index" 271 | checksum = "799e97dc9fdae36a5c8b8f2cae9ce2ee9fdce2058c57a93e6099d919fd982f79" 272 | dependencies = [ 273 | "itoa", 274 | "ryu", 275 | "serde", 276 | ] 277 | 278 | [[package]] 279 | name = "slab" 280 | version = "0.4.2" 281 | source = "registry+https://github.com/rust-lang/crates.io-index" 282 | checksum = "c111b5bd5695e56cffe5129854aa230b39c93a305372fdbb2668ca2394eea9f8" 283 | 284 | [[package]] 285 | name = "syn" 286 | version = "1.0.69" 287 | source = "registry+https://github.com/rust-lang/crates.io-index" 288 | checksum = "48fe99c6bd8b1cc636890bcc071842de909d902c81ac7dab53ba33c421ab8ffb" 289 | dependencies = [ 290 | "proc-macro2", 291 | "quote", 292 | "unicode-xid", 293 | ] 294 | 295 | [[package]] 296 | name = "synstructure" 297 | version = "0.12.4" 298 | source = "registry+https://github.com/rust-lang/crates.io-index" 299 | checksum = "b834f2d66f734cb897113e34aaff2f1ab4719ca946f9a7358dba8f8064148701" 300 | dependencies = [ 301 | "proc-macro2", 302 | "quote", 303 | "syn", 304 | "unicode-xid", 305 | ] 306 | 307 | [[package]] 308 | name = "time" 309 | version = "0.1.44" 310 | source = "registry+https://github.com/rust-lang/crates.io-index" 311 | checksum = "6db9e6914ab8b1ae1c260a4ae7a49b6c5611b40328a735b21862567685e73255" 312 | dependencies = [ 313 | "libc", 314 | "wasi", 315 | "winapi", 316 | ] 317 | 318 | [[package]] 319 | name = "timely" 320 | version = "0.12.0" 321 | source = "registry+https://github.com/rust-lang/crates.io-index" 322 | checksum = "211e9686a84038d2b052c1f70d7423f8716330fb7108fe21a8a9b0c2929b80a7" 323 | dependencies = [ 324 | "abomonation", 325 | "abomonation_derive", 326 | "crossbeam-channel", 327 | "futures-util", 328 | "getopts", 329 | "serde", 330 | "serde_derive", 331 | "timely_bytes", 332 | "timely_communication", 333 | "timely_logging", 334 | ] 335 | 336 | [[package]] 337 | name = "timely_bytes" 338 | version = "0.12.0" 339 | source = "registry+https://github.com/rust-lang/crates.io-index" 340 | checksum = "1f8a545b35b019b8b63afb913214695f542cbbf39032c7b7349e1a15b7f2a2cf" 341 | 342 | [[package]] 343 | name = "timely_communication" 344 | version = "0.12.0" 345 | source = "registry+https://github.com/rust-lang/crates.io-index" 346 | checksum = "3435bab8a9b9909b5bc907cc562b3659c62a05c1abbc1268e914b024b66dff60" 347 | dependencies = [ 348 | "abomonation", 349 | "abomonation_derive", 350 | "bincode", 351 | "crossbeam-channel", 352 | "getopts", 353 | "serde", 354 | "serde_derive", 355 | "timely_bytes", 356 | "timely_logging", 357 | ] 358 | 359 | [[package]] 360 | name = "timely_logging" 361 | version = "0.12.0" 362 | source = "registry+https://github.com/rust-lang/crates.io-index" 363 | checksum = "f9046af28827ac831479d245eb8afd9522599a3cbb22d6c42a82cb9e4ccdf858" 364 | 365 | [[package]] 366 | name = "unicode-width" 367 | version = "0.1.8" 368 | source = "registry+https://github.com/rust-lang/crates.io-index" 369 | checksum = "9337591893a19b88d8d87f2cec1e73fad5cdfd10e5a6f349f498ad6ea2ffb1e3" 370 | 371 | [[package]] 372 | name = "unicode-xid" 373 | version = "0.2.1" 374 | source = "registry+https://github.com/rust-lang/crates.io-index" 375 | checksum = "f7fe0bb3479651439c9112f72b6c505038574c9fbb575ed1bf3b797fa39dd564" 376 | 377 | [[package]] 378 | name = "wasi" 379 | version = "0.10.0+wasi-snapshot-preview1" 380 | source = "registry+https://github.com/rust-lang/crates.io-index" 381 | checksum = "1a143597ca7c7793eff794def352d41792a93c481eb1042423ff7ff72ba2c31f" 382 | 383 | [[package]] 384 | name = "winapi" 385 | version = "0.3.9" 386 | source = "registry+https://github.com/rust-lang/crates.io-index" 387 | checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" 388 | dependencies = [ 389 | "winapi-i686-pc-windows-gnu", 390 | "winapi-x86_64-pc-windows-gnu", 391 | ] 392 | 393 | [[package]] 394 | name = "winapi-i686-pc-windows-gnu" 395 | version = "0.4.0" 396 | source = "registry+https://github.com/rust-lang/crates.io-index" 397 | checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" 398 | 399 | [[package]] 400 | name = "winapi-x86_64-pc-windows-gnu" 401 | version = "0.4.0" 402 | source = "registry+https://github.com/rust-lang/crates.io-index" 403 | checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" 404 | -------------------------------------------------------------------------------- /differential-dataflow/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "differential-dataflow-demo" 3 | version = "0.1.0" 4 | authors = ["Jamie Brandon "] 5 | edition = "2018" 6 | 7 | [dependencies] 8 | timely = { version = "0.12.0", default-features = false, features = ["bincode"] } 9 | differential-dataflow = "0.12.0" 10 | chrono = "0.4.19" 11 | serde = { version = "1.0.59", features = ["derive"] } 12 | serde_json = "1.0.59" -------------------------------------------------------------------------------- /differential-dataflow/README.md: -------------------------------------------------------------------------------- 1 | Linux-only. Requires [nix](https://nixos.org/) to fetch dependencies. 2 | 3 | ``` 4 | nix-shell --pure --run './run.sh ../original-transactions.py' 5 | ``` 6 | 7 | You should eventually see `Done!`. 8 | 9 | Check out the outputs in `./tmp`. -------------------------------------------------------------------------------- /differential-dataflow/run.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | set -ue 4 | 5 | DATAGEN=$1 6 | 7 | THIS_DIR="$(cd "$(dirname "$0")"; pwd -P)" 8 | 9 | DATA_DIR=$THIS_DIR/tmp 10 | echo "Data will be stored in $DATA_DIR" 11 | rm -rf $DATA_DIR/* 12 | mkdir -p $DATA_DIR/{config,logs} 13 | 14 | echo "Feeding inputs" 15 | $DATAGEN | cut -d'|' -f2 > $DATA_DIR/transactions 16 | 17 | echo "Starting demo" 18 | cargo run --release -- -w 4 19 | 20 | echo "Done!" -------------------------------------------------------------------------------- /differential-dataflow/shell.nix: -------------------------------------------------------------------------------- 1 | let 2 | 3 | pkgs = import ../pkgs.nix; 4 | 5 | in 6 | 7 | pkgs.mkShell { 8 | buildInputs = [ 9 | pkgs.cargo 10 | 11 | pkgs.python38 12 | ]; 13 | } -------------------------------------------------------------------------------- /differential-dataflow/src/main.rs: -------------------------------------------------------------------------------- 1 | use chrono::{Duration, NaiveDateTime}; 2 | use differential_dataflow::input::InputSession; 3 | use differential_dataflow::operators::*; 4 | use serde::{Deserialize, Serialize}; 5 | use serde_json::Value; 6 | use std::fs::File; 7 | use std::io::prelude::*; 8 | use std::io::{BufRead, BufReader}; 9 | use std::sync::{Arc, Mutex}; 10 | use timely::dataflow::operators::capture::capture::Capture; 11 | use timely::dataflow::operators::capture::event::Event; 12 | use timely::dataflow::operators::exchange::Exchange; 13 | 14 | #[derive(Clone, Copy, PartialEq, PartialOrd, Eq, Ord, Debug, Serialize, Deserialize, Hash)] 15 | struct Transaction { 16 | id: i64, 17 | from_account: i64, 18 | to_account: i64, 19 | // This should be f64 20 | // but f64 does not impl Ord 21 | // and OrderedFloat does not impl Abomonate or Serialize 22 | // and orphan rules prevent us from fixing this without writing our own float wrapper 23 | amount: i64, 24 | // This should be NaiveDateTime but that also does not impl Serialize 25 | ts: i64, 26 | } 27 | 28 | fn main() { 29 | let handles = Arc::new(Mutex::new(vec![])); 30 | timely::execute_from_args(std::env::args(), { 31 | let handles = handles.clone(); 32 | move |worker| { 33 | let worker_index = worker.index(); 34 | 35 | let mut transactions: InputSession<_, Transaction, isize> = InputSession::new(); 36 | 37 | worker.dataflow(|scope| { 38 | let transactions = transactions.to_collection(scope); 39 | sink_to_file( 40 | worker_index, 41 | &handles, 42 | "accepted_transactions", 43 | &transactions, 44 | ); 45 | 46 | let debits = sum(transactions.map(|t| (t.from_account, t.amount))); 47 | sink_to_file(worker_index, &handles, "debits", &debits); 48 | 49 | let credits = sum(transactions.map(|t| (t.to_account, t.amount))); 50 | sink_to_file(worker_index, &handles, "credits", &credits); 51 | 52 | let balance = debits 53 | .join(&credits) 54 | .map(|(account, (credits, debits))| (account, credits - debits)); 55 | sink_to_file(worker_index, &handles, "balance", &balance); 56 | 57 | let total = sum(balance.map(|(_, balance)| ((), balance as i64))); 58 | sink_to_file(worker_index, &handles, "total", &total); 59 | }); 60 | 61 | if worker_index == 0 { 62 | let mut watermark = 0; 63 | let transactions_file = File::open("./tmp/transactions").unwrap(); 64 | for (i, line) in BufReader::new(transactions_file).lines().enumerate() { 65 | let line = line.unwrap(); 66 | let json: Value = serde_json::from_str(&line).unwrap(); 67 | let transaction = Transaction { 68 | id: json["id"].as_i64().unwrap(), 69 | from_account: json["from_account"].as_i64().unwrap(), 70 | to_account: json["to_account"].as_i64().unwrap(), 71 | amount: json["amount"].as_i64().unwrap(), 72 | ts: NaiveDateTime::parse_from_str( 73 | json["ts"].as_str().unwrap(), 74 | "%Y-%m-%d %H:%M:%S%.f", 75 | ) 76 | .unwrap() 77 | .timestamp_nanos(), 78 | }; 79 | // the watermark runs 5 seconds behind the most recently seen data 80 | watermark = watermark 81 | .max(transaction.ts - Duration::seconds(5).num_nanoseconds().unwrap()); 82 | if transaction.ts >= watermark { 83 | transactions.update_at(transaction, transaction.ts as isize, 1); 84 | } 85 | transactions.advance_to(watermark as isize); 86 | // 1000 transactions per batch 87 | if i % 1000 == 0 { 88 | transactions.flush(); 89 | worker.step(); 90 | } 91 | } 92 | // flush any remaining records 93 | transactions.flush(); 94 | worker.step(); 95 | } 96 | } 97 | }) 98 | .unwrap(); 99 | 100 | for handle in handles.lock().unwrap().drain(..) { 101 | handle.join().unwrap(); 102 | } 103 | } 104 | 105 | fn sum( 106 | collection: differential_dataflow::Collection, 107 | ) -> differential_dataflow::Collection 108 | where 109 | G: timely::dataflow::scopes::Scope, 110 | K: differential_dataflow::ExchangeData 111 | + differential_dataflow::hashable::Hashable 112 | + std::fmt::Debug, 113 | { 114 | collection.reduce(|_k, inputs, output| { 115 | let mut total = 0; 116 | for (num, diff) in inputs { 117 | total += **num * (*diff as i64); 118 | } 119 | output.push((total, 1)); 120 | }) 121 | } 122 | 123 | fn sink_to_file( 124 | worker_index: usize, 125 | handles: &Arc>>>, 126 | name: &str, 127 | collection: &differential_dataflow::Collection, 128 | ) where 129 | G: timely::dataflow::scopes::Scope, 130 | D: differential_dataflow::ExchangeData 131 | + differential_dataflow::hashable::Hashable 132 | + std::fmt::Debug, 133 | { 134 | let mut file = File::create(&format!("./tmp/{}", name)).unwrap(); 135 | let receiver = collection 136 | // only report outputs once the watermark passes 137 | .consolidate() 138 | .inner 139 | // move everything to worker 0 140 | .exchange(|_| 0) 141 | .capture(); 142 | if worker_index == 0 { 143 | let handle = std::thread::spawn(move || { 144 | while let Ok(event) = receiver.recv() { 145 | match event { 146 | Event::Messages(_, rows) => { 147 | for (row, timestamp, diff) in rows { 148 | let update = if diff > 0 { 149 | format!("insert {}x", diff) 150 | } else { 151 | format!("delete {}x", -diff) 152 | }; 153 | write!(&mut file, "{} {:?} at {:?}\n", update, row, timestamp).unwrap(); 154 | } 155 | } 156 | Event::Progress(timestamps) => { 157 | for (timestamp, diff) in timestamps { 158 | if diff > 0 { 159 | write!( 160 | &mut file, 161 | "no more updates with timestamp < {}\n", 162 | timestamp 163 | ) 164 | .unwrap(); 165 | } 166 | } 167 | } 168 | } 169 | } 170 | file.flush().unwrap(); 171 | }); 172 | handles.lock().unwrap().push(handle); 173 | } 174 | } 175 | -------------------------------------------------------------------------------- /flink-datastream/README.md: -------------------------------------------------------------------------------- 1 | __INCOMPLETE__ 2 | 3 | Linux-only. Requires [nix](https://nixos.org/) to fetch dependencies. 4 | 5 | ``` 6 | nix-shell --pure --run './run.sh ../original-transactions.py' 7 | ``` 8 | 9 | You should eventually see something like this: 10 | 11 | ``` 12 | All systems go. Hit ctrl-c when you're ready to shut everything down. 13 | ``` 14 | 15 | Leave that running for now and check out the outputs in `./tmp` in another terminal. -------------------------------------------------------------------------------- /flink-datastream/pom.xml: -------------------------------------------------------------------------------- 1 | 19 | 21 | 4.0.0 22 | 23 | net.scattered_thoughts.streaming_consistency 24 | demo 25 | 1.0.0 26 | jar 27 | 28 | 29 | UTF-8 30 | 1.12.1 31 | 1.8 32 | 2.11 33 | ${java.version} 34 | ${java.version} 35 | 36 | 37 | 38 | 39 | apache.snapshots 40 | Apache Development Snapshot Repository 41 | https://repository.apache.org/content/repositories/snapshots/ 42 | 43 | false 44 | 45 | 46 | true 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | org.apache.flink 55 | flink-java 56 | ${flink.version} 57 | 58 | 59 | 60 | org.apache.flink 61 | flink-clients_${scala.binary.version} 62 | ${flink.version} 63 | 64 | 65 | 66 | org.apache.flink 67 | flink-streaming-java_${scala.binary.version} 68 | ${flink.version} 69 | 70 | 71 | 72 | org.apache.flink 73 | flink-test-utils-junit 74 | ${flink.version} 75 | 76 | 77 | 78 | org.apache.flink 79 | flink-streaming-java_2.11 80 | ${flink.version} 81 | test 82 | test-jar 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | org.apache.maven.plugins 93 | maven-compiler-plugin 94 | 3.1 95 | 96 | ${java.version} 97 | ${java.version} 98 | 99 | 100 | 101 | 102 | 103 | 104 | org.apache.maven.plugins 105 | maven-shade-plugin 106 | 3.0.0 107 | 108 | 109 | 110 | package 111 | 112 | shade 113 | 114 | 115 | 116 | 117 | org.apache.flink:force-shading 118 | com.google.code.findbugs:jsr305 119 | org.slf4j:* 120 | log4j:* 121 | 122 | 123 | 124 | 125 | 127 | *:* 128 | 129 | META-INF/*.SF 130 | META-INF/*.DSA 131 | META-INF/*.RSA 132 | 133 | 134 | 135 | 136 | 137 | net.scattered_thoughts.streaming_consistency.Demo 138 | 139 | 140 | 141 | 142 | 143 | 144 | 145 | 146 | 147 | 148 | 149 | 150 | 151 | org.eclipse.m2e 152 | lifecycle-mapping 153 | 1.0.0 154 | 155 | 156 | 157 | 158 | 159 | org.apache.maven.plugins 160 | maven-shade-plugin 161 | [3.0.0,) 162 | 163 | shade 164 | 165 | 166 | 167 | 168 | 169 | 170 | 171 | 172 | org.apache.maven.plugins 173 | maven-compiler-plugin 174 | [3.1,) 175 | 176 | testCompile 177 | compile 178 | 179 | 180 | 181 | 182 | 183 | 184 | 185 | 186 | 187 | 188 | 189 | org.apache.rat 190 | apache-rat-plugin 191 | 0.13 192 | false 193 | 194 | 195 | verify 196 | 197 | check 198 | 199 | 200 | 201 | 202 | 203 | 204 | **/.*/** 205 | **/*.prefs 206 | **/*.log 207 | 208 | **/README.md 209 | **/CODE_OF_CONDUCT.md 210 | .github/** 211 | 212 | **/*.iml 213 | 214 | **/target/** 215 | **/dependency-reduced-pom.xml 216 | 217 | 218 | 219 | 220 | 221 | 222 | 223 | -------------------------------------------------------------------------------- /flink-datastream/run.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | set -ue 4 | 5 | DATAGEN=$1 6 | 7 | THIS_DIR="$(cd "$(dirname "$0")"; pwd -P)" 8 | 9 | DATA_DIR=$THIS_DIR/tmp 10 | echo "Data will be stored in $DATA_DIR" 11 | rm -rf $DATA_DIR/* 12 | mkdir -p $DATA_DIR/{config,logs} 13 | mkdir -p $DATA_DIR/logs/flink 14 | 15 | FLINK_DIR=$(dirname $(which flink))/../opt/flink/ 16 | 17 | check_port_is_available() { 18 | local name="$1" 19 | local port="$2" 20 | true &>/dev/null $DATA_DIR/transactions/1 36 | 37 | echo "Starting demo" 38 | flink run --detached ./target/demo-1.0.0.jar 39 | 40 | echo "All systems go. Hit ctrl-c when you're ready to shut everything down." 41 | read -r -d '' _ -------------------------------------------------------------------------------- /flink-datastream/shell.nix: -------------------------------------------------------------------------------- 1 | let 2 | 3 | pkgs = import ../pkgs.nix; 4 | 5 | in 6 | 7 | pkgs.mkShell { 8 | buildInputs = [ 9 | pkgs.maven 10 | pkgs.jre8 11 | pkgs.flink 12 | 13 | pkgs.which 14 | pkgs.python38 15 | ]; 16 | } -------------------------------------------------------------------------------- /flink-datastream/src/main/java/net/scattered_thoughts/streaming_consistency/Demo.java: -------------------------------------------------------------------------------- 1 | package net.scattered_thoughts.streaming_consistency; 2 | 3 | import org.apache.flink.api.common.functions.MapFunction; 4 | import org.apache.flink.api.common.state.*; 5 | import org.apache.flink.api.java.functions.KeySelector; 6 | import org.apache.flink.configuration.Configuration; 7 | import org.apache.flink.streaming.api.datastream.DataStream; 8 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 9 | import org.apache.flink.streaming.api.functions.KeyedProcessFunction; 10 | import org.apache.flink.api.java.tuple.*; 11 | import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; 12 | import org.apache.flink.util.Collector; 13 | 14 | import java.io.IOException; 15 | 16 | public class Demo { 17 | 18 | public static void main(String[] args) throws Exception { 19 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 20 | 21 | DataStream transactions = env 22 | .addSource(new TransactionsSource("./tmp/transactions", 100)); 23 | 24 | // timestamp - from_account - amount_total 25 | DataStream> debits = 26 | transactions 27 | .map(new ToDebit()) 28 | .keyBy(new GetAccount()) 29 | .process(new ProcessTransaction()); 30 | 31 | debits.writeAsText("./tmp/debits").setParallelism(1); 32 | 33 | // timestamp - to_account - amount_total 34 | DataStream> credits = 35 | transactions 36 | .map(new ToCredit()) 37 | .keyBy(new GetAccount()) 38 | .process(new ProcessTransaction()); 39 | 40 | credits.writeAsText("./tmp/credits").setParallelism(1); 41 | 42 | // timestamp - account - balance 43 | DataStream> balance = 44 | debits 45 | .keyBy(new GetAccount()) 46 | .union(credits) 47 | .keyBy(new GetAccount()) 48 | .process(new ProcessTransaction()); 49 | 50 | balance.writeAsText("./tmp/balance").setParallelism(1); 51 | 52 | DataStream> total = 53 | balance 54 | .keyBy(new GetOneKey()) 55 | .process(new ProcessTotal()).setParallelism(1); 56 | 57 | total.writeAsText("./tmp/total").setParallelism(1); 58 | 59 | env.execute("Demo"); 60 | } 61 | 62 | static final class ProcessTransaction extends KeyedProcessFunction, Tuple3> { 63 | 64 | private MapState balance; // timestamp -> amount 65 | 66 | @Override 67 | public void open(Configuration parameters) { 68 | MapStateDescriptor taskDescriptor = 69 | new MapStateDescriptor<>("amountState", Long.class, Double.class); 70 | balance = getRuntimeContext().getMapState(taskDescriptor); 71 | } 72 | 73 | @Override 74 | public void processElement(Tuple3 in, Context context, Collector> collector) throws Exception { 75 | 76 | // add update map and set timer 77 | long timestamp = context.timestamp(); 78 | double amount; 79 | if (balance.contains(timestamp)) { 80 | amount = balance.get(timestamp) + in.f2; 81 | } 82 | else { 83 | amount = in.f2; 84 | } 85 | balance.put(timestamp, amount); 86 | context.timerService().registerEventTimeTimer(context.timestamp()); 87 | } 88 | 89 | @Override 90 | public void onTimer(long timestamp, OnTimerContext ctx, Collector> out) throws Exception { 91 | // emit updates 92 | out.collect(Tuple3.of(timestamp, ctx.getCurrentKey(), balance.get(timestamp))); 93 | // clean up state 94 | balance.remove(timestamp); 95 | } 96 | } 97 | 98 | static final class ProcessTotal extends KeyedProcessFunction, Tuple3> { 99 | private ValueState total; 100 | 101 | @Override 102 | public void open(Configuration parameters) { 103 | ValueStateDescriptor taskDescriptor = new ValueStateDescriptor<>("totalState", Double.class); 104 | total = getRuntimeContext().getState(taskDescriptor); 105 | } 106 | 107 | @Override 108 | public void processElement(Tuple3 in, Context context, Collector> out) throws IOException { 109 | 110 | double amount = 0; 111 | if (total.value() != null) { 112 | amount = total.value(); 113 | } 114 | amount += in.f2; 115 | total.update(amount); 116 | context.timerService().registerEventTimeTimer(in.f0); 117 | } 118 | @Override 119 | public void onTimer(long timestamp, OnTimerContext ctx, Collector> out) throws IOException { 120 | // emit updates 121 | out.collect(Tuple3.of(timestamp, ctx.getCurrentKey(), total.value())); 122 | } 123 | } 124 | 125 | // project fields for debits 126 | static final class ToDebit implements MapFunction> { 127 | @Override 128 | public Tuple3 map(JsonNode in) { 129 | return Tuple3.of(in.get("ts").asLong(), in.get("from_account").asLong(), -in.get("amount").asDouble()); 130 | } 131 | } 132 | 133 | // project fields for credits 134 | static final class ToCredit implements MapFunction> { 135 | @Override 136 | public Tuple3 map(JsonNode in) { 137 | return Tuple3.of(in.get("ts").asLong(), in.get("to_account").asLong(), in.get("amount").asDouble()); 138 | } 139 | } 140 | 141 | static final class GetAccount implements KeySelector, Long> { 142 | @Override 143 | public Long getKey(Tuple3 in) { 144 | return in.f1; 145 | } 146 | } 147 | 148 | static final class GetOneKey implements KeySelector, Long> { 149 | @Override 150 | public Long getKey(Tuple3 in) throws Exception { 151 | return 0L; 152 | } 153 | } 154 | } 155 | -------------------------------------------------------------------------------- /flink-datastream/src/main/java/net/scattered_thoughts/streaming_consistency/JsonDeserializationSchema.java: -------------------------------------------------------------------------------- 1 | package net.scattered_thoughts.streaming_consistency; 2 | 3 | import org.apache.flink.api.common.serialization.DeserializationSchema; 4 | import org.apache.flink.api.common.typeinfo.TypeInformation; 5 | import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; 6 | import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; 7 | 8 | import java.io.IOException; 9 | 10 | public class JsonDeserializationSchema implements DeserializationSchema { 11 | 12 | private static final long serialVersionUID = 1L; 13 | 14 | private static final ObjectMapper objectMapper = new ObjectMapper(); 15 | 16 | @Override 17 | public JsonNode deserialize(byte[] message) throws IOException { 18 | return objectMapper.readTree(message); 19 | } 20 | 21 | @Override 22 | public boolean isEndOfStream(JsonNode nextElement) { 23 | return false; 24 | } 25 | 26 | @Override 27 | public TypeInformation getProducedType() { 28 | return TypeInformation.of(JsonNode.class); 29 | } 30 | } -------------------------------------------------------------------------------- /flink-datastream/src/main/java/net/scattered_thoughts/streaming_consistency/TransactionsSource.java: -------------------------------------------------------------------------------- 1 | package net.scattered_thoughts.streaming_consistency; 2 | 3 | import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; 4 | import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; 5 | import org.apache.flink.streaming.api.functions.source.SourceFunction; 6 | import org.apache.flink.streaming.api.watermark.Watermark; 7 | 8 | import java.io.*; 9 | import java.sql.Timestamp; 10 | import java.util.Arrays; 11 | 12 | public class TransactionsSource implements SourceFunction { 13 | 14 | private final String dataFilePath; 15 | private transient BufferedReader reader; 16 | private final int watermarkFrequency; 17 | 18 | public TransactionsSource(String dataFilePath, int watermarkFrequency) { 19 | this.dataFilePath = dataFilePath; 20 | this.watermarkFrequency = watermarkFrequency; 21 | } 22 | 23 | @Override 24 | public void run(SourceContext sourceContext) throws Exception { 25 | 26 | File directory = new File(dataFilePath); 27 | File[] files = directory.listFiles(); 28 | Arrays.sort(files); 29 | for (File file : files) { 30 | FileInputStream fis = new FileInputStream(file); 31 | reader = new BufferedReader(new InputStreamReader(fis, "UTF-8")); 32 | generateStream(sourceContext); 33 | this.reader.close(); 34 | this.reader = null; 35 | } 36 | } 37 | 38 | private void generateStream(SourceContext sourceContext) throws Exception { 39 | 40 | int numEvents = 0; 41 | String line; 42 | JsonNode event; 43 | ObjectMapper mapper = new ObjectMapper(); 44 | 45 | while (reader.ready() && (line = reader.readLine()) != null) { 46 | numEvents++; 47 | event = mapper.readTree(line); 48 | long timestamp = Timestamp.valueOf(event.get("ts").textValue()).getTime(); 49 | sourceContext.collectWithTimestamp(event, timestamp); 50 | 51 | // generate watermark 52 | if (numEvents == watermarkFrequency) { 53 | Watermark nextWatermark = new Watermark(timestamp - 5000); 54 | sourceContext.emitWatermark(nextWatermark); 55 | numEvents = 0; 56 | } 57 | } 58 | } 59 | 60 | @Override 61 | public void cancel() { 62 | try { 63 | if (this.reader != null) { 64 | this.reader.close(); 65 | } 66 | } catch (IOException ioe) { 67 | throw new RuntimeException("Could not cancel SourceFunction", ioe); 68 | } finally { 69 | this.reader = null; 70 | } 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /flink-table/README.md: -------------------------------------------------------------------------------- 1 | Linux-only. Requires [nix](https://nixos.org/) to fetch dependencies. 2 | 3 | ``` 4 | nix-shell --pure --run './run.sh ../original-transactions.py' 5 | ``` 6 | 7 | You should eventually see something like this: 8 | 9 | ``` 10 | All systems go. Hit ctrl-c when you're ready to shut everything down. 11 | ``` 12 | 13 | Leave that running for now and check out the outputs in `./tmp` in another terminal. -------------------------------------------------------------------------------- /flink-table/pom.xml: -------------------------------------------------------------------------------- 1 | 19 | 21 | 4.0.0 22 | 23 | net.scattered_thoughts.streaming_consistency 24 | demo 25 | 1.0.0 26 | jar 27 | 28 | 29 | UTF-8 30 | 1.12.1 31 | 1.8 32 | 2.11 33 | ${java.version} 34 | ${java.version} 35 | 36 | 37 | 38 | 39 | apache.snapshots 40 | Apache Development Snapshot Repository 41 | https://repository.apache.org/content/repositories/snapshots/ 42 | 43 | false 44 | 45 | 46 | true 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | org.apache.flink 55 | flink-table-api-java 56 | ${flink.version} 57 | provided 58 | 59 | 60 | org.apache.flink 61 | flink-table-api-java-bridge_${scala.binary.version} 62 | ${flink.version} 63 | provided 64 | 65 | 66 | 67 | org.apache.flink 68 | flink-streaming-scala_${scala.binary.version} 69 | ${flink.version} 70 | test 71 | 72 | 73 | org.apache.flink 74 | flink-table-planner-blink_${scala.binary.version} 75 | ${flink.version} 76 | test 77 | 78 | 79 | org.apache.flink 80 | flink-clients_${scala.binary.version} 81 | ${flink.version} 82 | test 83 | 84 | 85 | junit 86 | junit 87 | 4.12 88 | test 89 | 90 | 91 | org.hamcrest 92 | hamcrest-all 93 | 1.3 94 | test 95 | 96 | 97 | 98 | 99 | 100 | org.slf4j 101 | slf4j-log4j12 102 | 1.7.7 103 | runtime 104 | 105 | 106 | log4j 107 | log4j 108 | 1.2.17 109 | runtime 110 | 111 | 112 | 113 | org.apache.flink 114 | flink-connector-kafka_2.11 115 | 1.12.0 116 | 117 | 118 | 119 | org.apache.flink 120 | flink-csv 121 | 1.12.0 122 | 123 | 124 | 125 | 126 | 127 | 128 | 129 | 130 | 131 | org.apache.maven.plugins 132 | maven-compiler-plugin 133 | 3.1 134 | 135 | ${java.version} 136 | ${java.version} 137 | 138 | 139 | 140 | 141 | 142 | 143 | org.apache.maven.plugins 144 | maven-shade-plugin 145 | 3.0.0 146 | 147 | 148 | 149 | package 150 | 151 | shade 152 | 153 | 154 | 155 | 156 | org.apache.flink:force-shading 157 | com.google.code.findbugs:jsr305 158 | org.slf4j:* 159 | log4j:* 160 | 161 | 162 | 163 | 164 | 166 | *:* 167 | 168 | META-INF/*.SF 169 | META-INF/*.DSA 170 | META-INF/*.RSA 171 | 172 | 173 | 174 | 175 | 176 | net.scattered_thoughts.streaming_consistency.Demo 177 | 178 | 179 | 180 | 181 | 182 | 183 | 184 | 185 | 186 | 187 | 188 | 189 | 190 | org.eclipse.m2e 191 | lifecycle-mapping 192 | 1.0.0 193 | 194 | 195 | 196 | 197 | 198 | org.apache.maven.plugins 199 | maven-shade-plugin 200 | [3.0.0,) 201 | 202 | shade 203 | 204 | 205 | 206 | 207 | 208 | 209 | 210 | 211 | org.apache.maven.plugins 212 | maven-compiler-plugin 213 | [3.1,) 214 | 215 | testCompile 216 | compile 217 | 218 | 219 | 220 | 221 | 222 | 223 | 224 | 225 | 226 | 227 | 228 | org.apache.rat 229 | apache-rat-plugin 230 | 0.13 231 | false 232 | 233 | 234 | verify 235 | 236 | check 237 | 238 | 239 | 240 | 241 | 242 | 243 | **/.*/** 244 | **/*.prefs 245 | **/*.log 246 | 247 | **/README.md 248 | **/CODE_OF_CONDUCT.md 249 | .github/** 250 | 251 | **/*.iml 252 | 253 | **/target/** 254 | **/dependency-reduced-pom.xml 255 | 256 | 257 | 258 | 259 | 260 | 261 | 262 | -------------------------------------------------------------------------------- /flink-table/run.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | set -ue 4 | 5 | DATAGEN=$1 6 | 7 | THIS_DIR="$(cd "$(dirname "$0")"; pwd -P)" 8 | 9 | DATA_DIR=$THIS_DIR/tmp 10 | echo "Data will be stored in $DATA_DIR" 11 | rm -rf $DATA_DIR/* 12 | mkdir -p $DATA_DIR/{config,logs} 13 | mkdir -p $DATA_DIR/logs/flink 14 | 15 | KAFKA_DIR=$(dirname $(which kafka-server-start.sh))/.. 16 | 17 | FLINK_DIR=$(dirname $(which flink))/../opt/flink/ 18 | 19 | check_port_is_available() { 20 | local name="$1" 21 | local port="$2" 22 | true &>/dev/null /dev/null $DATA_DIR/config/zookeeper.properties <$DATA_DIR/logs/zookeeper & 51 | wait_for_port "zookeeper" 2181 52 | 53 | echo "Starting kafka" 54 | cat $KAFKA_DIR/config/server.properties > $DATA_DIR/config/server.properties 55 | cat >> $DATA_DIR/config/server.properties <$DATA_DIR/logs/kafka & 62 | wait_for_port "kafka" 9092 63 | 64 | echo "Creating topics" 65 | create_topic() { 66 | kafka-topics.sh --create \ 67 | --bootstrap-server localhost:9092 \ 68 | --replication-factor 1 \ 69 | --partitions 1 \ 70 | --config retention.ms=-1 \ 71 | --topic "$1" 72 | } 73 | create_topic transactions 74 | create_topic accepted_transactions 75 | create_topic outer_join_with_time 76 | create_topic outer_join_without_time 77 | create_topic credits 78 | create_topic debits 79 | create_topic balance 80 | create_topic total 81 | create_topic credits2 82 | create_topic debits2 83 | create_topic balance2 84 | create_topic total2 85 | 86 | echo "Starting flink" 87 | $FLINK_DIR/bin/start-cluster.sh 88 | #wait_for_port "flink" 6123 89 | 90 | echo "Compiling" 91 | mvn package 92 | 93 | echo "Starting demo" 94 | flink run --detached ./target/demo-1.0.0.jar 95 | 96 | echo "Feeding inputs" 97 | $DATAGEN | kafka-console-producer.sh \ 98 | --broker-list localhost:9092 \ 99 | --topic transactions \ 100 | --property "key.separator=|" \ 101 | --property "parse.key=true" \ 102 | > /dev/null & 103 | 104 | echo "Watching outputs" 105 | watch_topic() { 106 | kafka-console-consumer.sh \ 107 | --bootstrap-server localhost:9092 \ 108 | --topic "$1" \ 109 | --from-beginning \ 110 | --formatter kafka.tools.DefaultMessageFormatter \ 111 | --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ 112 | --property value.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ 113 | > "./tmp/$1" & 114 | } 115 | watch_topic transactions 116 | watch_topic accepted_transactions 117 | watch_topic outer_join_with_time 118 | watch_topic outer_join_without_time 119 | watch_topic credits 120 | watch_topic debits 121 | watch_topic balance 122 | watch_topic total 123 | watch_topic credits2 124 | watch_topic debits2 125 | watch_topic balance2 126 | watch_topic total2 127 | 128 | echo "All systems go. Hit ctrl-c when you're ready to shut everything down." 129 | read -r -d '' _ -------------------------------------------------------------------------------- /flink-table/shell.nix: -------------------------------------------------------------------------------- 1 | let 2 | 3 | pkgs = import ../pkgs.nix; 4 | 5 | in 6 | 7 | pkgs.mkShell { 8 | buildInputs = [ 9 | pkgs.maven 10 | pkgs.jre8 11 | pkgs.apacheKafka 12 | pkgs.flink 13 | 14 | pkgs.which 15 | pkgs.python38 16 | ]; 17 | } -------------------------------------------------------------------------------- /flink-table/src/main/java/Demo.java: -------------------------------------------------------------------------------- 1 | package net.scattered_thoughts.streaming_consistency; 2 | 3 | import org.apache.flink.streaming.api.datastream.DataStream; 4 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 5 | import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; 6 | import org.apache.flink.table.api.*; 7 | import org.apache.flink.table.expressions.*; 8 | import org.apache.flink.table.functions.ScalarFunction; 9 | import static org.apache.flink.table.api.Expressions.*; 10 | import org.apache.flink.api.java.typeutils.*; 11 | import org.apache.flink.api.java.tuple.*; 12 | import org.apache.flink.types.*; 13 | import java.sql.Timestamp; 14 | import java.util.Properties; 15 | import org.apache.flink.streaming.connectors.kafka.*; 16 | import org.apache.flink.api.common.serialization.*; 17 | 18 | public class Demo { 19 | 20 | public static void main(String[] args) throws Exception { 21 | EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build(); 22 | StreamExecutionEnvironment sEnv = StreamExecutionEnvironment.getExecutionEnvironment(); 23 | StreamTableEnvironment tEnv = StreamTableEnvironment.create(sEnv, settings); 24 | 25 | tEnv.executeSql(String.join("\n", 26 | "CREATE TABLE transactions (", 27 | " id BIGINT,", 28 | " from_account INT,", 29 | " to_account INT,", 30 | " amount DOUBLE,", 31 | " ts TIMESTAMP(3),", 32 | " WATERMARK FOR ts AS ts - INTERVAL '5' SECOND", 33 | ") WITH (", 34 | " 'connector' = 'kafka',", 35 | " 'topic' = 'transactions',", 36 | " 'properties.bootstrap.servers' = 'localhost:9092',", 37 | " 'properties.group.id' = 'demo',", 38 | " 'scan.startup.mode' = 'earliest-offset',", 39 | " 'format' = 'json',", 40 | " 'json.fail-on-missing-field' = 'true',", 41 | " 'json.ignore-parse-errors' = 'false'", 42 | ")" 43 | )); 44 | 45 | tEnv.executeSql(String.join("\n", 46 | "CREATE VIEW accepted_transactions(id) AS", 47 | "SELECT", 48 | " id", 49 | "FROM", 50 | " transactions" 51 | )); 52 | sinkToKafka(tEnv, "accepted_transactions"); 53 | 54 | tEnv.executeSql(String.join("\n", 55 | "CREATE VIEW outer_join_with_time(id, other_id) AS", 56 | "SELECT", 57 | " t1.id, t2.id as other_id", 58 | "FROM", 59 | " transactions as t1", 60 | "LEFT JOIN", 61 | " transactions as t2", 62 | "ON", 63 | " t1.id = t2.id AND t1.ts = t2.ts" 64 | )); 65 | sinkToKafka(tEnv, "outer_join_with_time"); 66 | 67 | tEnv.executeSql(String.join("\n", 68 | "CREATE VIEW outer_join_without_time(id, other_id) AS", 69 | "SELECT", 70 | " t1.id, t2.id as other_id", 71 | "FROM", 72 | " (SELECT id FROM transactions) as t1", 73 | "LEFT JOIN", 74 | " (SELECT id FROM transactions) as t2", 75 | "ON", 76 | " t1.id = t2.id" 77 | )); 78 | sinkToKafka(tEnv, "outer_join_without_time"); 79 | 80 | tEnv.executeSql(String.join("\n", 81 | "CREATE VIEW credits(account, credits) AS", 82 | "SELECT", 83 | " to_account as account, sum(amount) as credits", 84 | "FROM", 85 | " transactions", 86 | "GROUP BY", 87 | " to_account" 88 | )); 89 | sinkToKafka(tEnv, "credits"); 90 | tEnv.executeSql(String.join("\n", 91 | "CREATE VIEW debits(account, debits) AS", 92 | "SELECT", 93 | " from_account as account, sum(amount) as debits", 94 | "FROM", 95 | " transactions", 96 | "GROUP BY", 97 | " from_account" 98 | )); 99 | sinkToKafka(tEnv, "debits"); 100 | tEnv.executeSql(String.join("\n", 101 | "CREATE VIEW balance(account, balance) AS", 102 | "SELECT", 103 | " credits.account, credits - debits as balance", 104 | "FROM", 105 | " credits, debits", 106 | "WHERE", 107 | " credits.account = debits.account" 108 | )); 109 | sinkToKafka(tEnv, "balance"); 110 | tEnv.executeSql(String.join("\n", 111 | "CREATE VIEW total(total) AS", 112 | "SELECT", 113 | " sum(balance)", 114 | "FROM", 115 | " balance" 116 | )); 117 | sinkToKafka(tEnv, "total"); 118 | 119 | tEnv.executeSql(String.join("\n", 120 | "CREATE VIEW credits2(account, credits, ts) AS", 121 | "SELECT", 122 | " to_account as account, sum(amount) as credits, max(ts) as ts", 123 | "FROM", 124 | " transactions", 125 | "GROUP BY", 126 | " to_account" 127 | )); 128 | sinkToKafka(tEnv, "credits2"); 129 | tEnv.executeSql(String.join("\n", 130 | "CREATE VIEW debits2(account, debits, ts) AS", 131 | "SELECT", 132 | " from_account as account, sum(amount) as debits, max(ts) as ts", 133 | "FROM", 134 | " transactions", 135 | "GROUP BY", 136 | " from_account" 137 | )); 138 | sinkToKafka(tEnv, "debits2"); 139 | tEnv.executeSql(String.join("\n", 140 | "CREATE VIEW balance2(account, balance, ts) AS", 141 | "SELECT", 142 | " credits2.account, credits - debits as balance, credits2.ts", 143 | "FROM", 144 | " credits2, debits2", 145 | "WHERE", 146 | " credits2.account = debits2.account AND credits2.ts = debits2.ts" 147 | )); 148 | sinkToKafka(tEnv, "balance2"); 149 | tEnv.executeSql(String.join("\n", 150 | "CREATE VIEW total2(total) AS", 151 | "SELECT", 152 | " sum(balance)", 153 | "FROM", 154 | " balance2" 155 | )); 156 | sinkToKafka(tEnv, "total2"); 157 | 158 | sEnv.execute("Demo"); 159 | } 160 | 161 | public static void sinkToKafka(StreamTableEnvironment tEnv, String name) { 162 | Properties properties = new Properties(); 163 | properties.setProperty("bootstrap.servers", "localhost:9092"); 164 | FlinkKafkaProducer sink = new FlinkKafkaProducer( 165 | name, 166 | new SimpleStringSchema(), 167 | properties 168 | ); 169 | tEnv 170 | .toRetractStream(tEnv.from(name), Row.class) 171 | .map(kv -> 172 | (kv.getField(0) ? "insert" : "delete") 173 | + " " 174 | + kv.getField(1).toString() 175 | ) 176 | .addSink(sink); 177 | } 178 | } 179 | -------------------------------------------------------------------------------- /graph.nix: -------------------------------------------------------------------------------- 1 | let 2 | 3 | pkgs = import ./pkgs.nix; 4 | 5 | in 6 | 7 | pkgs.mkShell { 8 | buildInputs = [ 9 | (pkgs.rWrapper.override{packages = [ 10 | pkgs.rPackages.data_table 11 | pkgs.rPackages.ggplot2 12 | ];}) 13 | ]; 14 | } 15 | 16 | 17 | -------------------------------------------------------------------------------- /graph.r: -------------------------------------------------------------------------------- 1 | #! /usr/bin/env Rscript 2 | 3 | library(data.table) 4 | library(ggplot2) 5 | 6 | raw_totals <- fread('ksqldb-original-total', header=FALSE) 7 | totals <- data.frame(setNames(raw_totals, c('total'))) 8 | totals$input <- seq(1:nrow(totals)) 9 | ggplot(totals, aes(x = input, y = total)) + 10 | geom_jitter(shape = 'x', height = 1, alpha = 1) + 11 | ylab('total') + 12 | xlab('output #') + 13 | theme_bw() 14 | ggsave('ksqldb-original-timeseries.png') 15 | 16 | raw_totals <- fread('ksqldb-simplified-total', header=FALSE) 17 | totals <- data.frame(setNames(raw_totals, c('total'))) 18 | totals$input <- seq(1:nrow(totals)) 19 | ggplot(totals, aes(x = input, y = total)) + 20 | geom_jitter(shape = 'x', height = 1, alpha = 1) + 21 | ylab('total') + 22 | xlab('output #') + 23 | theme_bw() 24 | ggsave('ksqldb-simplified-timeseries.png') 25 | 26 | raw_totals <- fread('flink-original-total', header=FALSE) 27 | totals <- data.frame(setNames(raw_totals, c('total'))) 28 | totals$input <- seq(1:nrow(totals)) 29 | ggplot(totals, aes(x = input, y = total)) + 30 | geom_jitter(shape = '.', height = 1, alpha = 0.01) + 31 | ylab('total') + 32 | xlab('output #') + 33 | theme_bw() 34 | ggsave('flink-original-timeseries.png') 35 | 36 | raw_totals <- fread('flink-simplified-total', header=FALSE) 37 | totals <- data.frame(setNames(raw_totals, c('total'))) 38 | totals$input <- seq(1:nrow(totals)) 39 | ggplot(totals, aes(x = input, y = total)) + 40 | geom_jitter(shape = '.', height = 1, alpha = 0.01) + 41 | ylab('total') + 42 | xlab('output #') + 43 | theme_bw() 44 | ggsave('flink-simplified-timeseries.png') 45 | -------------------------------------------------------------------------------- /graph.sh: -------------------------------------------------------------------------------- 1 | #! /usr/bin/env bash 2 | 3 | cat ksqldb/original-results/total | cut -d':' -f 3 | cut -d '}' -f 1 > ksqldb-original-total 4 | cat ksqldb/simplified-results/total | cut -d':' -f 3 | cut -d '}' -f 1 > ksqldb-simplified-total 5 | grep insert flink-table/original-results/total | cut -d' ' -f 2 > flink-original-total 6 | grep insert flink-table/simplified-results/total | cut -d' ' -f 2 > flink-simplified-total 7 | ./graph.r 8 | mv ./*.png ~/blog/content/writing/internal-consistency-in-streaming-systems/ -------------------------------------------------------------------------------- /kafka-streams/README.md: -------------------------------------------------------------------------------- 1 | Linux-only. Requires [nix](https://nixos.org/) to fetch dependencies. 2 | 3 | ``` 4 | nix-shell --pure --run './run.sh ../original-transactions.py' 5 | ``` 6 | 7 | You should eventually see something like this: 8 | 9 | ``` 10 | All systems go. Hit ctrl-c when you're ready to shut everything down. 11 | ``` 12 | 13 | Leave that running for now and check out the outputs in `./tmp` in another terminal. -------------------------------------------------------------------------------- /kafka-streams/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | net.scattered_thoughts.streaming_consistency 8 | demo 9 | 1.0-SNAPSHOT 10 | 11 | 12 | 13 | 14 | org.apache.kafka 15 | kafka-streams 16 | 2.7.0 17 | 18 | 19 | 20 | 21 | 22 | org.slf4j 23 | slf4j-api 24 | 1.7.25 25 | 26 | 27 | 28 | 29 | org.slf4j 30 | slf4j-log4j12 31 | 1.7.25 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | org.apache.maven.plugins 41 | maven-compiler-plugin 42 | 3.6.1 43 | 44 | 1.8 45 | 1.8 46 | 47 | 48 | 49 | 50 | 51 | org.apache.maven.plugins 52 | maven-assembly-plugin 53 | 3.1.0 54 | 55 | 56 | jar-with-dependencies 57 | 58 | 59 | 60 | true 61 | net.scattered_thoughts.streaming_consistency.Demo 62 | 63 | 64 | 65 | 66 | 67 | assemble-all 68 | package 69 | 70 | single 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | -------------------------------------------------------------------------------- /kafka-streams/run.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | set -ue 4 | 5 | DATAGEN=$1 6 | 7 | THIS_DIR="$(cd "$(dirname "$0")"; pwd -P)" 8 | 9 | DATA_DIR=$THIS_DIR/tmp 10 | echo "Data will be stored in $DATA_DIR" 11 | rm -rf $DATA_DIR/* 12 | mkdir -p $DATA_DIR/{config,logs} 13 | 14 | KAFKA_DIR=$(dirname $(which kafka-server-start.sh))/.. 15 | 16 | check_port_is_available() { 17 | local name="$1" 18 | local port="$2" 19 | true &>/dev/null /dev/null $DATA_DIR/config/zookeeper.properties <$DATA_DIR/logs/zookeeper & 47 | wait_for_port "zookeeper" 2181 48 | 49 | echo "Starting kafka" 50 | cat $KAFKA_DIR/config/server.properties > $DATA_DIR/config/server.properties 51 | cat >> $DATA_DIR/config/server.properties <$DATA_DIR/logs/kafka & 58 | wait_for_port "kafka" 9092 59 | 60 | echo "Creating topics" 61 | create_topic() { 62 | kafka-topics.sh --create \ 63 | --bootstrap-server localhost:9092 \ 64 | --replication-factor 1 \ 65 | --partitions 1 \ 66 | --config retention.ms=-1 \ 67 | --topic "$1" 68 | } 69 | create_topic transactions 70 | create_topic accepted_transactions 71 | create_topic outer_join 72 | create_topic sums 73 | create_topic balance 74 | create_topic total 75 | 76 | echo "Compiling" 77 | mvn package 78 | 79 | echo "Running streams (check $DATA_DIR/logs/streams)" 80 | java -cp target/demo-1.0-SNAPSHOT-jar-with-dependencies.jar net.scattered_thoughts.streaming_consistency.Demo >$DATA_DIR/logs/streams & 81 | 82 | echo "Feeding inputs" 83 | $DATAGEN | kafka-console-producer.sh \ 84 | --broker-list localhost:9092 \ 85 | --topic transactions \ 86 | --property "key.separator=|" \ 87 | --property "parse.key=true" \ 88 | > /dev/null & 89 | 90 | echo "Watching outputs" 91 | watch_topic() { 92 | kafka-console-consumer.sh \ 93 | --bootstrap-server localhost:9092 \ 94 | --topic "$1" \ 95 | --from-beginning \ 96 | --formatter kafka.tools.DefaultMessageFormatter \ 97 | --property print.key=true \ 98 | --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ 99 | --property value.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ 100 | > "./tmp/$1" & 101 | } 102 | watch_topic transactions 103 | watch_topic accepted_transactions 104 | watch_topic outer_join 105 | watch_topic sums 106 | watch_topic balance 107 | watch_topic total 108 | 109 | echo "All systems go. Hit ctrl-c when you're ready to shut everything down." 110 | read -r -d '' _ -------------------------------------------------------------------------------- /kafka-streams/shell.nix: -------------------------------------------------------------------------------- 1 | let 2 | 3 | pkgs = import ../pkgs.nix; 4 | 5 | in 6 | 7 | pkgs.mkShell { 8 | buildInputs = [ 9 | pkgs.maven 10 | pkgs.jre8 11 | pkgs.apacheKafka 12 | 13 | pkgs.which 14 | pkgs.python38 15 | ]; 16 | } -------------------------------------------------------------------------------- /kafka-streams/src/main/java/Demo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package net.scattered_thoughts.streaming_consistency; 18 | 19 | import com.fasterxml.jackson.databind.JsonNode; 20 | import com.fasterxml.jackson.databind.node.JsonNodeFactory; 21 | import com.fasterxml.jackson.databind.node.ObjectNode; 22 | import java.time.Duration; 23 | import org.apache.kafka.clients.consumer.ConsumerConfig; 24 | import org.apache.kafka.common.serialization.Deserializer; 25 | import org.apache.kafka.common.serialization.Serde; 26 | import org.apache.kafka.common.serialization.Serdes; 27 | import org.apache.kafka.common.serialization.Serializer; 28 | import org.apache.kafka.connect.json.JsonDeserializer; 29 | import org.apache.kafka.connect.json.JsonSerializer; 30 | import org.apache.kafka.streams.kstream.Consumed; 31 | import org.apache.kafka.streams.KafkaStreams; 32 | import org.apache.kafka.streams.KeyValue; 33 | import org.apache.kafka.streams.StreamsBuilder; 34 | import org.apache.kafka.streams.StreamsConfig; 35 | import org.apache.kafka.streams.kstream.*; 36 | 37 | import java.util.Properties; 38 | 39 | public class Demo { 40 | 41 | public static void main(final String[] args) throws Exception { 42 | 43 | final Serializer jsonSerializer = new JsonSerializer(); 44 | final Deserializer jsonDeserializer = new JsonDeserializer(); 45 | final Serde jsonSerde = Serdes.serdeFrom(jsonSerializer, jsonDeserializer); 46 | final Consumed consumed = Consumed.with(Serdes.String(), jsonSerde); 47 | 48 | final StreamsBuilder builder = new StreamsBuilder(); 49 | final KTable transactions = builder.table("transactions", consumed); 50 | 51 | transactions 52 | .toStream().to("accepted_transactions", Produced.with(Serdes.String(), jsonSerde)); 53 | 54 | //transactions 55 | //.leftJoin(transactions, (t1, t2) -> 56 | //t2 == null ? "null" : t2.get("id").textValue()) 57 | //.toStream().to("outer_join", Produced.with(Serdes.String(), Serdes.String())); 58 | 59 | //transactions 60 | //.groupBy((k,v) -> KeyValue.pair("yolo", v), Grouped.with(Serdes.String(), jsonSerde)) 61 | //.aggregate( 62 | //() -> 0L, 63 | //(k, v, sum) -> sum + v.get("amount").longValue(), 64 | //(k, v, sum) -> sum - v.get("amount").longValue(), 65 | //Materialized.with(Serdes.String(), Serdes.Long())) 66 | //.mapValues(v -> v.toString()) 67 | //.toStream().to("sums", Produced.with(Serdes.String(), Serdes.String())); 68 | 69 | //KTable credits = transactions 70 | //.groupBy((k,v) -> KeyValue.pair(v.get("to_account").longValue(), v)) 71 | //.aggregate( 72 | //() -> 0L, 73 | //(k, v, sum) -> sum + v.get("amount").longValue(), 74 | //(k, v, sum) -> sum - v.get("amount").longValue()); 75 | //KTable debits = transactions 76 | //.groupBy((k,v) -> KeyValue.pair(v.get("from_account").longValue(), v)) 77 | //.aggregate( 78 | //() -> 0L, 79 | //(k, v, sum) -> sum + v.get("amount").longValue(), 80 | //(k, v, sum) -> sum - v.get("amount").longValue()); 81 | //KTable balance = credits 82 | //.join(debits, (c, d) -> c - d); 83 | //balance 84 | //.toStream() 85 | //.map((k,v) -> KeyValue.pair(k.toString(), v.toString())) 86 | //.to("balance", Produced.with(Serdes.String(), Serdes.String())); 87 | //balance 88 | //.groupBy((k,v) -> KeyValue.pair("yolo", v), Grouped.with(Serdes.String(), Serdes.Long())) 89 | //.aggregate( 90 | //() -> 0L, 91 | //(k, v, sum) -> sum + v, 92 | //(k, v, sum) -> sum - v) 93 | //.mapValues(v -> v.toString()) 94 | //.toStream().to("total", Produced.with(Serdes.String(), Serdes.String())); 95 | 96 | System.out.println(builder.build().describe()); 97 | 98 | final Properties props = new Properties(); 99 | props.put(StreamsConfig.APPLICATION_ID_CONFIG, "demo"); 100 | props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); 101 | props.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class); 102 | props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); 103 | props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); 104 | // neither of these seem to help, but feel free to uncomment them 105 | // props.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500); 106 | // props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10 * 1000); 107 | final KafkaStreams streams = new KafkaStreams(builder.build(), props); 108 | 109 | // Runtime.getRuntime().addShutdownHook(new Thread(streams::close)); 110 | streams.start(); 111 | } 112 | } 113 | -------------------------------------------------------------------------------- /kafka-streams/src/main/java/JsonTimestampExtractor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package net.scattered_thoughts.streaming_consistency; 18 | 19 | import com.fasterxml.jackson.databind.JsonNode; 20 | import org.apache.kafka.clients.consumer.ConsumerRecord; 21 | import org.apache.kafka.streams.processor.TimestampExtractor; 22 | 23 | /** 24 | * A timestamp extractor implementation that tries to extract event time from 25 | * the "timestamp" field in the Json formatted message. 26 | */ 27 | public class JsonTimestampExtractor implements TimestampExtractor { 28 | 29 | @Override 30 | public long extract(final ConsumerRecord record, final long partitionTime) { 31 | if (record.value() instanceof JsonNode) { 32 | return ((JsonNode) record.value()).get("ts").longValue(); 33 | } 34 | 35 | throw new IllegalArgumentException("JsonTimestampExtractor cannot recognize the record value " + record.value()); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /ksqldb/README.md: -------------------------------------------------------------------------------- 1 | Linux-only. Requires docker-compose and python3.8. 2 | 3 | ``` 4 | ./run.sh ../original-transactions.py 5 | ``` 6 | 7 | You should eventually see something like this: 8 | 9 | ``` 10 | All systems go. Hit ctrl-c when you're ready to shut everything down. 11 | ``` 12 | 13 | Leave that running for now and check out the outputs in `./tmp` in another terminal. -------------------------------------------------------------------------------- /ksqldb/docker-compose.yml: -------------------------------------------------------------------------------- 1 | --- 2 | version: '2' 3 | 4 | services: 5 | zookeeper: 6 | image: confluentinc/cp-zookeeper:6.1.0 7 | hostname: zookeeper 8 | container_name: zookeeper 9 | ports: 10 | - "2181:2181" 11 | environment: 12 | ZOOKEEPER_CLIENT_PORT: 2181 13 | ZOOKEEPER_TICK_TIME: 2000 14 | 15 | broker: 16 | image: confluentinc/cp-kafka:6.1.0 17 | hostname: broker 18 | container_name: broker 19 | depends_on: 20 | - zookeeper 21 | ports: 22 | - "29092:29092" 23 | environment: 24 | KAFKA_BROKER_ID: 1 25 | KAFKA_ZOOKEEPER_CONNECT: 'zookeeper:2181' 26 | KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT 27 | KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://broker:9092,PLAINTEXT_HOST://localhost:29092 28 | KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 29 | KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0 30 | KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 31 | KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 32 | 33 | ksqldb-server: 34 | image: confluentinc/ksqldb-server:0.15.0 35 | hostname: ksqldb-server 36 | container_name: ksqldb-server 37 | depends_on: 38 | - broker 39 | ports: 40 | - "8088:8088" 41 | environment: 42 | KSQL_LISTENERS: http://0.0.0.0:8088 43 | KSQL_BOOTSTRAP_SERVERS: broker:9092 44 | KSQL_KSQL_LOGGING_PROCESSING_STREAM_AUTO_CREATE: "true" 45 | KSQL_KSQL_LOGGING_PROCESSING_TOPIC_AUTO_CREATE: "true" 46 | 47 | ksqldb-cli: 48 | image: confluentinc/ksqldb-cli:0.15.0 49 | container_name: ksqldb-cli 50 | depends_on: 51 | - broker 52 | - ksqldb-server 53 | entrypoint: /bin/sh 54 | tty: true 55 | -------------------------------------------------------------------------------- /ksqldb/run.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | set -ue 4 | 5 | DATAGEN=$1 6 | 7 | # cleanup processes on exit 8 | cleanup() { 9 | echo "Cleaning up" 10 | docker-compose down 11 | echo "Done" 12 | } 13 | trap cleanup EXIT 14 | 15 | THIS_DIR="$(cd "$(dirname "$0")"; pwd -P)" 16 | 17 | DATA_DIR=$THIS_DIR/tmp 18 | echo "Data will be stored in $DATA_DIR" 19 | rm -rf $DATA_DIR/* 20 | mkdir -p $DATA_DIR/{config,logs} 21 | 22 | check_port_is_available() { 23 | local name="$1" 24 | local port="$2" 25 | true &>/dev/null /dev/null $DATA_DIR/logs/ksqldb 2>&1 & 47 | wait_for_port zookeeper 2181 48 | wait_for_port kafka 29092 49 | wait_for_port ksqldb 8088 50 | 51 | echo "Creating topics" 52 | create_topic() { 53 | COMPOSE_INTERACTIVE_NO_CLI=1 docker-compose exec -T broker kafka-topics --create \ 54 | --bootstrap-server localhost:9092 \ 55 | --replication-factor 1 \ 56 | --partitions 1 \ 57 | --config retention.ms=-1 \ 58 | --topic "$1" 59 | } 60 | create_topic transactions 61 | create_topic accepted_transactions 62 | create_topic outer_join 63 | create_topic sums 64 | create_topic balance 65 | create_topic total 66 | 67 | echo "Waiting until ksqldb is ready for commands" 68 | while ! $(docker-compose exec -T ksqldb-cli ksql http://ksqldb-server:8088 -e 'show topics;' 2> /dev/null | grep -q "default_ksql_processing_log") 69 | do 70 | echo -n "." 71 | done 72 | echo 73 | 74 | echo "Installing views" 75 | # cant do all the views in one command - produces "Failed to guarantee existence of topic accepted_transactions" 76 | docker-compose exec -T ksqldb-cli ksql http://ksqldb-server:8088 -e "$(cat views1.sql)" > /dev/null 2> /dev/null 77 | docker-compose exec -T ksqldb-cli ksql http://ksqldb-server:8088 -e "$(cat views2.sql)" > /dev/null 2> /dev/null 78 | 79 | # TODO this exits, probably same problem as above 80 | echo "Watching outputs" 81 | watch_topic() { 82 | COMPOSE_INTERACTIVE_NO_CLI=1 docker-compose exec -T broker kafka-console-consumer \ 83 | --bootstrap-server localhost:29092 \ 84 | --topic "$1" \ 85 | --from-beginning \ 86 | --formatter kafka.tools.DefaultMessageFormatter \ 87 | --property print.timestamp=true \ 88 | --property print.key=true \ 89 | --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ 90 | --property value.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ 91 | > "./tmp/$1" & 92 | } 93 | watch_topic transactions 94 | watch_topic accepted_transactions 95 | watch_topic outer_join 96 | watch_topic credits 97 | watch_topic debits 98 | watch_topic balance 99 | watch_topic total 100 | 101 | echo "Feeding inputs" 102 | $DATAGEN | docker-compose exec -T broker kafka-console-producer \ 103 | --broker-list localhost:29092 \ 104 | --topic transactions \ 105 | --property "key.separator=|" \ 106 | --property "parse.key=true" \ 107 | > /dev/null & 108 | 109 | echo "All systems go. Hit ctrl-c when you're ready to shut everything down." 110 | read -r -d '' _ -------------------------------------------------------------------------------- /ksqldb/shell.nix: -------------------------------------------------------------------------------- 1 | let 2 | 3 | pkgs = import ../pkgs.nix; 4 | 5 | in 6 | 7 | pkgs.mkShell { 8 | buildInputs = [ 9 | pkgs.which 10 | pkgs.python38 11 | ]; 12 | } -------------------------------------------------------------------------------- /ksqldb/views1.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE transactions ( 2 | id VARCHAR PRIMARY KEY, 3 | from_account INT, 4 | to_account INT, 5 | amount DOUBLE, 6 | ts VARCHAR 7 | ) WITH ( 8 | kafka_topic='transactions', 9 | value_format='json', 10 | partitions=1, 11 | timestamp='ts', 12 | timestamp_format='yyyy-MM-dd HH:mm:ss.SSS' 13 | ); -------------------------------------------------------------------------------- /ksqldb/views2.sql: -------------------------------------------------------------------------------- 1 | CREATE TABLE accepted_transactions WITH ( 2 | kafka_topic='accepted_transactions', 3 | value_format='json') AS 4 | SELECT 5 | * 6 | FROM 7 | transactions 8 | EMIT CHANGES; 9 | 10 | CREATE TABLE outer_join WITH ( 11 | kafka_topic='outer_join', 12 | value_format='json') AS 13 | SELECT 14 | t1.id, 15 | t2.id as other_id 16 | FROM 17 | transactions as t1 18 | LEFT JOIN 19 | accepted_transactions as t2 20 | ON 21 | t1.id = t2.id 22 | EMIT CHANGES; 23 | 24 | CREATE TABLE credits WITH ( 25 | kafka_topic='credits', 26 | value_format='json') AS 27 | SELECT 28 | to_account AS account, 29 | sum(amount) AS credits 30 | FROM 31 | transactions 32 | GROUP BY 33 | to_account 34 | EMIT CHANGES; 35 | 36 | CREATE TABLE debits WITH ( 37 | kafka_topic='debits', 38 | value_format='json') AS 39 | SELECT 40 | from_account AS account, 41 | sum(amount) AS debits 42 | FROM 43 | transactions 44 | GROUP BY 45 | from_account 46 | EMIT CHANGES; 47 | 48 | CREATE TABLE balance WITH ( 49 | kafka_topic='balance', 50 | value_format='json') AS 51 | SELECT 52 | credits.account AS account, 53 | credits - debits AS balance 54 | FROM 55 | credits 56 | INNER JOIN 57 | debits 58 | ON 59 | credits.account = debits.account 60 | EMIT CHANGES; 61 | 62 | CREATE TABLE total WITH ( 63 | kafka_topic='total', 64 | value_format='json') AS 65 | SELECT 66 | 'foo', 67 | sum(balance) 68 | FROM 69 | balance 70 | GROUP BY 71 | 'foo' 72 | EMIT CHANGES; -------------------------------------------------------------------------------- /materialize/README.md: -------------------------------------------------------------------------------- 1 | Linux-only. Requires [nix](https://nixos.org/) to fetch dependencies. 2 | 3 | ``` 4 | nix-shell --pure --run './run.sh ../original-transactions.py' 5 | ``` 6 | 7 | You should eventually see something like this: 8 | 9 | ``` 10 | All systems go. Hit ctrl-c when you're ready to shut everything down. 11 | ``` 12 | 13 | Leave that running for now and check out the outputs in `./tmp` in another terminal. -------------------------------------------------------------------------------- /materialize/run.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | set -ue 4 | 5 | DATAGEN=$1 6 | 7 | THIS_DIR="$(cd "$(dirname "$0")"; pwd -P)" 8 | 9 | DATA_DIR=$THIS_DIR/tmp 10 | echo "Data will be stored in $DATA_DIR" 11 | rm -rf $DATA_DIR/* 12 | mkdir -p $DATA_DIR/{config,logs} 13 | 14 | check_port_is_available() { 15 | local name="$1" 16 | local port="$2" 17 | true &>/dev/null /dev/null $DATA_DIR/logs/materialized 2>&1 & 42 | wait_for_port "materialized" 6875 43 | 44 | echo "Feeding inputs" 45 | $DATAGEN | cut -d'|' -f2 >> $DATA_DIR/transactions 46 | 47 | echo "Creating views" 48 | touch "$DATA_DIR/transactions" 49 | # have to make source separately to get the absolute path 50 | psql postgres://materialize@localhost:6875/materialize -c "CREATE SOURCE transactions_source FROM FILE '$DATA_DIR/transactions' FORMAT BYTES;" 51 | psql postgres://materialize@localhost:6875/materialize -f ./views.sql 52 | 53 | echo "Watching outputs" 54 | watch_view() { 55 | unbuffer psql postgres://materialize@localhost:6875/materialize -c "COPY (TAIL $1 AS OF 0) TO STDOUT" > $DATA_DIR/$1 & 56 | } 57 | watch_view accepted_transactions 58 | watch_view outer_join 59 | watch_view credits 60 | watch_view debits 61 | watch_view balance 62 | watch_view total 63 | 64 | echo "All systems go. Hit ctrl-c when you're ready to shut everything down." 65 | read -r -d '' _ -------------------------------------------------------------------------------- /materialize/shell.nix: -------------------------------------------------------------------------------- 1 | let 2 | 3 | pkgs = import ../pkgs.nix; 4 | 5 | in 6 | 7 | pkgs.mkShell { 8 | buildInputs = [ 9 | pkgs.materialized 10 | 11 | pkgs.python38 12 | pkgs.python38Packages.psycopg2 13 | pkgs.postgresql # provides psql 14 | pkgs.expect # provides unbuffer 15 | ]; 16 | } -------------------------------------------------------------------------------- /materialize/views.sql: -------------------------------------------------------------------------------- 1 | CREATE MATERIALIZED VIEW transactions_without_time AS 2 | SELECT 3 | CAST(data->'id' AS INT) as id, 4 | CAST(data->'from_account' AS INT) as from_account, 5 | CAST(data->'to_account' AS INT) as to_account, 6 | CAST(data->'amount' AS DOUBLE) as amount, 7 | CAST(CAST(data->'ts' AS TEXT) AS TIMESTAMP) as ts 8 | FROM ( 9 | SELECT CAST(convert_from(data, 'utf8') AS jsonb) AS data 10 | FROM transactions_source 11 | ); 12 | 13 | CREATE MATERIALIZED VIEW transactions AS 14 | SELECT 15 | * 16 | FROM 17 | transactions_without_time 18 | WHERE 19 | (CAST(EXTRACT(EPOCH FROM ts) AS NUMERIC) * 10000) <= mz_logical_timestamp(); 20 | 21 | CREATE MATERIALIZED VIEW accepted_transactions AS 22 | SELECT 23 | id 24 | FROM 25 | transactions; 26 | 27 | CREATE MATERIALIZED VIEW outer_join AS 28 | SELECT 29 | t1.id AS id, 30 | t2.id AS other_id 31 | FROM 32 | (SELECT id FROM transactions) AS t1 33 | LEFT JOIN 34 | (SELECT id FROM transactions) AS t2 35 | ON 36 | t1.id = t2.id; 37 | 38 | CREATE MATERIALIZED VIEW credits AS 39 | SELECT 40 | to_account AS account, 41 | sum(amount) AS credits 42 | FROM 43 | transactions 44 | GROUP BY 45 | to_account; 46 | 47 | CREATE MATERIALIZED VIEW debits AS 48 | SELECT 49 | from_account AS account, 50 | sum(amount) AS debits 51 | FROM 52 | transactions 53 | GROUP BY 54 | from_account; 55 | 56 | CREATE MATERIALIZED VIEW balance AS 57 | SELECT 58 | credits.account AS account, 59 | credits - debits AS balance 60 | FROM 61 | credits, 62 | debits 63 | WHERE 64 | credits.account = debits.account; 65 | 66 | CREATE MATERIALIZED VIEW total AS 67 | SELECT 68 | sum(balance) 69 | FROM 70 | balance; -------------------------------------------------------------------------------- /original-transactions.py: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env python3.8 2 | 3 | import random 4 | import json 5 | 6 | random.seed(42) 7 | max_id = 10000000 8 | transactions = [] 9 | for id in range(0,max_id): 10 | second = ((60 * id) // max_id) 11 | delay = random.uniform(0, 10) 12 | row = json.dumps({ 13 | 'id': id, 14 | 'from_account': random.randint(0,9), 15 | 'to_account': random.randint(0,9), 16 | 'amount': 1, 17 | 'ts': f'2021-01-01 00:00:{second:02d}.000', 18 | }) 19 | transactions.append((second + delay, id, row)) 20 | transactions.sort() 21 | for (_, id, row) in transactions: 22 | print(f'{id}|{row}') -------------------------------------------------------------------------------- /pkgs.nix: -------------------------------------------------------------------------------- 1 | # pin package index to nixos-20.09 2 | # TODO will this fail with patch releases? 3 | import (builtins.fetchTarball { 4 | name = "nixos-20.09"; 5 | url = "https://github.com/NixOS/nixpkgs/archive/20.09.tar.gz"; 6 | sha256 = "1wg61h4gndm3vcprdcg7rc4s1v3jkm5xd7lw8r2f67w502y94gcy"; 7 | }) { 8 | overlays = [(self: super: { 9 | 10 | # override kafka to v2.7.0 11 | apacheKafka = super.apacheKafka.overrideAttrs (oldAttrs: rec { 12 | version = "2.7.0"; 13 | src = self.fetchurl { 14 | url = "mirror://apache/kafka/${version}/kafka_2.13-${version}.tgz"; 15 | sha256 = "143zrghrq40lrwacwyzqkyzg4asax7kxg9cgnkn2z83n6rv4pn0x"; 16 | }; 17 | # install zookeeper scripts 18 | installPhase = '' 19 | mkdir -p $out/bin 20 | cp bin/zookeeper* $out/bin 21 | '' + oldAttrs.installPhase; 22 | }); 23 | 24 | # override flink to v1.12.2 25 | flink = super.flink.overrideAttrs (oldAttrs: rec { 26 | flinkVersion = "1.12.2"; 27 | name = "flink-${flinkVersion}"; 28 | src = self.fetchurl { 29 | url = "mirror://apache/flink/${name}/${name}-bin-scala_2.11.tgz"; 30 | sha256 = "17c2v185m3q58dcwvpyzgaymf7767m8dap0xb318ijphb9sapvpk"; 31 | }; 32 | installPhase = '' 33 | rm bin/*.bat || true 34 | 35 | mkdir -p $out/bin $out/opt/flink 36 | mv * $out/opt/flink/ 37 | makeWrapper $out/opt/flink/bin/flink $out/bin/flink \ 38 | --prefix PATH : ${super.jre}/bin 39 | 40 | cat <> $out/opt/flink/conf/flink-conf.yaml 41 | env.java.home: ${super.jre} 42 | io.tmp.dirs: ./tmp 43 | env.log.dir: ./tmp/logs/flink/ 44 | jobmanager.memory.process.size: 2000m 45 | taskmanager.memory.process.size: 32000m 46 | EOF 47 | ''; 48 | }); 49 | 50 | # override spark to v3.1.1 51 | spark = super.spark.overrideAttrs (oldAttrs: rec { 52 | version = "3.1.1"; 53 | buildInputs = oldAttrs.buildInputs ++ [ super.procps super.nettools ]; 54 | src = super.fetchzip { 55 | url = "mirror://apache/spark/spark-${version}/spark-${version}-bin-without-hadoop.tgz"; 56 | sha256 = "1l6jwf79wc1wbr8h1fgggd9fmn173bmj6lkjmn0m6a4xd7nk6pv7"; 57 | }; 58 | }); 59 | 60 | # download materialized v0.7.1 61 | materialized = super.stdenv.mkDerivation { 62 | name = "materialized"; 63 | src = fetchTarball { 64 | url = "https://downloads.mtrlz.dev/materialized-v0.7.1-x86_64-unknown-linux-gnu.tar.gz"; 65 | sha256 = "1lhx0fvjsqih3wxnm0vkdnynm2ly43cr04qm917k83bj1gv71lnp"; 66 | }; 67 | nativeBuildInputs = [ 68 | super.autoPatchelfHook 69 | ]; 70 | dontConfigure = true; 71 | dontBuild = true; 72 | installPhase = '' 73 | mkdir -p $out 74 | mv ./* $out 75 | ''; 76 | }; 77 | 78 | })]; 79 | } -------------------------------------------------------------------------------- /simplified-transactions.py: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env python3.8 2 | 3 | import random 4 | import json 5 | 6 | random.seed(42) 7 | max_id = 10000000 8 | for id in range(0,max_id): 9 | second = ((60 * id) // max_id) 10 | from_account = id % 10 11 | to_account = (from_account + 1) % 10 12 | row = json.dumps({ 13 | 'id': id, 14 | 'from_account': from_account, 15 | 'to_account': to_account, 16 | 'amount': 1, 17 | 'ts': f'2021-01-01 00:00:{second:02d}.000', 18 | }) 19 | print(f'{id}|{row}') -------------------------------------------------------------------------------- /spark-structured-streaming/README.md: -------------------------------------------------------------------------------- 1 | __INCOMPLETE__ 2 | 3 | Linux-only. Requires [nix](https://nixos.org/) to fetch dependencies. 4 | 5 | ``` 6 | NIXPKGS_ALLOW_INSECURE=1 nix-shell --pure --run './run.sh ../original-transactions.py' 7 | ``` -------------------------------------------------------------------------------- /spark-structured-streaming/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | edu.berkeley 3 | demo 4 | 4.0.0 5 | Demo 6 | jar 7 | 1.0 8 | 9 | 10 | org.apache.spark 11 | spark-sql_2.12 12 | 3.1.1 13 | provided 14 | 15 | 16 | org.apache.spark 17 | spark-sql-kafka-0-10_2.12 18 | 3.1.1 19 | provided 20 | 21 | 22 | 23 | 24 | 25 | 26 | org.apache.maven.plugins 27 | maven-compiler-plugin 28 | 3.6.1 29 | 30 | 1.8 31 | 1.8 32 | 33 | 34 | 35 | 36 | 37 | -------------------------------------------------------------------------------- /spark-structured-streaming/run.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | set -ue 4 | 5 | DATAGEN=$1 6 | 7 | THIS_DIR="$(cd "$(dirname "$0")"; pwd -P)" 8 | 9 | DATA_DIR=$THIS_DIR/tmp 10 | echo "Data will be stored in $DATA_DIR" 11 | rm -rf $DATA_DIR/* 12 | mkdir -p $DATA_DIR/{config,logs} 13 | 14 | KAFKA_DIR=$(dirname $(which kafka-server-start.sh))/.. 15 | 16 | SPARK_DIR=$(dirname $(which spark-submit))/../lib/spark-2.4.4-bin-without-hadoop/ 17 | 18 | check_port_is_available() { 19 | local name="$1" 20 | local port="$2" 21 | true &>/dev/null /dev/null $DATA_DIR/config/zookeeper.properties <$DATA_DIR/logs/zookeeper & 50 | wait_for_port "zookeeper" 2181 51 | 52 | echo "Starting kafka" 53 | cat $KAFKA_DIR/config/server.properties > $DATA_DIR/config/server.properties 54 | cat >> $DATA_DIR/config/server.properties <$DATA_DIR/logs/kafka & 61 | wait_for_port "kafka" 9092 62 | 63 | echo "Creating topics" 64 | create_topic() { 65 | kafka-topics.sh --create \ 66 | --bootstrap-server localhost:9092 \ 67 | --replication-factor 1 \ 68 | --partitions 1 \ 69 | --config retention.ms=-1 \ 70 | --topic "$1" 71 | } 72 | create_topic transactions 73 | create_topic accepted_transactions 74 | create_topic outer_join_with_time 75 | create_topic outer_join_without_time 76 | create_topic credits 77 | create_topic debits 78 | create_topic balance 79 | create_topic total 80 | create_topic total2 81 | 82 | # # This doesn't work - it ignores the properties file and tries to write to $SPARK_DIR/logs anyway 83 | # 84 | # cat $SPARK_DIR/conf/spark-defaults.conf.template > $DATA_DIR/config/spark.conf 85 | # cat >> $DATA_DIR/config/spark.conf <> $DATA_DIR/spark/conf/spark-env.sh < $DATA_DIR/logs/demo 2>&1 & 115 | 116 | echo "Feeding inputs" 117 | $DATAGEN | kafka-console-producer.sh \ 118 | --broker-list localhost:9092 \ 119 | --topic transactions \ 120 | --property "key.separator=|" \ 121 | --property "parse.key=true" \ 122 | > /dev/null & 123 | 124 | echo "Watching outputs" 125 | watch_topic() { 126 | kafka-console-consumer.sh \ 127 | --bootstrap-server localhost:9092 \ 128 | --topic "$1" \ 129 | --from-beginning \ 130 | --formatter kafka.tools.DefaultMessageFormatter \ 131 | --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ 132 | --property value.deserializer=org.apache.kafka.common.serialization.StringDeserializer \ 133 | > "./tmp/$1" & 134 | } 135 | watch_topic transactions 136 | watch_topic accepted_transactions 137 | watch_topic outer_join_with_time 138 | watch_topic outer_join_without_time 139 | watch_topic credits 140 | watch_topic debits 141 | watch_topic balance 142 | watch_topic total 143 | watch_topic total2 144 | 145 | echo "All systems go. Hit ctrl-c when you're ready to shut everything down." 146 | read -r -d '' _ -------------------------------------------------------------------------------- /spark-structured-streaming/shell.nix: -------------------------------------------------------------------------------- 1 | let 2 | 3 | pkgs = import ../pkgs.nix; 4 | 5 | in 6 | 7 | pkgs.mkShell { 8 | buildInputs = [ 9 | pkgs.maven 10 | pkgs.jre8 11 | pkgs.apacheKafka 12 | pkgs.spark 13 | 14 | pkgs.python38 15 | pkgs.which 16 | ]; 17 | } -------------------------------------------------------------------------------- /spark-structured-streaming/src/main/java/Demo.java: -------------------------------------------------------------------------------- 1 | import org.apache.spark.sql.SparkSession; 2 | import org.apache.spark.sql.Dataset; 3 | 4 | public class Demo { 5 | public static void main(String[] args) throws Exception { 6 | 7 | SparkSession spark = SparkSession.builder().appName("Simple Application").getOrCreate(); 8 | 9 | Dataset df = spark 10 | .readStream() 11 | .format("kafka") 12 | .option("kafka.bootstrap.servers", "localhost:9092") 13 | .option("subscribe", "transactions") 14 | .option("startingOffsets", "earliest") 15 | .load(); 16 | 17 | df 18 | .writeStream() 19 | .format("kafka") 20 | .option("kafka.bootstrap.servers", "localhost:9092") 21 | .option("topic", "accepted_transactions") 22 | .option("checkpointLocation", "./tmp/") 23 | .start(); 24 | 25 | spark.stop(); 26 | } 27 | } --------------------------------------------------------------------------------