crabka-client-streams 0.3.2

KIP-1071 Kafka Streams rebalance-protocol client for Apache Kafka in Rust
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
use std::collections::HashMap;
use std::sync::Arc;
use tokio::sync::Mutex as TokioMutex;

use crate::error::StreamsClientError;
use crate::membership::StreamsAssignment;
use crate::runtime::eos::{ProcessingGuarantee, StreamsGroupMeta, TransactionalProducer};
use crate::runtime::io::{BeginTxnGate, OffsetStore, RecordFetcher, RecordProducer};
use crate::runtime::iq::{IqRequest, answer_iq};
use crate::runtime::task::{StreamTask, TaskRole};
use crate::topology::BuiltTopology;

pub(crate) struct StreamThread {
    tasks: HashMap<(String, i32), StreamTask>,
    /// Shared fetcher reference kept for restore (replaying changelog on task creation).
    fetcher: Arc<dyn RecordFetcher>,
    /// Storage backend to use when instantiating new task graphs.
    backend: crate::store::backend::StoreBackend,
    /// Application ID passed to `instantiate` for changelog-name derivation and
    /// backend path construction.
    application_id: String,
    /// The shared, fully-replicated global stores for this app. Built + bootstrapped
    /// once from the topology's global store factories (on the first assignment that
    /// has work), then lent by `Arc` clone into every task's graph so a
    /// stream-globaltable join reads the same global state. Empty (default) when the
    /// topology declares no `GlobalKTable`.
    globals: crate::runtime::global::GlobalStateManager,
    /// Whether `globals` has been built + bootstrapped yet. Guards the one-time
    /// lazy build at the top of `apply_assignment`.
    globals_ready: bool,
    /// Per-`(global topic, partition)` next-offset, seeded by the bootstrap read and
    /// advanced by each `poll_all` live-update pass. Empty when the topology declares
    /// no `GlobalKTable`.
    global_offsets: std::collections::HashMap<(String, i32), i64>,
    /// Wall-clock source driving wall-clock punctuation between polls. Defaults to
    /// `SystemClock`; tests inject a `ManualClock` via `with_clock` for determinism.
    clock: Arc<dyn crate::runtime::clock::Clock>,
    /// Delivery guarantee for this thread. Set by `apply_assignment`; defaults to
    /// at-least-once until the first assignment arrives.
    guarantee: ProcessingGuarantee,
    /// The EOS-v2 transactional producer (the same object the tasks `send`
    /// through, viewed as a `TransactionalProducer`). `None` under at-least-once.
    txn: Option<Arc<dyn TransactionalProducer>>,
    /// Whether `init_transactions` has run (one-time, on the first EOS assignment).
    initialized: bool,
    /// Whether a transaction is currently open (`begin_transaction` called, not yet
    /// committed/aborted). Drives the begin-on-first-poll / commit barrier.
    in_txn: bool,
}

impl StreamThread {
    pub fn new(
        fetcher: Arc<dyn RecordFetcher>,
        backend: crate::store::backend::StoreBackend,
        application_id: String,
    ) -> Self {
        Self {
            tasks: HashMap::new(),
            fetcher,
            backend,
            application_id,
            globals: crate::runtime::global::GlobalStateManager::default(),
            globals_ready: false,
            global_offsets: std::collections::HashMap::new(),
            clock: Arc::new(crate::runtime::clock::SystemClock),
            guarantee: ProcessingGuarantee::AtLeastOnce,
            txn: None,
            initialized: false,
            in_txn: false,
        }
    }

    /// Test-only: swap in a deterministic clock (e.g. `ManualClock`) so wall-clock
    /// punctuation can be driven without real time passing.
    #[cfg(test)]
    #[must_use]
    pub fn with_clock(mut self, clock: Arc<dyn crate::runtime::clock::Clock>) -> Self {
        self.clock = clock;
        self
    }

    #[cfg(test)]
    #[must_use]
    pub fn task_count(&self) -> usize {
        self.tasks.len()
    }

    /// Test-only: typed read from a named KV store on the task at `key`.
    #[cfg(test)]
    async fn task_store_get_i64(
        &mut self,
        task: &(String, i32),
        store: &str,
        k: &String,
    ) -> Option<i64> {
        self.tasks.get_mut(task)?.store_get_i64(store, k).await
    }

    /// Test-only: whether the task at `key` has pending (uncommitted) offsets.
    #[cfg(test)]
    fn task_has_pending(&self, task: &(String, i32)) -> bool {
        self.tasks
            .get(task)
            .is_some_and(|t| !t.pending_offsets().is_empty())
    }

