|
1 | | -fn main() { |
2 | | - // [Leader] |
3 | | - // If it already exists, delete the "financial_transactions" topic |
4 | | - // Create a new topic named "financial_transactions" with 10 partitions. |
| 1 | +use std::collections::HashMap; |
| 2 | +use std::sync::Arc; |
5 | 3 |
|
6 | | - // Produce 1 million transactions into the financial_transactions topic, spread evenly |
7 | | - // across the 10 partitions |
| 4 | +use clap::{ArgAction, Parser}; |
| 5 | +use hydro_deploy::gcp::GcpNetwork; |
| 6 | +use hydro_deploy::{AwsNetwork, Deployment, Host}; |
| 7 | +use hydro_lang::deploy::TrybuildHost; |
| 8 | +use hydro_lang::live_collections::stream::{ExactlyOnce, TotalOrder}; |
| 9 | +use hydro_lang::location::Location; |
| 10 | +use hydro_lang::nondet::nondet; |
| 11 | +use hydro_lang::viz::config::GraphConfig; |
| 12 | +use hydro_test::kafka::{dest_kafka, kafka_consumer, kafka_producer, setup_topic}; |
| 13 | +use stageleft::q; |
8 | 14 |
|
9 | | - // [Consumers] |
10 | | - // Once the cluster of consumers receives the go-ahead from the leader stream, it may then |
11 | | - // begin to consume the financial transactions and compute the balance of each bank account |
| 15 | +type HostCreator = Box<dyn Fn(&mut Deployment) -> Arc<dyn Host>>; |
| 16 | + |
| 17 | +const TOPIC: &str = "financial_transactions"; |
| 18 | +const NUM_PARTITIONS: i32 = 10; |
| 19 | +const NUM_TRANSACTIONS: usize = 1_000_000; |
| 20 | +const NUM_CONSUMERS: usize = 3; |
| 21 | + |
| 22 | +// cargo run -p hydro_test --example kafka --features kafka -- --brokers 'localhost:9092' |
| 23 | +#[derive(Parser, Debug)] |
| 24 | +#[command(group( |
| 25 | + clap::ArgGroup::new("cloud") |
| 26 | + .args(&["gcp", "aws"]) |
| 27 | + .multiple(false) |
| 28 | +))] |
| 29 | +struct Args { |
| 30 | + #[clap(flatten)] |
| 31 | + graph: GraphConfig, |
| 32 | + |
| 33 | + /// Use GCP for deployment (provide project name) |
| 34 | + #[arg(long)] |
| 35 | + gcp: Option<String>, |
| 36 | + |
| 37 | + /// Use AWS, make sure credentials are set up |
| 38 | + #[arg(long, action = ArgAction::SetTrue)] |
| 39 | + aws: bool, |
| 40 | + |
| 41 | + /// Kafka bootstrap servers |
| 42 | + #[arg(long, default_value = "localhost:9092")] |
| 43 | + brokers: String, |
| 44 | +} |
| 45 | + |
| 46 | +enum Leader {} |
| 47 | +enum Consumer {} |
| 48 | + |
| 49 | +#[tokio::main] |
| 50 | +async fn main() -> Result<(), Box<dyn std::error::Error>> { |
| 51 | + let args = Args::parse(); |
| 52 | + let mut deployment = Deployment::new(); |
| 53 | + |
| 54 | + let create_host: HostCreator = if let Some(project) = &args.gcp { |
| 55 | + let network = GcpNetwork::new(project, None); |
| 56 | + let project = project.clone(); |
| 57 | + |
| 58 | + Box::new(move |deployment| -> Arc<dyn Host> { |
| 59 | + deployment |
| 60 | + .GcpComputeEngineHost() |
| 61 | + .project(&project) |
| 62 | + .machine_type("e2-micro") |
| 63 | + .image("debian-cloud/debian-11") |
| 64 | + .region("us-west1-a") |
| 65 | + .network(network.clone()) |
| 66 | + .add() |
| 67 | + }) |
| 68 | + } else if args.aws { |
| 69 | + let region = "us-east-1"; |
| 70 | + let network = AwsNetwork::new(region, None); |
| 71 | + |
| 72 | + Box::new(move |deployment| -> Arc<dyn Host> { |
| 73 | + deployment |
| 74 | + .AwsEc2Host() |
| 75 | + .region(region) |
| 76 | + .instance_type("t3.micro") |
| 77 | + .ami("ami-0e95a5e2743ec9ec9") // Amazon Linux 2 |
| 78 | + .network(network.clone()) |
| 79 | + .add() |
| 80 | + }) |
| 81 | + } else { |
| 82 | + let localhost = deployment.Localhost(); |
| 83 | + Box::new(move |_| -> Arc<dyn Host> { localhost.clone() }) |
| 84 | + }; |
| 85 | + |
| 86 | + // [Leader] Setup topic and produce transactions |
| 87 | + setup_topic(&args.brokers, TOPIC, NUM_PARTITIONS).await; |
| 88 | + println!("Topic '{}' created with {} partitions", TOPIC, NUM_PARTITIONS); |
| 89 | + |
| 90 | + let mut flow = hydro_lang::compile::builder::FlowBuilder::new(); |
| 91 | + let leader = flow.process::<Leader>(); |
| 92 | + let consumers = flow.cluster::<Consumer>(); |
| 93 | + |
| 94 | + // Leader: produce 1M transactions spread across 10 partitions. |
| 95 | + // Each transaction is (account_id, amount) serialized as key=account, value=amount. |
| 96 | + { |
| 97 | + let producer = kafka_producer(&leader, &args.brokers); |
| 98 | + let transactions = leader.source_iter(q!({ |
| 99 | + (0..NUM_TRANSACTIONS).map(|i| { |
| 100 | + let account = format!("account_{}", i % 100); |
| 101 | + let amount = format!("{}", (i % 201) as i64 - 100); // range [-100, 100] |
| 102 | + (account, amount) |
| 103 | + }) |
| 104 | + })); |
| 105 | + dest_kafka(producer, transactions, TOPIC); |
| 106 | + } |
| 107 | + |
| 108 | + // Consumers: read from topic and compute per-account balances. |
| 109 | + { |
| 110 | + let messages = kafka_consumer(&consumers, &args.brokers, "kafka_example_consumers", TOPIC) |
| 111 | + .assume_ordering::<TotalOrder>( |
| 112 | + nondet!(/** Safe: side effect is only printing final balances. */), |
| 113 | + ) |
| 114 | + .assume_retries::<ExactlyOnce>( |
| 115 | + nondet!(/** Safe: side effect is only printing final balances. */), |
| 116 | + ) |
| 117 | + .filter_map(q!(|msg| { |
| 118 | + let key = rdkafka::Message::key(&msg) |
| 119 | + .map(|k| String::from_utf8_lossy(k).to_string())?; |
| 120 | + let value = rdkafka::Message::payload(&msg) |
| 121 | + .map(|v| String::from_utf8_lossy(v).to_string())?; |
| 122 | + let amount: i64 = value.parse().ok()?; |
| 123 | + Some((key, amount)) |
| 124 | + })) |
| 125 | + .fold( |
| 126 | + q!(|| HashMap::<String, i64>::new()), |
| 127 | + q!(|balances, (account, amount)| { |
| 128 | + *balances.entry(account).or_insert(0) += amount; |
| 129 | + }), |
| 130 | + ); |
| 131 | + |
| 132 | + messages.into_stream().for_each(q!(|balances: HashMap<String, i64>| { |
| 133 | + println!("Final balances ({} accounts):", balances.len()); |
| 134 | + let mut sorted: Vec<_> = balances.into_iter().collect(); |
| 135 | + sorted.sort_by(|a, b| a.0.cmp(&b.0)); |
| 136 | + for (account, balance) in sorted.iter().take(10) { |
| 137 | + println!(" {}: {}", account, balance); |
| 138 | + } |
| 139 | + if sorted.len() > 10 { |
| 140 | + println!(" ... and {} more accounts", sorted.len() - 10); |
| 141 | + } |
| 142 | + })); |
| 143 | + } |
| 144 | + |
| 145 | + // Extract the IR BEFORE the builder is consumed by deployment methods |
| 146 | + let built = flow.finalize(); |
| 147 | + |
| 148 | + // Generate graph visualizations based on command line arguments |
| 149 | + built.generate_graph_with_config(&args.graph, None)?; |
| 150 | + |
| 151 | + // If we're just generating a graph file, exit early |
| 152 | + if args.graph.should_exit_after_graph_generation() { |
| 153 | + return Ok(()); |
| 154 | + } |
| 155 | + |
| 156 | + // Now use the built flow for deployment with optimization |
| 157 | + let _nodes = built |
| 158 | + .with_default_optimize() |
| 159 | + .with_process( |
| 160 | + &leader, |
| 161 | + TrybuildHost::new(create_host(&mut deployment)) |
| 162 | + .features(vec!["kafka".to_owned()]), |
| 163 | + ) |
| 164 | + .with_cluster( |
| 165 | + &consumers, |
| 166 | + (0..NUM_CONSUMERS).map(|_| { |
| 167 | + TrybuildHost::new(create_host(&mut deployment)) |
| 168 | + .features(vec!["kafka".to_owned()]) |
| 169 | + }), |
| 170 | + ) |
| 171 | + .deploy(&mut deployment); |
| 172 | + |
| 173 | + deployment.deploy().await.unwrap(); |
| 174 | + deployment.start().await.unwrap(); |
| 175 | + |
| 176 | + println!("Running Kafka financial transactions example..."); |
| 177 | + println!("Press Ctrl+C to stop."); |
| 178 | + |
| 179 | + tokio::signal::ctrl_c().await.unwrap(); |
| 180 | + Ok(()) |
12 | 181 | } |
0 commit comments