@@ -16,7 +16,7 @@ type HostCreator = Box<dyn Fn(&mut Deployment) -> Arc<dyn Host>>;
1616
1717const TOPIC_PREFIX : & str = "financial_transactions" ;
1818const NUM_PARTITIONS : i32 = 10 ;
19- const NUM_TRANSACTIONS : usize = 100_000 ;
19+ const NUM_TRANSACTIONS : usize = 10_000 ;
2020const NUM_CONSUMERS : usize = 3 ;
2121
2222// cargo run -p hydro_test --example kafka --features kafka -- --brokers 'localhost:9092'
@@ -45,6 +45,14 @@ struct Args {
4545 /// Kafka security protocol (plaintext or SSL for MSK)
4646 #[ arg( long, default_value = "SSL" ) ]
4747 security_protocol : String ,
48+
49+ /// Run mode: "produce" (produce only, prints topic name), "consume" (consume only, requires --topic), or "both" (default)
50+ #[ arg( long, default_value = "both" ) ]
51+ mode : String ,
52+
53+ /// Topic name for consume-only mode (use the topic printed by a produce run)
54+ #[ arg( long) ]
55+ topic : Option < String > ,
4856}
4957
5058enum Leader { }
@@ -95,16 +103,22 @@ async fn main() -> Result<(), Box<dyn std::error::Error>> {
95103 Box :: new ( move |_| -> Arc < dyn Host > { localhost. clone ( ) } )
96104 } ;
97105
98- // Use a unique topic name per run to avoid stale messages from previous runs.
99- let topic = format ! ( "{}_{}" , TOPIC_PREFIX , std:: process:: id( ) ) ;
106+ let produce = args. mode == "produce" || args. mode == "both" ;
107+ let consume = args. mode == "consume" || args. mode == "both" ;
108+
109+ // For consume-only, require --topic; otherwise generate a unique one.
110+ let topic = if let Some ( t) = & args. topic {
111+ t. clone ( )
112+ } else {
113+ format ! ( "{}_{}" , TOPIC_PREFIX , std:: process:: id( ) )
114+ } ;
100115
101116 let mut flow = hydro_lang:: compile:: builder:: FlowBuilder :: new ( ) ;
102117 let leader = flow. process :: < Leader > ( ) ;
103118 let consumers = flow. cluster :: < Consumer > ( ) ;
104119
105120 // Leader: produce transactions spread across partitions.
106- // Each transaction is (account_id, amount) serialized as key=account, value=amount.
107- {
121+ if produce {
108122 let producer = kafka_producer (
109123 & leader,
110124 & args. brokers ,
@@ -119,21 +133,29 @@ async fn main() -> Result<(), Box<dyn std::error::Error>> {
119133 ( account, amount)
120134 } )
121135 } ) ) ;
122- dest_kafka ( producer, transactions, & topic) ;
123- // Sentinel so the runner knows when producing is done.
124- leader
125- . source_iter ( q ! ( std:: iter:: once( "PRODUCE_DONE" . to_string( ) ) ) )
126- . for_each ( q ! ( |msg| println!( "{}" , msg) ) ) ;
136+ let sent = dest_kafka ( producer, transactions, & topic) ;
137+ sent. for_each ( q ! ( {
138+ let count = std:: cell:: Cell :: new( 0usize ) ;
139+ move |producer| {
140+ let c = count. get( ) + 1 ;
141+ count. set( c) ;
142+ if c >= NUM_TRANSACTIONS {
143+ rdkafka:: producer:: Producer :: flush( & * producer, std:: time:: Duration :: from_secs( 30 ) )
144+ . expect( "Failed to flush producer" ) ;
145+ println!( "PRODUCE_DONE {}" , c) ;
146+ }
147+ }
148+ } ) ) ;
127149 }
128150
129- // Consumers: read from topic and compute per-account balances .
130- {
131- let messages = kafka_consumer ( & consumers, & args. brokers , "kafka_example_consumers" , & topic, & args. security_protocol )
151+ // Consumers: read from topic and print each message .
152+ if consume {
153+ let _messages = kafka_consumer ( & consumers, & args. brokers , "kafka_example_consumers" , & topic, & args. security_protocol )
132154 . assume_ordering :: < TotalOrder > (
133- nondet ! ( /** Safe: side effect is only printing final balances . */ ) ,
155+ nondet ! ( /** Safe: side effect is only printing. */ ) ,
134156 )
135157 . assume_retries :: < ExactlyOnce > (
136- nondet ! ( /** Safe: side effect is only printing final balances . */ ) ,
158+ nondet ! ( /** Safe: side effect is only printing. */ ) ,
137159 )
138160 . filter_map ( q ! ( |msg| {
139161 let key = rdkafka:: Message :: key( & msg)
@@ -156,90 +178,97 @@ async fn main() -> Result<(), Box<dyn std::error::Error>> {
156178 return Ok ( ( ) ) ;
157179 }
158180
159- // Now use the built flow for deployment with optimization
160- let nodes = built
161- . with_default_optimize ( )
162- . with_process (
163- & leader,
181+ // Deploy
182+ let mut hosts_builder = built. with_default_optimize ( ) ;
183+ hosts_builder = hosts_builder. with_process (
184+ & leader,
185+ TrybuildHost :: new ( create_host ( & mut deployment) )
186+ . features ( vec ! [ "kafka" . to_owned( ) ] ) ,
187+ ) ;
188+ hosts_builder = hosts_builder. with_cluster (
189+ & consumers,
190+ ( 0 ..NUM_CONSUMERS ) . map ( |_| {
164191 TrybuildHost :: new ( create_host ( & mut deployment) )
165- . features ( vec ! [ "kafka" . to_owned( ) ] ) ,
166- )
167- . with_cluster (
168- & consumers,
169- ( 0 ..NUM_CONSUMERS ) . map ( |_| {
170- TrybuildHost :: new ( create_host ( & mut deployment) )
171- . features ( vec ! [ "kafka" . to_owned( ) ] )
172- } ) ,
173- )
174- . deploy ( & mut deployment) ;
192+ . features ( vec ! [ "kafka" . to_owned( ) ] )
193+ } ) ,
194+ ) ;
195+ let nodes = hosts_builder. deploy ( & mut deployment) ;
175196
176197 deployment. deploy ( ) . await . unwrap ( ) ;
177198 deployment. start ( ) . await . unwrap ( ) ;
178199
179- // Subscribe to stdout from all deployed nodes and count messages.
200+ println ! ( "Running Kafka example (mode={}, topic={topic}, {NUM_TRANSACTIONS} messages)..." , args. mode) ;
201+
180202 let start = std:: time:: Instant :: now ( ) ;
181203 let total = std:: sync:: Arc :: new ( std:: sync:: atomic:: AtomicUsize :: new ( 0 ) ) ;
182204 let ( done_tx, mut done_rx) = tokio:: sync:: mpsc:: channel :: < ( ) > ( 1 ) ;
183205 let ( produce_done_tx, produce_done_rx) = tokio:: sync:: oneshot:: channel :: < ( ) > ( ) ;
184206 {
185207 use hydro_lang:: deploy:: DeployCrateWrapper ;
186208
187- let leader_node = nodes. get_process ( & leader) ;
188- let mut leader_out = leader_node. stdout ( ) ;
189- let produce_done_tx = std:: sync:: Mutex :: new ( Some ( produce_done_tx) ) ;
190- tokio:: spawn ( async move {
191- while let Some ( line) = leader_out. recv ( ) . await {
192- if line. trim ( ) == "PRODUCE_DONE" {
193- if let Some ( tx) = produce_done_tx. lock ( ) . unwrap ( ) . take ( ) {
194- let _ = tx. send ( ( ) ) ;
195- }
196- } else {
197- println ! ( "[Leader] {line}" ) ;
198- }
199- }
200- } ) ;
201-
202- for ( i, member) in nodes. get_cluster ( & consumers) . members ( ) . into_iter ( ) . enumerate ( ) {
203- let mut member_out = member. stdout ( ) ;
204- let total = total. clone ( ) ;
205- let done_tx = done_tx. clone ( ) ;
209+ if produce {
210+ let leader_node = nodes. get_process ( & leader) ;
211+ let mut leader_out = leader_node. stdout ( ) ;
212+ let produce_done_tx = std:: sync:: Mutex :: new ( Some ( produce_done_tx) ) ;
206213 tokio:: spawn ( async move {
207- while let Some ( _line) = member_out. recv ( ) . await {
208- let t = total. fetch_add ( 1 , std:: sync:: atomic:: Ordering :: Relaxed ) + 1 ;
209- if t % 10_000 == 0 {
210- println ! ( "[Consumer {i}] ... {t} total messages consumed so far" ) ;
211- }
212- if t >= NUM_TRANSACTIONS {
213- let _ = done_tx. send ( ( ) ) . await ;
214- return ;
214+ while let Some ( line) = leader_out. recv ( ) . await {
215+ if line. starts_with ( "PRODUCE_DONE" ) {
216+ if let Some ( tx) = produce_done_tx. lock ( ) . unwrap ( ) . take ( ) {
217+ let _ = tx. send ( ( ) ) ;
218+ }
219+ } else {
220+ println ! ( "[Leader] {line}" ) ;
215221 }
216222 }
217223 } ) ;
218224 }
225+
226+ if consume {
227+ for ( i, member) in nodes. get_cluster ( & consumers) . members ( ) . into_iter ( ) . enumerate ( ) {
228+ let mut member_out = member. stdout ( ) ;
229+ let total = total. clone ( ) ;
230+ let done_tx = done_tx. clone ( ) ;
231+ tokio:: spawn ( async move {
232+ while let Some ( _line) = member_out. recv ( ) . await {
233+ let t = total. fetch_add ( 1 , std:: sync:: atomic:: Ordering :: Relaxed ) + 1 ;
234+ if t % 1_000 == 0 {
235+ println ! ( "[Consumer {i}] ... {t} total messages consumed so far" ) ;
236+ }
237+ if t >= NUM_TRANSACTIONS {
238+ let _ = done_tx. send ( ( ) ) . await ;
239+ return ;
240+ }
241+ }
242+ } ) ;
243+ }
244+ }
219245 }
220246 drop ( done_tx) ;
221247
222- println ! ( "Running Kafka financial transactions example ({NUM_TRANSACTIONS} messages)..." ) ;
248+ if produce {
249+ let _ = produce_done_rx. await ;
250+ let produce_elapsed = start. elapsed ( ) ;
251+ println ! (
252+ "Produce: {NUM_TRANSACTIONS} messages in {:.2?} ({:.0} msgs/sec)" ,
253+ produce_elapsed,
254+ NUM_TRANSACTIONS as f64 / produce_elapsed. as_secs_f64( )
255+ ) ;
256+ if !consume {
257+ println ! ( "Topic: {topic}" ) ;
258+ println ! ( "Run consume with: --mode consume --topic {topic}" ) ;
259+ return Ok ( ( ) ) ;
260+ }
261+ }
223262
224- let _ = produce_done_rx. await ;
225- let produce_elapsed = start. elapsed ( ) ;
226- println ! (
227- "Produce: {NUM_TRANSACTIONS} messages in {:.2?} ({:.0} msgs/sec)" ,
228- produce_elapsed,
229- NUM_TRANSACTIONS as f64 / produce_elapsed. as_secs_f64( )
230- ) ;
263+ if consume {
264+ done_rx. recv ( ) . await ;
265+ let elapsed = start. elapsed ( ) ;
266+ println ! (
267+ "Consume: {NUM_TRANSACTIONS} messages in {:.2?} ({:.0} msgs/sec)" ,
268+ elapsed,
269+ NUM_TRANSACTIONS as f64 / elapsed. as_secs_f64( )
270+ ) ;
271+ }
231272
232- done_rx. recv ( ) . await ;
233- let total_elapsed = start. elapsed ( ) ;
234- let consume_elapsed = total_elapsed - produce_elapsed;
235- println ! (
236- "Consume: {NUM_TRANSACTIONS} messages in {:.2?} ({:.0} msgs/sec)" ,
237- consume_elapsed,
238- NUM_TRANSACTIONS as f64 / consume_elapsed. as_secs_f64( )
239- ) ;
240- println ! (
241- "Total: {:.2?}" ,
242- total_elapsed,
243- ) ;
244273 Ok ( ( ) )
245274}
0 commit comments