    /// Reconcile tasks to `assignment`. Reconciles active, standby, and warmup tasks.
    ///
    /// `guarantee` + `txn` configure the EOS commit path: under
    /// [`ProcessingGuarantee::ExactlyOnceV2`] the same producer object is also
    /// passed as `txn` (a [`TransactionalProducer`] view), and the first EOS
    /// assignment runs `init_transactions` once (fencing any zombie).
    #[allow(clippy::too_many_lines)]
    pub async fn apply_assignment(
        &mut self,
        assignment: &StreamsAssignment,
        topology: &BuiltTopology,
        producer: &Arc<dyn RecordProducer>,
        store: &Arc<dyn OffsetStore>,
        guarantee: ProcessingGuarantee,
        txn: Option<Arc<dyn TransactionalProducer>>,
    ) -> Result<(), StreamsClientError> {
        self.guarantee = guarantee;
        self.txn = txn;
        // One-time `init_transactions` on the first EOS assignment (bumps the
        // producer epoch, fencing a zombie of the same transactional id).
        if self.guarantee == ProcessingGuarantee::ExactlyOnceV2 && !self.initialized {
            let txn = self
                .txn
                .as_ref()
                .expect("EOS requires a transactional producer");
            txn.init_transactions().await?;
            self.initialized = true;
        }

        // Lazily build + bootstrap the shared global store(s) exactly once, before
        // any task processes. Kafka blocks task start until the global store is
        // ready, so we drain every partition of each global source topic here.
        if !self.globals_ready {
            let factories = topology.global_store_factories();
            if !factories.is_empty() {
                self.globals = crate::runtime::global::GlobalStateManager::build(
                    factories,
                    topology.global_store_topics(),
                    &self.backend,
                    &self.application_id,
                )
                .await;
                // Bootstrap from all partitions BEFORE any task processes
                // (bootstrap-before-process is the required behavior); the returned
                // resume offsets seed the live-update poll in `poll_all`.
                self.global_offsets = self.globals.bootstrap(&*self.fetcher).await?;
            }
            self.globals_ready = true;
        }

        // Desired (subtopology_id, partition) -> (TaskRole, &TaskAssignment).
        let mut desired: HashMap<(String, i32), (TaskRole, &crate::membership::TaskAssignment)> =
            HashMap::new();
        for ta in &assignment.active {
            for &p in &ta.partitions {
                desired.insert((ta.subtopology_id.clone(), p), (TaskRole::Active, ta));
            }
        }
        for ta in &assignment.standby {
            for &p in &ta.partitions {
                desired.insert((ta.subtopology_id.clone(), p), (TaskRole::Standby, ta));
            }
        }
        for ta in &assignment.warmup {
            for &p in &ta.partitions {
                desired.insert((ta.subtopology_id.clone(), p), (TaskRole::Warmup, ta));
            }
        }

        // Drop removed: close processors, commit, then drop.
        let to_remove: Vec<(String, i32)> = self
            .tasks
            .keys()
            .filter(|k| !desired.contains_key(*k))
            .cloned()
            .collect();
        for k in to_remove {
            if let Some(mut t) = self.tasks.remove(&k).filter(|t| t.role == TaskRole::Active) {
                t.close_processors().await;
                t.commit().await?;
            }
        }

        // Transition existing tasks whose role has changed.
        for (key, &(desired_role, _ta)) in &desired {
            if let Some(task) = self.tasks.get_mut(key).filter(|t| t.role != desired_role) {
                let old_role = task.role;
                if desired_role == TaskRole::Active {
                    // Promotion: catch up remaining restore, seek source partitions, and init processors.
                    task.restore(&*self.fetcher).await?;
                    task.seek_to_start().await?;
                    task.init().await?;
                } else if old_role == TaskRole::Active {
                    // Demotion: close processors and commit offsets.
                    task.close_processors().await;
                    task.commit().await?;
                }
                task.role = desired_role;
            }
        }

        // Add new.
        for (key, &(desired_role, ta)) in &desired {
            if self.tasks.contains_key(key) {
                continue;
            }
            let mut graph = topology
                .instantiate(&self.backend, &self.application_id)
                .await
                .map_err(|e| StreamsClientError::Runtime(e.to_string()))?;
            // Lend the shared, bootstrapped global manager to this task's graph so a
            // stream-globaltable join reads the same fully-replicated global state.
            graph.globals = self.globals.clone();
            let sources: Vec<crate::membership::TopicPartition> = ta
                .source_topic_partitions
                .iter()
                .filter(|tp| tp.partition == key.1)
                .cloned()
                .collect();
            let mut task = StreamTask::new(
                key.0.clone(),
                graph,
                sources,
                Arc::clone(producer),
                Arc::clone(store),
                desired_role,
                self.guarantee,
            );
            if desired_role == TaskRole::Active {
                // Seek positions to committed offsets (or earliest) BEFORE restore so
                // that normal processing knows where to start after restore completes.
                task.seek_to_start().await?;
                // Restore state stores from changelog, then initialise processors.
                task.restore(&*self.fetcher).await?;
                task.init().await?;
            }
            self.tasks.insert(key.clone(), task);
        }
        Ok(())
    }

    /// Abort the in-flight txn and roll back every task to the last committed
    /// state (rewind source offsets, wipe stores, re-restore from the committed
    /// changelog). Called on any error during an EOS process/commit cycle.
    //
    // All EOS-cycle errors are treated as retryable abort+rollback here; the
    // fenced-fatal distinction (a `ProducerFenced` must shut the thread down, not
    // retry) is a follow-up.
    async fn abort_and_rollback(&mut self) -> Result<(), StreamsClientError> {
        if let Some(txn) = self.txn.as_ref() {
            let _ = txn.abort_transaction().await;
        }
        self.in_txn = false;
        let fetcher = Arc::clone(&self.fetcher);
        for task in self.tasks.values_mut() {
            task.rollback(&*fetcher).await?;
        }
        Ok(())
    }

    pub async fn poll_all(
        &mut self,
        fetcher: &dyn RecordFetcher,
        tracker: &Arc<TokioMutex<crate::membership::TaskOffsetTracker>>,
    ) -> Result<(), StreamsClientError> {
        // Apply any new global-topic records to the shared global store(s) before
        // processing, so stream-globaltable joins see live updates (Kafka keeps the
        // global store current after the initial bootstrap). No-op without globals.
        if !self.global_offsets.is_empty() {
            self.globals
                .poll_once(fetcher, &mut self.global_offsets)
                .await?;
        }
        match self.guarantee {
            ProcessingGuarantee::AtLeastOnce => {
                for task in self.tasks.values_mut() {
                    task.process_once(fetcher, None).await?;
                }
            }
            ProcessingGuarantee::ExactlyOnceV2 => {
                // EOS: the transaction is opened lazily, on the FIRST produced
                // record of the interval (via the begin-gate handed to each
                // task). An interval that fetches no records opens no
                // transaction, so `commit_all` is a no-op (no empty-txn churn on
                // an idle app). Idempotent across polls within a commit interval
                // — `in_txn` stays set until `commit_all`. Any error mid-begin or
                // mid-process aborts the txn and rolls every task back to the last
                // commit; the cycle is then re-begun on the next poll (so
                // `poll_all` returns Ok).
                let res = self.eos_begin_and_process(fetcher).await;
                if res.is_err() {
                    self.abort_and_rollback().await?;
                    return Ok(());
                }
            }
        }
        // Wall-clock punctuation tick (independent of the delivery guarantee): read
        // the clock once, then fire every task's due WALL_CLOCK_TIME punctuators.
        // Forwarded records go through each task's producer — under EOS they join
        // the interval's open transaction (committed by the next `commit_all`).
        let now = self.clock.now_ms();
        for task in self.tasks.values_mut() {
            if task.role == TaskRole::Active {
                task.punctuate_wall_clock(now).await?;
            }
        }

        // Update task offsets in the shared tracker.
        let mut task_offsets = std::collections::HashMap::new();
        let mut task_end_offsets = std::collections::HashMap::new();
        for (key, task) in &mut self.tasks {
            let (curr, end) = task.compute_changelog_offsets().await?;
            task_offsets.insert(key.clone(), curr);
            task_end_offsets.insert(key.clone(), end);
        }
        {
            let mut lock = tracker.lock().await;
            lock.task_offsets = task_offsets;
            lock.task_end_offsets = task_end_offsets;
        }

        Ok(())
    }

    /// EOS begin-on-first-record + per-task process, captured so `poll_all` can
    /// turn any `Err` into an abort + rollback.
    ///
    /// The transaction is NOT begun up front. Instead each task is handed an
    /// [`EosBeginGate`] that begins the transaction lazily, right before the
    /// task's first produced record of the interval. If no task fetches any
    /// records the gate is never tripped, so the interval opens no transaction
    /// (and `commit_all` becomes a no-op) — matching the JVM's "gate on records
    /// processed since last commit" behaviour.
    async fn eos_begin_and_process(
        &mut self,
        fetcher: &dyn RecordFetcher,
    ) -> Result<(), StreamsClientError> {
        let txn = Arc::clone(self.txn.as_ref().expect("EOS txn producer"));
        let mut gate = EosBeginGate {
            txn,
            begun: self.in_txn,
        };
        let res: Result<(), StreamsClientError> = async {
            for task in self.tasks.values_mut() {
                if task.role == TaskRole::Active {
                    task.process_once(fetcher, Some(&mut gate)).await?;
                } else {
                    task.restore_step(fetcher).await?;
                }
            }
            Ok(())
        }
        .await;
        // Reflect any lazily-opened transaction back onto the thread so
        // `commit_all` / `abort_and_rollback` see it, even on the error path.
        self.in_txn = gate.begun;
        res
    }

    /// EOS commit barrier: fold every task's pending source offsets into a single
    /// `send_offsets_to_transaction`, then `commit_transaction`. Captured so
    /// `commit_all` can turn any `Err` into an abort + rollback. Does NOT clear
    /// pending (the caller does that only on success).
    async fn eos_send_offsets_and_commit(
        &mut self,
        meta: Option<&StreamsGroupMeta>,
    ) -> Result<(), StreamsClientError> {
        let txn = self.txn.as_ref().expect("EOS txn producer");
        let mut offsets = Vec::new();
        for task in self.tasks.values() {
            offsets.extend(task.pending_offsets());
        }
        let meta = meta.expect("EOS commit requires group metadata");
        txn.send_offsets_to_transaction(&offsets, meta).await?;
        txn.commit_transaction().await?;
        Ok(())
    }

    /// Commit advanced offsets.
    ///
    /// At-least-once: per-task `flush` + offset commit (`meta` ignored).
    /// Exactly-once-v2: fold every task's pending source offsets into a single
    /// `send_offsets_to_transaction`, then `commit_transaction` atomically, and
    /// clear the tasks' pending offsets. Requires `meta` (the streams group
    /// metadata). A no-op when no transaction is open (nothing produced since the
    /// last commit).
    pub async fn commit_all(
        &mut self,
        meta: Option<&StreamsGroupMeta>,
    ) -> Result<(), StreamsClientError> {
        match self.guarantee {
            ProcessingGuarantee::AtLeastOnce => {
                for task in self.tasks.values_mut() {
                    task.commit().await?;
                }
            }
            ProcessingGuarantee::ExactlyOnceV2 => {
                if !self.in_txn {
                    return Ok(()); // nothing produced since last commit
                }
                // Capture the txn-commit sequence so any error (e.g. a failed
                // `commit_transaction`) aborts the txn and rolls every task back to
                // the last committed state. The cycle is then retried on the next
                // interval (so `commit_all` returns Ok after a clean rollback).
                let res = self.eos_send_offsets_and_commit(meta).await;
                if res.is_err() {
                    self.abort_and_rollback().await?;
                    return Ok(());
                }
                for task in self.tasks.values_mut() {
                    task.clear_pending();
                }
                self.in_txn = false;
            }
        }
        Ok(())
    }

    /// Serve one interactive query against this thread's local tasks. Composite
    /// across every task whose registry hosts the named store.
    ///
    /// Takes `&mut self` (not `&self`): the query borrows `&dyn IqQueryable`
    /// views out of the task graphs and holds them across `answer_iq`'s awaits.
    /// A `&self` body would capture `&StreamThread` across the await, requiring
    /// `StreamThread: Sync` — but the graph holds `Box<dyn StateStore>` /
    /// `Box<dyn ErasedNode>` which are `Send` but not `Sync`, so the supervisor's
    /// spawned future would not be `Send`. `&mut self` only needs `Send`.
    pub(crate) async fn serve_iq(&mut self, req: IqRequest) {
        let matching: Vec<&dyn crate::store::iq::IqQueryable> = self
            .tasks
            .values()
            .filter_map(|t| t.registry().iq_get(&req.store))
            .collect();
        let result = answer_iq(
            matching,
            req.kind,
            &req.op,
            &req.store,
            !self.tasks.is_empty(),
        )
        .await;
        let _ = req.reply.send(result);
    }

    /// Commit + drop all tasks (on Fenced / shutdown).
    ///
    /// Under EOS, an open transaction is aborted (best-effort) rather than
    /// committed — a fence/shutdown mid-cycle must not leak a half-written txn.
    pub async fn close_all(
        &mut self,
        meta: Option<&StreamsGroupMeta>,
    ) -> Result<(), StreamsClientError> {
        match self.guarantee {
            ProcessingGuarantee::AtLeastOnce => {
                self.commit_all(meta).await?;
            }
            ProcessingGuarantee::ExactlyOnceV2 => {
                // Abort any in-flight txn (best-effort) — a fence/shutdown mid-cycle
                // must not leak a half-written transaction. (Clean rollback = T4.)
                if self.in_txn {
                    if let Some(t) = &self.txn {
                        let _ = t.abort_transaction().await;
                    }
                    self.in_txn = false;
                }
            }
        }
        self.tasks.clear();
        Ok(())
    }
}

/// Lazy begin-transaction gate handed to each task's `process_once` under
/// EOS-v2. The first task to produce a record this interval calls
/// [`BeginTxnGate::ensure_begun`], which begins the transaction exactly once;
/// subsequent calls (further records / partitions / tasks) are no-ops. When no
/// task produces anything the gate is never tripped and no transaction opens.
struct EosBeginGate {
    txn: Arc<dyn TransactionalProducer>,
    /// Whether a transaction is currently open. Seeded from the thread's
    /// `in_txn` (so a re-poll within an already-open interval doesn't re-begin)
    /// and read back into it after processing.
    begun: bool,
}

#[async_trait::async_trait]
impl BeginTxnGate for EosBeginGate {
    async fn ensure_begun(&mut self) -> Result<(), StreamsClientError> {
        if !self.begun {
            self.txn.begin_transaction().await?;
            self.begun = true;
        }
        Ok(())
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::membership::{StreamsAssignment, TaskAssignment, TaskOffsetTracker, TopicPartition};
    use crate::processor::api::{Processor, ProcessorContext};
    use crate::processor::record::Record;
    use crate::processor::serde::{Consumed, I64Serde, Produced, StringSerde};
    use crate::runtime::io::{
        FetchBatch, FetchedRec, IsolationLevel, OffsetStore, RecordFetcher, RecordProducer,
    };
    use crate::topology::Topology;
    use assert2::check;
    use std::collections::HashMap;
    use std::sync::Arc;
    use std::sync::Mutex as StdMutex;

    // ─── stateless Upper processor ────────────────────────────────────────────

    struct Upper;
    #[async_trait::async_trait]
    impl Processor<String, String, String, String> for Upper {
        async fn process(
            &mut self,
            ctx: &mut ProcessorContext<'_, '_, String, String>,
            r: Record<String, String>,
        ) {
            ctx.forward(Record::new(r.key, r.value.to_uppercase(), r.timestamp));
        }
    }

    fn built() -> crate::topology::BuiltTopology {
        let mut t = Topology::new();
        let src = t.add_source("src", ["in"], Consumed::with(StringSerde, StringSerde));
        let up = t.add_processor("up", || Upper, [&src]);
        t.add_sink(
            "out",
            "out",
            [&up],
            Produced::with(StringSerde, StringSerde),
        );
        t.build("app").unwrap()
    }

    // ─── stateful Counter processor ───────────────────────────────────────────

    struct Counter;
    #[async_trait::async_trait]
    impl Processor<String, String, String, i64> for Counter {
        async fn process(
            &mut self,
            ctx: &mut ProcessorContext<'_, '_, String, i64>,
            r: Record<String, String>,
        ) {
            let n = {
                let store = ctx.get_state_store::<String, i64>("counts").unwrap();
                let n = store.get(&r.value).await.unwrap_or(0) + 1;
                store.put(r.value.clone(), n).await;
                n
            };
            ctx.forward(Record::new(Some(r.value), n, r.timestamp));
        }
    }

    fn stateful_built() -> crate::topology::BuiltTopology {
        let mut t = Topology::new();
        let src = t.add_source("src", ["in"], Consumed::with(StringSerde, StringSerde));
        let c = t.add_processor("c", || Counter, [&src]);
        t.add_state_store("counts", StringSerde, I64Serde, [c.name()]);
        t.add_sink("out", "out", [&c], Produced::with(StringSerde, I64Serde));
        t.build("app").unwrap()
    }

    // ─── wall-clock punctuator scheduler ──────────────────────────────────────

    struct EmitTs;
    #[async_trait::async_trait]
    impl crate::processor::punctuation::Punctuator<String, i64> for EmitTs {
        async fn punctuate(&mut self, ctx: &mut ProcessorContext<'_, '_, String, i64>, ts: i64) {
            ctx.forward(Record::new(None, ts, ts));
        }
    }

    /// Schedules a `WALL_CLOCK_TIME` punctuator (interval 100ms) in `init`; no-op on
    /// records (so any sink output is from the wall-clock punctuator).
    struct WallClockScheduler;
    #[async_trait::async_trait]
    impl Processor<String, String, String, i64> for WallClockScheduler {
        async fn init(&mut self, ctx: &mut ProcessorContext<'_, '_, String, i64>) {
            ctx.schedule(
                std::time::Duration::from_millis(100),
                crate::processor::punctuation::PunctuationType::WallClockTime,
                EmitTs,
            );
        }
        async fn process(
            &mut self,
            _ctx: &mut ProcessorContext<'_, '_, String, i64>,
            _r: Record<String, String>,
        ) {
        }
    }

    fn wall_clock_built() -> crate::topology::BuiltTopology {
        let mut t = Topology::new();
        let src = t.add_source("src", ["in"], Consumed::with(StringSerde, StringSerde));
        let p = t.add_processor("p", || WallClockScheduler, [&src]);
        t.add_sink("out", "out", [&p], Produced::with(StringSerde, I64Serde));
        t.build("app").unwrap()
    }

    // ─── fakes ────────────────────────────────────────────────────────────────

    /// Returns one batch at its scripted (topic, partition, offset), then empty.
    struct ScriptedFetcher {
        scripts: StdMutex<HashMap<(String, i32, i64), FetchBatch>>,
    }

    impl ScriptedFetcher {
        fn new(scripts: Vec<((String, i32, i64), FetchBatch)>) -> Self {
            Self {
                scripts: StdMutex::new(scripts.into_iter().collect()),
            }
        }
    }

    #[async_trait::async_trait]
    impl RecordFetcher for ScriptedFetcher {
        async fn fetch(
            &self,
            t: &str,
            p: i32,
            o: i64,
            _isolation: IsolationLevel,
        ) -> Result<FetchBatch, crate::StreamsClientError> {
            Ok(self
                .scripts
                .lock()
                .unwrap()
                .remove(&(t.to_string(), p, o))
                .unwrap_or_default())
        }
    }

    struct OneShot {
        batch: StdMutex<Option<FetchBatch>>,
    }

    #[async_trait::async_trait]
    impl RecordFetcher for OneShot {
        async fn fetch(
            &self,
            _t: &str,
            _p: i32,
            _o: i64,
            _isolation: IsolationLevel,
        ) -> Result<FetchBatch, crate::StreamsClientError> {
            Ok(self.batch.lock().unwrap().take().unwrap_or_default())
        }
    }

    type SentRecord = (
        String,
        Option<i32>,
        Option<bytes::Bytes>,
        Option<bytes::Bytes>,
    );

    #[derive(Default)]
    struct CollectProducer {
        /// (topic, partition, key, value)
        sent: StdMutex<Vec<SentRecord>>,
        flushes: StdMutex<u32>,
    }

    #[async_trait::async_trait]
    impl RecordProducer for CollectProducer {
        async fn send(
            &self,
            topic: &str,
            partition: Option<i32>,
            k: Option<bytes::Bytes>,
            v: Option<bytes::Bytes>,
        ) -> Result<(), crate::StreamsClientError> {
            self.sent
                .lock()
                .unwrap()
                .push((topic.to_string(), partition, k, v));
            Ok(())
        }

        async fn flush(&self) -> Result<(), crate::StreamsClientError> {
            *self.flushes.lock().unwrap() += 1;
            Ok(())
        }
    }

    #[derive(Default)]
    struct MemStore {
        committed: StdMutex<HashMap<(String, i32), i64>>,
    }

    #[async_trait::async_trait]
    impl OffsetStore for MemStore {
        async fn committed(
            &self,
            t: &str,
            p: i32,
        ) -> Result<Option<i64>, crate::StreamsClientError> {
            Ok(self
                .committed
                .lock()
                .unwrap()
                .get(&(t.to_string(), p))
                .copied())
        }

        async fn earliest(&self, _t: &str, _p: i32) -> Result<i64, crate::StreamsClientError> {
            Ok(0)
        }

        async fn latest(&self, _t: &str, _p: i32) -> Result<i64, crate::StreamsClientError> {
            Ok(0)
        }

        async fn commit(
            &self,
            offs: &[(String, i32, i64)],
        ) -> Result<(), crate::StreamsClientError> {
            let mut m = self.committed.lock().unwrap();
            for (t, p, o) in offs {
                m.insert((t.clone(), *p), *o);
            }
            Ok(())
        }
    }

    fn assignment() -> StreamsAssignment {
        StreamsAssignment {
            active: vec![TaskAssignment {
                subtopology_id: "0".into(),
                partitions: vec![0],
                source_topic_partitions: vec![TopicPartition {
                    topic: "in".into(),
                    partition: 0,
                }],
            }],
            standby: vec![],
            warmup: vec![],
        }
    }

    fn empty_fetcher() -> Arc<dyn RecordFetcher> {
        // Returns empty for all fetches; used when restore has nothing to replay.
        Arc::new(ScriptedFetcher::new(vec![])) as Arc<dyn RecordFetcher>
    }

    // ─── tests ────────────────────────────────────────────────────────────────

    /// `poll_all` must fire due `WALL_CLOCK_TIME` punctuators between polls, driven
    /// by the injected `Clock`. We use a `ManualClock` over a shared atomic so we
    /// can advance wall time deterministically:
    ///   - `init` schedules the punctuator at base `wall_clock`=0 → next fire = 100.
    ///   - clock=0: first `poll_all` → now=0 < 100, no fire.
    ///   - advance clock to 150: second `poll_all` → now=150 >= 100, fires ONCE,
    ///     emitting value = now = 150 to the "out" sink.
    #[tokio::test]
    async fn poll_all_fires_wall_clock_punctuation_via_manual_clock() {
        use std::sync::atomic::AtomicI64;

        let producer_c = Arc::new(CollectProducer::default());
        let store_c = Arc::new(MemStore::default());
        let producer: Arc<dyn RecordProducer> = Arc::clone(&producer_c) as _;
        let store: Arc<dyn OffsetStore> = Arc::clone(&store_c) as _;
        let built = wall_clock_built();
        let tracker = Arc::new(TokioMutex::new(TaskOffsetTracker::default()));

        let now = Arc::new(AtomicI64::new(0));
        let clock: Arc<dyn crate::runtime::clock::Clock> =
            Arc::new(crate::runtime::clock::ManualClock(Arc::clone(&now)));
        let mut thread = StreamThread::new(
            empty_fetcher(),
            crate::store::backend::StoreBackend::InMemory,
            "app".into(),
        )
        .with_clock(clock);
        thread
            .apply_assignment(
                &assignment(),
                &built,
                &producer,
                &store,
                crate::runtime::eos::ProcessingGuarantee::AtLeastOnce,
                None,
            )
            .await
            .unwrap();
        check!(thread.task_count() == 1);

        // clock=0 → no record source, no wall-clock fire yet (now=0 < next=100).
        thread.poll_all(&*empty_fetcher(), &tracker).await.unwrap();
        check!(
            !producer_c
                .sent
                .lock()
                .unwrap()
                .iter()
                .any(|(t, _p, _k, _v)| t == "out"),
            "no wall-clock punctuation should fire before the interval elapses"
        );

        // Advance wall time past one interval; the next poll must fire the
        // punctuator (value = now = 150) and produce it to "out".
        now.store(150, std::sync::atomic::Ordering::SeqCst);
        thread.poll_all(&*empty_fetcher(), &tracker).await.unwrap();
        check!(
            producer_c
                .sent
                .lock()
                .unwrap()
                .iter()
                .any(|(t, _p, _k, v)| t == "out"
                    && v.as_deref() == Some(150i64.to_be_bytes().as_ref())),
            "wall-clock punctuator must fire from poll_all once the ManualClock passes the interval, emitting value=150"
        );
    }

    #[tokio::test]
    async fn apply_assignment_creates_task_polls_commits() {
        let producer_c = Arc::new(CollectProducer::default());
        let store_c = Arc::new(MemStore::default());
        let producer: Arc<dyn RecordProducer> = Arc::clone(&producer_c) as _;
        let store: Arc<dyn OffsetStore> = Arc::clone(&store_c) as _;
        let built = built();
        let mut thread = StreamThread::new(
            empty_fetcher(),
            crate::store::backend::StoreBackend::InMemory,
            "app".into(),
        );
        thread
            .apply_assignment(
                &assignment(),
                &built,
                &producer,
                &store,
                ProcessingGuarantee::AtLeastOnce,
                None,
            )
            .await
            .unwrap();
        check!(thread.task_count() == 1);

        let tracker = Arc::new(TokioMutex::new(TaskOffsetTracker::default()));
        let fetcher = OneShot {
            batch: StdMutex::new(Some(FetchBatch {
                records: vec![FetchedRec {
                    offset: 0,
                    key: Some("k".into()),
                    value: Some("hi".into()),
                    timestamp: -1,
                }],
            })),
        };
        thread.poll_all(&fetcher, &tracker).await.unwrap();
        thread.commit_all(None).await.unwrap();
        check!(
            producer_c
                .sent
                .lock()
                .unwrap()
                .iter()
                .any(|(t, _p, _k, v)| t == "out" && v.as_deref() == Some(b"HI".as_ref()))
        );
        check!(
            store_c
                .committed
                .lock()
                .unwrap()
                .get(&("in".to_string(), 0))
                == Some(&1)
        );

        // empty assignment → task removed (close_processors + committed on the way out)
        thread
            .apply_assignment(
                &StreamsAssignment::default(),
                &built,
                &producer,
                &store,
                ProcessingGuarantee::AtLeastOnce,
                None,
            )
            .await
            .unwrap();
        check!(thread.task_count() == 0);
    }

    /// Verify that `apply_assignment` replays changelog records into the task's
    /// store during restore, so that the first `process_once` continues from the
    /// restored count rather than from zero.
    #[tokio::test]
    async fn stateful_apply_assignment_restores_store_from_changelog() {
        // Changelog: key="a", value=i64 BE 7 at offset 0 on "app-counts-changelog".
        let cl_key = bytes::Bytes::copy_from_slice(b"a");
        let cl_val = bytes::Bytes::copy_from_slice(&7i64.to_be_bytes());
        let restore_fetcher: Arc<dyn RecordFetcher> = Arc::new(ScriptedFetcher::new(vec![(
            ("app-counts-changelog".to_string(), 0, 0),
            FetchBatch {
                records: vec![FetchedRec {
                    offset: 0,
                    key: Some(cl_key),
                    value: Some(cl_val),
                    timestamp: -1,
                }],
            },
        )]));

        let producer_c = Arc::new(CollectProducer::default());
        let store_c = Arc::new(MemStore::default());
        let producer: Arc<dyn RecordProducer> = Arc::clone(&producer_c) as _;
        let store: Arc<dyn OffsetStore> = Arc::clone(&store_c) as _;
        let built = stateful_built();

        let mut thread = StreamThread::new(
            Arc::clone(&restore_fetcher),
            crate::store::backend::StoreBackend::InMemory,
            "app".into(),
        );
        thread
            .apply_assignment(
                &assignment(),
                &built,
                &producer,
                &store,
                ProcessingGuarantee::AtLeastOnce,
                None,
            )
            .await
            .unwrap();
        check!(thread.task_count() == 1);

        // Now process one "a" record.  Restored count is 7, so output must be 8.
        let tracker = Arc::new(TokioMutex::new(TaskOffsetTracker::default()));
        let process_fetcher = ScriptedFetcher::new(vec![(
            ("in".to_string(), 0, 0),
            FetchBatch {
                records: vec![FetchedRec {
                    offset: 0,
                    key: None,
                    value: Some("a".into()),
                    timestamp: -1,
                }],
            },
        )]);
        thread.poll_all(&process_fetcher, &tracker).await.unwrap();
        thread.commit_all(None).await.unwrap();

        let sent = producer_c.sent.lock().unwrap();
        check!(
            sent.iter()
                .any(|(t, _p, _k, v)| t == "out"
                    && v.as_deref() == Some(8i64.to_be_bytes().as_ref())),
            "after restore with N=7, processing 'a' must emit count = 8"
        );
    }

    /// `serve_iq` must resolve a `KvGet` against the live, restored task store:
    /// after restoring `counts` with `a=7` from the changelog (same setup as
    /// `stateful_apply_assignment_restores_store_from_changelog`), a `KvGet` for
    /// "a" returns the i64-BE bytes for 7. A thread with no tasks (rebalancing)
    /// returns `RebalanceInProgress`.
    #[tokio::test]
    async fn serve_iq_reads_restored_kv_store() {
        use crate::processor::serde::{I64Serde, Serde, StringSerde};
        use crate::runtime::iq::{IqError, IqOp, IqPayload, IqRequest};
        use crate::store::iq::StoreKind;

        // --- build a thread + restore `counts` with a=7 (copied from
        //     stateful_apply_assignment_restores_store_from_changelog) ---
        // Changelog: key="a", value=i64 BE 7 at offset 0 on "app-counts-changelog".
        let cl_key = bytes::Bytes::copy_from_slice(b"a");
        let cl_val = bytes::Bytes::copy_from_slice(&7i64.to_be_bytes());
        let restore_fetcher: Arc<dyn RecordFetcher> = Arc::new(ScriptedFetcher::new(vec![(
            ("app-counts-changelog".to_string(), 0, 0),
            FetchBatch {
                records: vec![FetchedRec {
                    offset: 0,
                    key: Some(cl_key),
                    value: Some(cl_val),
                    timestamp: -1,
                }],
            },
        )]));

        let producer_c = Arc::new(CollectProducer::default());
        let store_c = Arc::new(MemStore::default());
        let producer: Arc<dyn RecordProducer> = Arc::clone(&producer_c) as _;
        let store: Arc<dyn OffsetStore> = Arc::clone(&store_c) as _;
        let built = stateful_built();

        let mut thread = StreamThread::new(
            Arc::clone(&restore_fetcher),
            crate::store::backend::StoreBackend::InMemory,
            "app".into(),
        );
        thread
            .apply_assignment(
                &assignment(),
                &built,
                &producer,
                &store,
                crate::runtime::eos::ProcessingGuarantee::AtLeastOnce,
                None,
            )
            .await
            .unwrap();
        check!(thread.task_count() == 1);

        // happy path: get "a" -> 7
        let (reply, rx) = tokio::sync::oneshot::channel();
        thread
            .serve_iq(IqRequest {
                store: "counts".into(),
                kind: StoreKind::KeyValue,
                op: IqOp::KvGet {
                    key: StringSerde.serialize(&"a".to_string()),
                },
                reply,
            })
            .await;
        assert_eq!(
            rx.await.unwrap().unwrap(),
            IqPayload::Value(Some(I64Serde.serialize(&7_i64)))
        );

        // empty thread (no tasks) -> RebalanceInProgress
        let mut empty = StreamThread::new(
            empty_fetcher(),
            crate::store::backend::StoreBackend::InMemory,
            "app".into(),
        );
        let (reply2, rx2) = tokio::sync::oneshot::channel();
        empty
            .serve_iq(IqRequest {
                store: "counts".into(),
                kind: StoreKind::KeyValue,
                op: IqOp::KvGet {
                    key: StringSerde.serialize(&"a".to_string()),
                },
                reply: reply2,
            })
            .await;
        assert!(matches!(
            rx2.await.unwrap(),
            Err(IqError::RebalanceInProgress)
        ));
    }

    /// End-to-end of the real runtime global-store path: `StreamThread` builds +
    /// bootstraps the shared `GlobalStateManager` from the broker BEFORE any task
    /// processes, then a stream-globaltable join reads the bootstrapped value.
    ///
    /// Topology: a `GlobalKTable` over topic "global" (store "g-store"), and a
    /// stream "in" that joins it with `key_mapper = |_k, v| v.clone()` (lookup key
    /// = the record value) and `joiner = |sv, gv| sv + gv`. The global store is
    /// seeded on the broker at (global, 0, 0) = ("gk", "GV"); the stream record is
    /// (key "k", value "gk"). The derived lookup key "gk" hits "GV", so the join
    /// emits key "k", value "gkGV". Proves bootstrap-before-process wires the
    /// shared manager into the task graph in the real runtime.
    #[tokio::test]
    async fn global_apply_assignment_bootstraps_store_before_join() {
        use crate::dsl::{GlobalKTable, Materialized, StreamsBuilder};

        // Build the global-table join topology via the DSL.
        let b = StreamsBuilder::new();
        let g: GlobalKTable<String, String> = b.global_table_explicit(
            "global",
            Consumed::with(StringSerde, StringSerde),
            Materialized::with(StringSerde, StringSerde).as_store("g-store"),
        );
        b.stream_explicit(["in"], Consumed::with(StringSerde, StringSerde))
            .join_global(
                &g,
                |_k: &String, v: &String| v.clone(),
                |sv: &String, gv: &String| format!("{sv}{gv}"),
            )
            .to_explicit("out", Produced::with(StringSerde, StringSerde));
        drop(g);
        let built = b.build("app").unwrap();

        // Bootstrap fetcher: serves the single global record at (global, 0, 0), then
        // empty. The default `partitions("global")` is vec![0], which matches the
        // single-partition global topic. The "in" restore replays nothing (no state
        // store on the stream subtopology).
        let boot_fetcher: Arc<dyn RecordFetcher> = Arc::new(ScriptedFetcher::new(vec![(
            ("global".to_string(), 0, 0),
            FetchBatch {
                records: vec![FetchedRec {
                    offset: 0,
                    key: Some("gk".into()),
                    value: Some("GV".into()),
                    timestamp: -1,
                }],
            },
        )]));

        let producer_c = Arc::new(CollectProducer::default());
        let store_c = Arc::new(MemStore::default());
        let producer: Arc<dyn RecordProducer> = Arc::clone(&producer_c) as _;
        let store: Arc<dyn OffsetStore> = Arc::clone(&store_c) as _;

        let mut thread = StreamThread::new(
            Arc::clone(&boot_fetcher),
            crate::store::backend::StoreBackend::InMemory,
            "app".into(),
        );

        // The global-table topology emits the stream subtopology as id "1".
        let assignment = StreamsAssignment {
            active: vec![TaskAssignment {
                subtopology_id: "1".into(),
                partitions: vec![0],
                source_topic_partitions: vec![TopicPartition {
                    topic: "in".into(),
                    partition: 0,
                }],
            }],
            standby: vec![],
            warmup: vec![],
        };
        thread
            .apply_assignment(
                &assignment,
                &built,
                &producer,
                &store,
                ProcessingGuarantee::AtLeastOnce,
                None,
            )
            .await
            .unwrap();
        check!(thread.task_count() == 1);

        // Now process one stream record (key "k", value "gk"). The key-mapper derives
        // lookup key "gk", which the bootstrapped global store resolves to "GV", so
        // the join emits key "k", value "gk" + "GV" = "gkGV".
        let tracker = Arc::new(TokioMutex::new(TaskOffsetTracker::default()));
        let process_fetcher = ScriptedFetcher::new(vec![(
            ("in".to_string(), 0, 0),
            FetchBatch {
                records: vec![FetchedRec {
                    offset: 0,
                    key: Some("k".into()),
                    value: Some("gk".into()),
                    timestamp: -1,
                }],
            },
        )]);
        thread.poll_all(&process_fetcher, &tracker).await.unwrap();
        thread.commit_all(None).await.unwrap();

        let sent = producer_c.sent.lock().unwrap();
        check!(
            sent.iter().any(|(t, _p, k, v)| t == "out"
                && k.as_deref() == Some(b"k".as_ref())
                && v.as_deref() == Some(b"gkGV".as_ref())),
            "join must see the bootstrapped global value: ('out', key 'k', value 'gkGV')"
        );
    }

    #[tokio::test]
    async fn reconciles_active_standby_warmup_roles_and_transitions() {
        let producer_c = Arc::new(CollectProducer::default());
        let store_c = Arc::new(MemStore::default());
        let producer: Arc<dyn RecordProducer> = Arc::clone(&producer_c) as _;
        let store: Arc<dyn OffsetStore> = Arc::clone(&store_c) as _;
        let built = built();

        let mut thread = StreamThread::new(
            empty_fetcher(),
            crate::store::backend::StoreBackend::InMemory,
            "app".into(),
        );

        // 1. Initial assignment:
        // Subtopology 0 Partition 0 -> Active
        // Subtopology 0 Partition 1 -> Standby
        // Subtopology 0 Partition 2 -> Warmup
        let assignment1 = StreamsAssignment {
            active: vec![TaskAssignment {
                subtopology_id: "0".into(),
                partitions: vec![0],
                source_topic_partitions: vec![TopicPartition {
                    topic: "in".into(),
                    partition: 0,
                }],
            }],
            standby: vec![TaskAssignment {
                subtopology_id: "0".into(),
                partitions: vec![1],
                source_topic_partitions: vec![TopicPartition {
                    topic: "in".into(),
                    partition: 1,
                }],
            }],
            warmup: vec![TaskAssignment {
                subtopology_id: "0".into(),
                partitions: vec![2],
                source_topic_partitions: vec![TopicPartition {
                    topic: "in".into(),
                    partition: 2,
                }],
            }],
        };

        thread
            .apply_assignment(
                &assignment1,
                &built,
                &producer,
                &store,
                ProcessingGuarantee::AtLeastOnce,
                None,
            )
            .await
            .unwrap();
        check!(thread.task_count() == 3);

        check!(thread.tasks.get(&("0".to_string(), 0)).map(|t| t.role) == Some(TaskRole::Active));
        check!(thread.tasks.get(&("0".to_string(), 1)).map(|t| t.role) == Some(TaskRole::Standby));
        check!(thread.tasks.get(&("0".to_string(), 2)).map(|t| t.role) == Some(TaskRole::Warmup));

        // 2. Updated assignment:
        // Subtopology 0 Partition 0 -> Standby (Demoted)
        // Subtopology 0 Partition 1 -> removed
        // Subtopology 0 Partition 2 -> Active (Promoted)
        // Subtopology 0 Partition 3 -> Warmup (New)
        let assignment2 = StreamsAssignment {
            active: vec![TaskAssignment {
                subtopology_id: "0".into(),
                partitions: vec![2],
                source_topic_partitions: vec![TopicPartition {
                    topic: "in".into(),
                    partition: 2,
                }],
            }],
            standby: vec![TaskAssignment {
                subtopology_id: "0".into(),
                partitions: vec![0],
                source_topic_partitions: vec![TopicPartition {
                    topic: "in".into(),
                    partition: 0,
                }],
            }],
            warmup: vec![TaskAssignment {
                subtopology_id: "0".into(),
                partitions: vec![3],
                source_topic_partitions: vec![TopicPartition {
                    topic: "in".into(),
                    partition: 3,
                }],
            }],
        };

        thread
            .apply_assignment(
                &assignment2,
                &built,
                &producer,
                &store,
                ProcessingGuarantee::AtLeastOnce,
                None,
            )
            .await
            .unwrap();
        check!(thread.task_count() == 3);

        check!(thread.tasks.get(&("0".to_string(), 0)).map(|t| t.role) == Some(TaskRole::Standby));
        check!(!thread.tasks.contains_key(&("0".to_string(), 1)));
        check!(thread.tasks.get(&("0".to_string(), 2)).map(|t| t.role) == Some(TaskRole::Active));
        check!(thread.tasks.get(&("0".to_string(), 3)).map(|t| t.role) == Some(TaskRole::Warmup));
    }

    /// EOS-v2 happy path: the thread runs the full transactional commit lifecycle
    /// over a stateless `source → up → sink` topology. The single
    /// `MockTransactionalProducer` is shared as BOTH the task `RecordProducer`
    /// (for the sink `send`) AND the thread's `TransactionalProducer`, so the
    /// recorded call sequence is the cross-product of both views.
    ///
    /// Expected sequence: `Init` (`apply_assignment`), `Begin` (first `poll_all`),
    /// `Send` (the sink emit during process), then `SendOffsets` + `Commit`
    /// (`commit_all`). The sink record must also be logged in `.sent`.
    #[tokio::test]
    async fn eos_happy_path_runs_begin_send_offsets_commit() {
        use crate::runtime::eos::mock::{MockTransactionalProducer, Step};

        // One mock object, two trait-object views (same Arc).
        let mock = Arc::new(MockTransactionalProducer::default());
        let producer: Arc<dyn RecordProducer> = Arc::clone(&mock) as _;
        let txn: Arc<dyn TransactionalProducer> = Arc::clone(&mock) as _;
        let store: Arc<dyn OffsetStore> = Arc::new(MemStore::default());

        let built = built();
        let mut thread = StreamThread::new(
            empty_fetcher(),
            crate::store::backend::StoreBackend::InMemory,
            "app".into(),
        );
        // EOS assignment: passes the txn producer and ExactlyOnceV2.
        thread
            .apply_assignment(
                &assignment(),
                &built,
                &producer,
                &store,
                ProcessingGuarantee::ExactlyOnceV2,
                Some(Arc::clone(&txn)),
            )
            .await
            .unwrap();
        check!(thread.task_count() == 1);

        // One input batch → the sink emits one uppercased record.
        let fetcher = OneShot {
            batch: StdMutex::new(Some(FetchBatch {
                records: vec![FetchedRec {
                    offset: 0,
                    key: Some("k".into()),
                    value: Some("hi".into()),
                    timestamp: -1,
                }],
            })),
        };
        let tracker = Arc::new(TokioMutex::new(TaskOffsetTracker::default()));
        thread.poll_all(&fetcher, &tracker).await.unwrap();

        let meta = crate::runtime::eos::StreamsGroupMeta {
            group_id: "app".into(),
            generation_id: 3,
            member_id: "m".into(),
            group_instance_id: None,
        };
        thread.commit_all(Some(&meta)).await.unwrap();

        // The full transactional lifecycle, in order.
        check!(
            *mock.calls.lock().unwrap()
                == vec![
                    Step::Init,
                    Step::Begin,
                    Step::Send,
                    Step::SendOffsets,
                    Step::Commit,
                ]
        );
        // The sink record was produced through the transactional producer.
        check!(
            mock.sent
                .lock()
                .unwrap()
                .iter()
                .any(|(t, _p, _k, v)| t == "out" && v.as_deref() == Some(b"HI".as_ref()))
        );
    }

    /// EOS-v2 idle interval: when a `poll_all` fetches NO records, the runtime
    /// must NOT begin a transaction, and the following `commit_all` must be a
    /// no-op — no `Begin`, no `Send`, no `SendOffsets`, no `Commit`. (Regression
    /// guard for the empty-transaction churn the begin-on-first-record gate
    /// fixes: the old eager begin opened + committed an empty txn every interval
    /// on an idle app.) Only `Init` (from `apply_assignment`) is recorded.
    #[tokio::test]
    async fn eos_idle_interval_opens_no_transaction() {
        use crate::runtime::eos::mock::{MockTransactionalProducer, Step};

        let mock = Arc::new(MockTransactionalProducer::default());
        let producer: Arc<dyn RecordProducer> = Arc::clone(&mock) as _;
        let txn: Arc<dyn TransactionalProducer> = Arc::clone(&mock) as _;
        let store: Arc<dyn OffsetStore> = Arc::new(MemStore::default());

        let built = built();
        let mut thread = StreamThread::new(
            empty_fetcher(),
            crate::store::backend::StoreBackend::InMemory,
            "app".into(),
        );
        thread
            .apply_assignment(
                &assignment(),
                &built,
                &producer,
                &store,
                ProcessingGuarantee::ExactlyOnceV2,
                Some(Arc::clone(&txn)),
            )
            .await
            .unwrap();
        check!(thread.task_count() == 1);

        // Idle interval: the fetcher returns empty for every fetch.
        let idle_fetcher = empty_fetcher();
        let tracker = Arc::new(TokioMutex::new(TaskOffsetTracker::default()));
        thread.poll_all(&*idle_fetcher, &tracker).await.unwrap();

        let meta = crate::runtime::eos::StreamsGroupMeta {
            group_id: "app".into(),
            generation_id: 3,
            member_id: "m".into(),
            group_instance_id: None,
        };
        thread.commit_all(Some(&meta)).await.unwrap();

        // Only the one-time Init ran — no transaction was opened or committed.
        check!(
            *mock.calls.lock().unwrap() == vec![Step::Init],
            "idle interval must open no transaction; got {:?}",
            *mock.calls.lock().unwrap()
        );
        check!(mock.sent.lock().unwrap().is_empty());
    }

    /// EOS-v2 abort + rollback: a `commit_transaction` failure mid-cycle must
    /// abort the txn and roll every task back to its last committed state —
    /// rewinding source offsets, wiping the stores, and re-restoring from the
    /// (here empty) changelog. A subsequent successful cycle then reprocesses the
    /// re-fetched batch without double-counting.
    ///
    /// Topology: stateful `source → counter (counts store) → sink`. The fetcher
    /// returns the SAME "a" record for `("in", 0, 0)` on every fetch (so the
    /// rewound cycle re-reads it) and an empty changelog (so re-restore yields an
    /// empty store).
    #[tokio::test]
    async fn eos_commit_failure_aborts_and_rolls_back() {
        use crate::runtime::eos::mock::{MockTransactionalProducer, Step};

        /// A fetcher that ALWAYS returns the "a" record at `("in", 0, 0)`
        /// regardless of how many times it's fetched (it never consumes the
        /// script), and an empty changelog. Re-fetching after a rewind re-reads
        /// the same input — proving the rollback rewound the source offset.
        struct ReplayFetcher;
        #[async_trait::async_trait]
        impl RecordFetcher for ReplayFetcher {
            async fn fetch(
                &self,
                t: &str,
                p: i32,
                o: i64,
                _isolation: IsolationLevel,
            ) -> Result<FetchBatch, crate::StreamsClientError> {
                if t == "in" && p == 0 && o == 0 {
                    Ok(FetchBatch {
                        records: vec![FetchedRec {
                            offset: 0,
                            key: None,
                            value: Some("a".into()),
                            timestamp: -1,
                        }],
                    })
                } else {
                    Ok(FetchBatch::default())
                }
            }
        }

        // One mock object, two trait-object views (same Arc). Fail the FIRST commit.
        let mock = Arc::new(MockTransactionalProducer {
            fail_at: StdMutex::new(Some(Step::Commit)),
            ..Default::default()
        });
        let producer: Arc<dyn RecordProducer> = Arc::clone(&mock) as _;
        let txn: Arc<dyn TransactionalProducer> = Arc::clone(&mock) as _;
        let store: Arc<dyn OffsetStore> = Arc::new(MemStore::default());

        let built = stateful_built();
        let replay: Arc<dyn RecordFetcher> = Arc::new(ReplayFetcher);
        let mut thread = StreamThread::new(
            Arc::clone(&replay),
            crate::store::backend::StoreBackend::InMemory,
            "app".into(),
        );
        thread
            .apply_assignment(
                &assignment(),
                &built,
                &producer,
                &store,
                ProcessingGuarantee::ExactlyOnceV2,
                Some(Arc::clone(&txn)),
            )
            .await
            .unwrap();
        check!(thread.task_count() == 1);

        let meta = crate::runtime::eos::StreamsGroupMeta {
            group_id: "app".into(),
            generation_id: 3,
            member_id: "m".into(),
            group_instance_id: None,
        };
        let key = ("0".to_string(), 0);
        let tracker = Arc::new(TokioMutex::new(TaskOffsetTracker::default()));

        // ── Cycle 1: begin + process (count "a" → 1, store dirty), then commit
        //    FAILS → abort + rollback. ──────────────────────────────────────────
        thread.poll_all(&*replay, &tracker).await.unwrap();
        // The dirty count is in the store before commit.
        check!(
            thread
                .task_store_get_i64(&key, "counts", &"a".to_string())
                .await
                == Some(1)
        );
        // Commit fails internally → abort + rollback, but commit_all returns Ok
        // (the cycle is rolled back; the next interval re-begins).
        thread.commit_all(Some(&meta)).await.unwrap();

        // The mock recorded the abort.
        check!(mock.calls.lock().unwrap().contains(&Step::Abort));
        // Pending offsets were cleared by the rollback.
        check!(!thread.task_has_pending(&key));
        // The store was rolled back: re-restored from the empty changelog, so the
        // dirty count is gone.
        check!(
            thread
                .task_store_get_i64(&key, "counts", &"a".to_string())
                .await
                == None,
            "store must be rolled back to the (empty) committed changelog state"
        );

        // ── Cycle 2: fail_at is now None → the re-fetched "a" batch reprocesses
        //    and yields count = 1 (NOT double-counted to 2). ────────────────────
        thread.poll_all(&*replay, &tracker).await.unwrap();
        thread.commit_all(Some(&meta)).await.unwrap();
        check!(
            thread
                .task_store_get_i64(&key, "counts", &"a".to_string())
                .await
                == Some(1),
            "after rollback + reprocess the count must be 1, not double-counted"
        );
        // The second commit succeeded.
        check!(
            mock.calls
                .lock()
                .unwrap()
                .iter()
                .filter(|s| **s == Step::Commit)
                .count()
                == 2
        );
    }
}