fluss-rs 0.1.0

The official rust client of Apache Fluss (Incubating)
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
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

use crate::client::broadcast;
use crate::client::write::IdempotenceManager;
use crate::client::write::batch::WriteBatch::{ArrowLog, Kv};
use crate::client::write::batch::{ArrowLogWriteBatch, KvWriteBatch, WriteBatch};
use crate::client::{LogWriteRecord, Record, ResultHandle, WriteRecord};
use crate::cluster::{BucketLocation, Cluster, ServerNode};
use crate::config::Config;
use crate::error::{Error, Result};
use crate::metadata::{PhysicalTablePath, TableBucket};
use crate::record::{NO_BATCH_SEQUENCE, NO_WRITER_ID};
use crate::util::current_time_ms;
use crate::{BucketId, PartitionId, TableId};
use dashmap::DashMap;
use parking_lot::{Condvar, Mutex, RwLock};
use std::collections::{HashMap, HashSet, VecDeque};
use std::sync::Arc;
use std::sync::atomic::{AtomicBool, AtomicI32, AtomicI64, AtomicUsize, Ordering};
use std::time::{Duration, Instant};
use tokio::sync::Notify;

/// Byte-counting semaphore that blocks producers when total buffered memory
/// exceeds the configured limit. Matches Java's `LazyMemorySegmentPool` behavior.
///
/// TODO: Replace `notify_all()` with per-waiter FIFO signaling (Java uses per-request
/// Condition objects in a Deque) to avoid thundering herd under high contention.
///
/// TODO: Track actual batch memory usage instead of reserving a fixed `writer_batch_size`
/// per batch. This over-counts when batches don't fill completely, reducing effective
/// throughput. Requires tighter coupling with batch internals.
pub(crate) struct MemoryLimiter {
    state: Mutex<usize>,
    cond: Condvar,
    max_memory: usize,
    wait_timeout: Duration,
    closed: AtomicBool,
    waiting_count: AtomicUsize,
}

impl MemoryLimiter {
    pub fn new(max_memory: usize, wait_timeout: Duration) -> Self {
        Self {
            state: Mutex::new(0),
            cond: Condvar::new(),
            max_memory,
            wait_timeout,
            closed: AtomicBool::new(false),
            waiting_count: AtomicUsize::new(0),
        }
    }

    /// Try to acquire `size` bytes. Blocks until memory is available,
    /// the timeout expires, or the limiter is closed.
    /// Returns a `MemoryPermit` on success.
    pub fn acquire(self: &Arc<Self>, size: usize) -> Result<MemoryPermit> {
        if self.closed.load(Ordering::Acquire) {
            return Err(Error::WriterClosed {
                message: "Memory limiter is closed".to_string(),
            });
        }

        if size > self.max_memory {
            return Err(Error::IllegalArgument {
                message: format!(
                    "Batch size {} exceeds total buffer memory limit {}",
                    size, self.max_memory
                ),
            });
        }

        let mut used = self.state.lock();
        let deadline = Instant::now() + self.wait_timeout;
        while *used + size > self.max_memory {
            self.waiting_count.fetch_add(1, Ordering::Relaxed);
            let result = self.cond.wait_until(&mut used, deadline);
            self.waiting_count.fetch_sub(1, Ordering::Relaxed);

            if self.closed.load(Ordering::Acquire) {
                return Err(Error::WriterClosed {
                    message: "Memory limiter is closed".to_string(),
                });
            }
            if result.timed_out() && *used + size > self.max_memory {
                return Err(Error::BufferExhausted {
                    message: format!(
                        "Failed to allocate {} bytes for write batch within {}ms. \
                         {} of {} bytes in use, {} threads waiting.",
                        size,
                        self.wait_timeout.as_millis(),
                        *used,
                        self.max_memory,
                        self.waiting_count.load(Ordering::Relaxed),
                    ),
                });
            }
        }

        *used += size;
        Ok(MemoryPermit {
            limiter: Arc::clone(self),
            size,
        })
    }

    fn release(&self, size: usize) {
        let mut used = self.state.lock();
        *used = used.saturating_sub(size);
        self.cond.notify_all();
    }

    /// Returns true if any producers are currently blocked waiting for memory.
    /// Used by `ready()` to mark all batches as immediately sendable when
    /// memory is exhausted (matching Java's `exhausted` flag).
    pub fn has_waiters(&self) -> bool {
        self.waiting_count.load(Ordering::Relaxed) > 0
    }

    /// Mark the limiter as closed and wake all blocked producers.
    fn close(&self) {
        self.closed.store(true, Ordering::Release);
        self.cond.notify_all();
    }
}

/// RAII guard that releases memory back to the `MemoryLimiter` on drop.
pub(crate) struct MemoryPermit {
    limiter: Arc<MemoryLimiter>,
    size: usize,
}

impl std::fmt::Debug for MemoryPermit {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("MemoryPermit")
            .field("size", &self.size)
            .finish_non_exhaustive()
    }
}

impl Drop for MemoryPermit {
    fn drop(&mut self) {
        if self.size > 0 {
            self.limiter.release(self.size);
        }
    }
}

// Type alias to simplify complex nested types
type BucketBatches = Vec<(BucketId, Arc<Mutex<VecDeque<WriteBatch>>>)>;

#[allow(dead_code)]
pub struct RecordAccumulator {
    config: Config,
    write_batches: DashMap<Arc<PhysicalTablePath>, BucketAndWriteBatches>,
    // batch_id -> (complete callback, memory permit)
    incomplete_batches: RwLock<HashMap<i64, (ResultHandle, MemoryPermit)>>,
    batch_timeout_ms: i64,
    closed: AtomicBool,
    flushes_in_progress: AtomicI32,
    appends_in_progress: i32,
    nodes_drain_index: Mutex<HashMap<i32, usize>>,
    batch_id: AtomicI64,
    idempotence_manager: Arc<IdempotenceManager>,
    memory_limiter: Arc<MemoryLimiter>,
    /// Wakes the sender task when new batches are created or existing batches
    /// become full, so the sender can drain them immediately instead of waiting
    /// for its next poll cycle. This is the Rust equivalent of Java's
    /// `Sender.wakeup()` / Kafka's `RecordAccumulator.wakeup()`.
    sender_wakeup: Notify,
}

impl RecordAccumulator {
    pub fn new(config: Config, idempotence_manager: Arc<IdempotenceManager>) -> Self {
        let batch_timeout_ms = config.writer_batch_timeout_ms;
        let memory_limiter = Arc::new(MemoryLimiter::new(
            config.writer_buffer_memory_size,
            Duration::from_millis(config.writer_buffer_wait_timeout_ms),
        ));
        RecordAccumulator {
            config,
            write_batches: Default::default(),
            incomplete_batches: Default::default(),
            batch_timeout_ms,
            closed: Default::default(),
            flushes_in_progress: Default::default(),
            appends_in_progress: Default::default(),
            nodes_drain_index: Default::default(),
            batch_id: Default::default(),
            idempotence_manager,
            memory_limiter,
            sender_wakeup: Notify::new(),
        }
    }

    fn try_append(
        &self,
        record: &WriteRecord,
        dq: &mut VecDeque<WriteBatch>,
    ) -> Result<Option<RecordAppendResult>> {
        let dq_size = dq.len();
        if let Some(last_batch) = dq.back_mut() {
            return if let Some(result_handle) = last_batch.try_append(record)? {
                Ok(Some(RecordAppendResult::new(
                    result_handle,
                    dq_size > 1 || last_batch.is_closed(),
                    false,
                    false,
                )))
            } else {
                Ok(None)
            };
        }
        Ok(None)
    }

    fn append_new_batch(
        &self,
        cluster: &Cluster,
        record: &WriteRecord,
        dq: &mut VecDeque<WriteBatch>,
        permit: MemoryPermit,
        alloc_size: usize,
    ) -> Result<RecordAppendResult> {
        let physical_table_path = &record.physical_table_path;
        let table_path = physical_table_path.get_table_path();
        let table_info = cluster.get_table(table_path)?;
        let arrow_compression_info = table_info.get_table_config().get_arrow_compression_info()?;
        let row_type = &table_info.row_type;

        let schema_id = table_info.schema_id;

        let mut batch: WriteBatch = match record.record() {
            Record::Log(_) => ArrowLog(ArrowLogWriteBatch::new(
                self.batch_id.fetch_add(1, Ordering::Relaxed),
                Arc::clone(physical_table_path),
                schema_id,
                arrow_compression_info,
                row_type,
                current_time_ms(),
                matches!(&record.record, Record::Log(LogWriteRecord::RecordBatch(_))),
            )?),
            Record::Kv(kv_record) => Kv(KvWriteBatch::new(
                self.batch_id.fetch_add(1, Ordering::Relaxed),
                Arc::clone(physical_table_path),
                schema_id,
                alloc_size,
                record.write_format.to_kv_format()?,
                kv_record.target_columns.clone(),
                current_time_ms(),
            )),
        };

        let batch_id = batch.batch_id();

        let result_handle = batch
            .try_append(record)?
            .expect("must append to a new batch");

        let batch_is_closed = batch.is_closed();
        dq.push_back(batch);

        self.incomplete_batches
            .write()
            .insert(batch_id, (result_handle.clone(), permit));
        Ok(RecordAppendResult::new(
            result_handle,
            dq.len() > 1 || batch_is_closed,
            true,
            false,
        ))
    }

    pub fn append(
        &self,
        record: &WriteRecord<'_>,
        bucket_id: BucketId,
        cluster: &Cluster,
        abort_if_batch_full: bool,
    ) -> Result<RecordAppendResult> {
        let physical_table_path = &record.physical_table_path;
        let table_path = physical_table_path.get_table_path();
        let table_info = cluster.get_table(table_path)?;
        let is_partitioned_table = table_info.is_partitioned();

        let partition_id = if is_partitioned_table {
            cluster.get_partition_id(physical_table_path)
        } else {
            None
        };

        let dq = {
            let mut binding = self
                .write_batches
                .entry(Arc::clone(physical_table_path))
                .or_insert_with(|| BucketAndWriteBatches {
                    table_id: table_info.table_id,
                    is_partitioned_table,
                    partition_id,
                    batches: Default::default(),
                });
            let bucket_and_batches = binding.value_mut();
            bucket_and_batches
                .batches
                .entry(bucket_id)
                .or_insert_with(|| Arc::new(Mutex::new(VecDeque::new())))
                .clone()
        };

        let mut dq_guard = dq.lock();
        if let Some(append_result) = self.try_append(record, &mut dq_guard)? {
            return Ok(append_result);
        }

        if abort_if_batch_full {
            return Ok(RecordAppendResult::new_without_result_handle(
                true, false, true,
            ));
        }

        // Drop dq lock before blocking on memory to prevent deadlock:
        // producer holds dq + blocks on memory, while sender needs dq to drain.
        drop(dq_guard);

        let batch_size = self.config.writer_batch_size as usize;
        let record_size = record.estimated_record_size();
        let alloc_size = batch_size.max(record_size);
        let permit = self.memory_limiter.acquire(alloc_size)?;

        // Re-acquire dq lock after memory is available
        let mut dq_guard = dq.lock();
        // Re-try: another thread may have created a batch while we waited
        if let Some(append_result) = self.try_append(record, &mut dq_guard)? {
            return Ok(append_result); // permit drops here, memory released
        }

        self.append_new_batch(cluster, record, &mut dq_guard, permit, alloc_size)
    }

    pub fn ready(&self, cluster: &Arc<Cluster>) -> Result<ReadyCheckResult> {
        // Snapshot just the Arcs we need, avoiding cloning the entire BucketAndWriteBatches struct
        let entries: Vec<(Arc<PhysicalTablePath>, Option<PartitionId>, BucketBatches)> = self
            .write_batches
            .iter()
            .map(|entry| {
                let physical_table_path = Arc::clone(entry.key());
                let partition_id = entry.value().partition_id;
                let bucket_batches: Vec<_> = entry
                    .value()
                    .batches
                    .iter()
                    .map(|(bucket_id, batch_arc)| (*bucket_id, batch_arc.clone()))
                    .collect();
                (physical_table_path, partition_id, bucket_batches)
            })
            .collect();

        let mut ready_nodes = HashSet::new();
        let mut next_ready_check_delay_ms = self.batch_timeout_ms;
        let mut unknown_leader_tables = HashSet::new();
        let exhausted = self.memory_limiter.has_waiters();

        for (physical_table_path, mut partition_id, bucket_batches) in entries {
            next_ready_check_delay_ms = self.bucket_ready(
                &physical_table_path,
                physical_table_path.get_partition_name().is_some(),
                &mut partition_id,
                bucket_batches,
                &mut ready_nodes,
                &mut unknown_leader_tables,
                cluster,
                next_ready_check_delay_ms,
                exhausted,
            )?
        }

        Ok(ReadyCheckResult {
            ready_nodes,
            next_ready_check_delay_ms,
            unknown_leader_tables,
        })
    }

    #[allow(clippy::too_many_arguments)]
    fn bucket_ready(
        &self,
        physical_table_path: &Arc<PhysicalTablePath>,
        is_partitioned_table: bool,
        partition_id: &mut Option<PartitionId>,
        bucket_batches: BucketBatches,
        ready_nodes: &mut HashSet<ServerNode>,
        unknown_leader_tables: &mut HashSet<Arc<PhysicalTablePath>>,
        cluster: &Cluster,
        next_ready_check_delay_ms: i64,
        exhausted: bool,
    ) -> Result<i64> {
        let mut next_delay = next_ready_check_delay_ms;

        // First check this table has partitionId.
        if is_partitioned_table && partition_id.is_none() {
            let partition_id = cluster.get_partition_id(physical_table_path);

            if partition_id.is_some() {
                // Update the cached partition_id
                if let Some(mut entry) = self.write_batches.get_mut(physical_table_path) {
                    entry.partition_id = partition_id;
                }
            } else {
                log::debug!(
                    "Partition does not exist for {}, bucket will not be set to ready",
                    physical_table_path.as_ref()
                );

                // TODO: we shouldn't add unready partitions to unknownLeaderTables,
                // because it cases PartitionNotExistException later
                unknown_leader_tables.insert(Arc::clone(physical_table_path));
                return Ok(next_delay);
            }
        }

        for (bucket_id, batch) in bucket_batches {
            let batch_guard = batch.lock();
            if batch_guard.is_empty() {
                continue;
            }

            let batch = batch_guard.front().unwrap();
            let waited_time_ms = batch.waited_time_ms(current_time_ms());
            let deque_size = batch_guard.len();
            let full = deque_size > 1 || batch.is_closed();
            let table_bucket = cluster.get_table_bucket(physical_table_path, bucket_id)?;
            if let Some(leader) = cluster.leader_for(&table_bucket) {
                next_delay = self.batch_ready(
                    leader,
                    waited_time_ms,
                    full,
                    exhausted,
                    ready_nodes,
                    next_delay,
                );
            } else {
                unknown_leader_tables.insert(Arc::clone(physical_table_path));
            }
        }
        Ok(next_delay)
    }

    fn batch_ready(
        &self,
        leader: &ServerNode,
        waited_time_ms: i64,
        full: bool,
        exhausted: bool,
        ready_nodes: &mut HashSet<ServerNode>,
        next_ready_check_delay_ms: i64,
    ) -> i64 {
        if !ready_nodes.contains(leader) {
            let expired = waited_time_ms >= self.batch_timeout_ms;
            let sendable = full
                || expired
                || exhausted
                || self.closed.load(Ordering::Acquire)
                || self.flush_in_progress();

            if sendable {
                ready_nodes.insert(leader.clone());
            } else {
                let time_left_ms = self.batch_timeout_ms.saturating_sub(waited_time_ms);
                return next_ready_check_delay_ms.min(time_left_ms);
            }
        }
        next_ready_check_delay_ms
    }

    pub fn drain(
        &self,
        cluster: Arc<Cluster>,
        nodes: &HashSet<ServerNode>,
        max_size: i32,
    ) -> Result<HashMap<i32, Vec<ReadyWriteBatch>>> {
        if nodes.is_empty() {
            return Ok(HashMap::new());
        }
        let mut batches = HashMap::new();
        for node in nodes {
            let ready = self.drain_batches_for_one_node(&cluster, node, max_size)?;
            if !ready.is_empty() {
                batches.insert(node.id(), ready);
            }
        }

        Ok(batches)
    }

    /// Matches Java's `shouldStopDrainBatchesForBucket`. Returns true if
    /// this bucket should be skipped during drain.
    fn should_stop_drain_batches_for_bucket(
        &self,
        first: &WriteBatch,
        table_bucket: &TableBucket,
    ) -> bool {
        if !self.idempotence_manager.is_enabled() {
            return false;
        }
        if !self.idempotence_manager.is_writer_id_valid() {
            return true;
        }

        // Use batch_id comparison instead of sequence comparison. After
        // handle_failed_batch adjusts InFlightBatch sequences, the WriteBatch's
        // stored sequence may be stale (re_enqueue syncs it, but this is more
        // robust). Java can compare sequences because resetWriterState mutates
        // the batch directly; Rust uses lightweight InFlightBatch proxies.
        let is_first_in_flight = self.idempotence_manager.in_flight_count(table_bucket) == 0
            || (first.has_batch_sequence()
                && self
                    .idempotence_manager
                    .is_first_in_flight_batch(table_bucket, first.batch_id()));

        if is_first_in_flight {
            return false;
        }

        if !first.has_batch_sequence() {
            // Fresh batch: respect max in-flight limit
            !self
                .idempotence_manager
                .can_send_more_requests(table_bucket)
        } else {
            // Re-enqueued batch that's NOT first in-flight: stop
            true
        }
    }

    fn drain_batches_for_one_node(
        &self,
        cluster: &Cluster,
        node: &ServerNode,
        max_size: i32,
    ) -> Result<Vec<ReadyWriteBatch>> {
        let mut size: usize = 0;
        let buckets = self.get_all_buckets_in_current_node(node, cluster);
        let mut ready = Vec::new();

        if buckets.is_empty() {
            return Ok(ready);
        }

        let start = {
            let mut nodes_drain_index_guard = self.nodes_drain_index.lock();
            let drain_index = nodes_drain_index_guard.entry(node.id()).or_insert(0);
            *drain_index % buckets.len()
        };

        let mut current_index = start;
        let mut last_processed_index;

        loop {
            let bucket = &buckets[current_index];
            let table_path = bucket.physical_table_path();
            let table_bucket = bucket.table_bucket.clone();
            last_processed_index = current_index;
            current_index = (current_index + 1) % buckets.len();

            let deque = self
                .write_batches
                .get(table_path)
                .and_then(|bucket_and_write_batches| {
                    bucket_and_write_batches
                        .batches
                        .get(&table_bucket.bucket_id())
                        .cloned()
                });

            if let Some(deque) = deque {
                let mut maybe_batch = None;
                {
                    let mut batch_lock = deque.lock();
                    if !batch_lock.is_empty() {
                        let first_batch = batch_lock.front().unwrap();

                        if size + first_batch.estimated_size_in_bytes() > max_size as usize
                            && !ready.is_empty()
                        {
                            // there is a rare case that a single batch size is larger than the request size
                            // due to compression; in this case we will still eventually send this batch in
                            // a single request.
                            break;
                        }

                        // Improvement: `continue` instead of `break` to skip
                        // only this bucket, not all buckets for the node.
                        if self.should_stop_drain_batches_for_bucket(first_batch, &table_bucket) {
                            if current_index == start {
                                break;
                            }
                            continue;
                        }

                        maybe_batch = Some(batch_lock.pop_front().unwrap());
                    }
                }

                if let Some(ref mut batch) = maybe_batch {
                    // Assign writer state to fresh batches (matching Java's drain loop)
                    let writer_id = if self.idempotence_manager.is_enabled() {
                        self.idempotence_manager.writer_id()
                    } else {
                        NO_WRITER_ID
                    };
                    if writer_id != NO_WRITER_ID && !batch.has_batch_sequence() {
                        self.idempotence_manager
                            .maybe_update_writer_id(&table_bucket);
                        let seq = self
                            .idempotence_manager
                            .next_sequence_and_increment(&table_bucket);
                        batch.set_writer_state(writer_id, seq);
                        self.idempotence_manager.add_in_flight_batch(
                            &table_bucket,
                            seq,
                            batch.batch_id(),
                        );
                    }
                }

                if let Some(mut batch) = maybe_batch {
                    let current_batch_size = batch.estimated_size_in_bytes();
                    size += current_batch_size;

                    // mark the batch as drained.
                    batch.drained(current_time_ms());
                    ready.push(ReadyWriteBatch {
                        table_bucket,
                        write_batch: batch,
                    });
                }
            }
            if current_index == start {
                break;
            }
        }

        // Store the last processed index to maintain round-robin fairness
        {
            let mut nodes_drain_index_guard = self.nodes_drain_index.lock();
            nodes_drain_index_guard.insert(node.id(), last_processed_index);
        }

        Ok(ready)
    }

    pub fn remove_incomplete_batches(&self, batch_id: i64) {
        self.incomplete_batches.write().remove(&batch_id);
    }

    pub fn re_enqueue(&self, mut ready_write_batch: ReadyWriteBatch) {
        ready_write_batch.write_batch.re_enqueued();

        // Sync WriteBatch sequence with IdempotenceManager's adjusted sequence.
        // When handle_failed_batch adjusts InFlightBatch sequences (after a prior
        // batch fails), the WriteBatch is not updated (unlike Java which calls
        // resetWriterState on the actual batch). We must sync here so that:
        // 1. should_stop_drain_batches_for_bucket comparisons work correctly
        // 2. build() produces bytes with the correct (adjusted) sequence
        if self.idempotence_manager.is_enabled()
            && ready_write_batch.write_batch.has_batch_sequence()
        {
            if let Some(adjusted_seq) = self.idempotence_manager.get_adjusted_sequence(
                &ready_write_batch.table_bucket,
                ready_write_batch.write_batch.batch_id(),
            ) {
                if adjusted_seq != ready_write_batch.write_batch.batch_sequence() {
                    let writer_id = ready_write_batch.write_batch.writer_id();
                    ready_write_batch
                        .write_batch
                        .set_writer_state(writer_id, adjusted_seq);
                }
            }
        }

        let dq = self.get_or_create_deque(&ready_write_batch);
        let mut dq_guard = dq.lock();
        if self.idempotence_manager.is_enabled() {
            self.insert_in_sequence_order(&mut dq_guard, ready_write_batch);
        } else {
            dq_guard.push_front(ready_write_batch.write_batch);
        }
    }

    /// Insert a re-enqueued batch in sequence order. Matches Java's
    /// `insertInSequenceOrder`. If the batch is the next expected in-flight,
    /// push to front; otherwise, find the correct sorted position.
    fn insert_in_sequence_order(
        &self,
        dq: &mut VecDeque<WriteBatch>,
        ready_write_batch: ReadyWriteBatch,
    ) {
        debug_assert!(
            ready_write_batch.write_batch.batch_sequence() != NO_BATCH_SEQUENCE,
            "Re-enqueuing a batch without a sequence (batch_id={})",
            ready_write_batch.write_batch.batch_id()
        );
        debug_assert!(
            self.idempotence_manager
                .in_flight_count(&ready_write_batch.table_bucket)
                > 0,
            "Re-enqueuing a batch not tracked in in-flight (batch_id={}, bucket={})",
            ready_write_batch.write_batch.batch_id(),
            ready_write_batch.table_bucket
        );

        if dq.is_empty() {
            dq.push_front(ready_write_batch.write_batch);
            return;
        }

        // If it's the first in-flight batch for its bucket, push to front
        if self.idempotence_manager.is_first_in_flight_batch(
            &ready_write_batch.table_bucket,
            ready_write_batch.write_batch.batch_id(),
        ) {
            dq.push_front(ready_write_batch.write_batch);
            return;
        }

        // Find the correct position sorted by batch_sequence
        let batch_seq = ready_write_batch.write_batch.batch_sequence();
        let mut insert_pos = dq.len();
        for (i, existing) in dq.iter().enumerate() {
            if existing.has_batch_sequence() && existing.batch_sequence() > batch_seq {
                insert_pos = i;
                break;
            }
        }
        dq.insert(insert_pos, ready_write_batch.write_batch);
    }

    fn get_or_create_deque(
        &self,
        ready_write_batch: &ReadyWriteBatch,
    ) -> Arc<Mutex<VecDeque<WriteBatch>>> {
        let physical_table_path = ready_write_batch.write_batch.physical_table_path();
        let bucket_id = ready_write_batch.table_bucket.bucket_id();
        let table_id = ready_write_batch.table_bucket.table_id();
        let partition_id = ready_write_batch.table_bucket.partition_id();
        let is_partitioned_table = partition_id.is_some();

        let mut binding = self
            .write_batches
            .entry(Arc::clone(physical_table_path))
            .or_insert_with(|| BucketAndWriteBatches {
                table_id,
                is_partitioned_table,
                partition_id,
                batches: Default::default(),
            });
        let bucket_and_batches = binding.value_mut();
        bucket_and_batches
            .batches
            .entry(bucket_id)
            .or_insert_with(|| Arc::new(Mutex::new(VecDeque::new())))
            .clone()
    }

    /// Mark the accumulator as closed. All batches become immediately ready
    /// (sendable) in `batch_ready`, triggering a full drain without waiting
    /// for `batch_timeout_ms`. Matches Java's `RecordAccumulator.close()`.
    pub fn close(&self) {
        self.closed.store(true, Ordering::Release);
        self.wakeup_sender();
    }

    pub fn is_closed(&self) -> bool {
        self.closed.load(Ordering::Acquire)
    }

    pub fn abort_batches(&self, error: broadcast::Error) {
        self.memory_limiter.close();
        // Complete batches still in deques (not yet drained).
        for mut entry in self.write_batches.iter_mut() {
            for (_bucket_id, deque) in entry.value_mut().batches.iter_mut() {
                let mut dq = deque.lock();
                while let Some(batch) = dq.pop_front() {
                    batch.complete(Err(error.clone()));
                }
            }
        }
        // Fail any remaining handles (including in-flight batches that were
        // drained but not yet completed). This is a no-op for handles already
        // completed above via WriteBatch::complete.
        let mut incomplete = self.incomplete_batches.write();
        for (handle, _permit) in incomplete.values() {
            handle.fail(error.clone());
        }
        incomplete.clear();
    }

    pub fn has_incomplete(&self) -> bool {
        !self.incomplete_batches.read().is_empty()
    }

    /// Wake the sender task so it can drain ready batches immediately.
    pub fn wakeup_sender(&self) {
        self.sender_wakeup.notify_one();
    }

    /// Returns a future that completes when `wakeup_sender()` is called.
    pub fn notified(&self) -> tokio::sync::futures::Notified<'_> {
        self.sender_wakeup.notified()
    }

    fn get_all_buckets_in_current_node(
        &self,
        current: &ServerNode,
        cluster: &Cluster,
    ) -> Vec<BucketLocation> {
        let mut buckets = vec![];
        for bucket_locations in cluster.get_bucket_locations_by_path().values() {
            for bucket_location in bucket_locations {
                if let Some(leader) = bucket_location.leader() {
                    if current.id() == leader.id() {
                        buckets.push(bucket_location.clone());
                    }
                }
            }
        }
        buckets
    }

    pub fn has_undrained(&self) -> bool {
        for entry in self.write_batches.iter() {
            for (_, batch_deque) in entry.value().batches.iter() {
                if !batch_deque.lock().is_empty() {
                    return true;
                }
            }
        }
        false
    }

    pub fn get_physical_table_paths_in_batches(&self) -> Vec<Arc<PhysicalTablePath>> {
        self.write_batches
            .iter()
            .map(|entry| Arc::clone(entry.key()))
            .collect()
    }

    fn flush_in_progress(&self) -> bool {
        self.flushes_in_progress.load(Ordering::SeqCst) > 0
    }

    pub fn begin_flush(&self) {
        self.flushes_in_progress.fetch_add(1, Ordering::SeqCst);
        self.wakeup_sender();
    }

    #[allow(unused_must_use)]
    pub async fn await_flush_completion(&self) -> Result<()> {
        // Clone handles before awaiting to avoid holding RwLock read guard across await points
        let handles: Vec<_> = self
            .incomplete_batches
            .read()
            .values()
            .map(|(h, _)| h.clone())
            .collect();

        // Await on all handles
        let result = async {
            for result_handle in handles {
                result_handle.wait().await?;
            }
            Ok(())
        }
        .await;

        // Always decrement flushes_in_progress, even if an error occurred
        // This mimics the Java finally block behavior
        self.flushes_in_progress.fetch_sub(1, Ordering::SeqCst);

        result
    }
}

pub struct ReadyWriteBatch {
    pub table_bucket: TableBucket,
    pub write_batch: WriteBatch,
}

impl ReadyWriteBatch {
    pub fn write_batch(&self) -> &WriteBatch {
        &self.write_batch
    }
}

#[allow(dead_code)]
struct BucketAndWriteBatches {
    table_id: TableId,
    is_partitioned_table: bool,
    partition_id: Option<PartitionId>,
    batches: HashMap<BucketId, Arc<Mutex<VecDeque<WriteBatch>>>>,
}

pub struct RecordAppendResult {
    pub batch_is_full: bool,
    pub new_batch_created: bool,
    pub abort_record_for_new_batch: bool,
    pub result_handle: Option<ResultHandle>,
}

impl RecordAppendResult {
    fn new(
        result_handle: ResultHandle,
        batch_is_full: bool,
        new_batch_created: bool,
        abort_record_for_new_batch: bool,
    ) -> Self {
        Self {
            batch_is_full,
            new_batch_created,
            abort_record_for_new_batch,
            result_handle: Some(result_handle),
        }
    }

    fn new_without_result_handle(
        batch_is_full: bool,
        new_batch_created: bool,
        abort_record_for_new_batch: bool,
    ) -> Self {
        Self {
            batch_is_full,
            new_batch_created,
            abort_record_for_new_batch,
            result_handle: None,
        }
    }
}

pub struct ReadyCheckResult {
    pub ready_nodes: HashSet<ServerNode>,
    pub next_ready_check_delay_ms: i64,
    pub unknown_leader_tables: HashSet<Arc<PhysicalTablePath>>,
}

impl ReadyCheckResult {
    pub fn new(
        ready_nodes: HashSet<ServerNode>,
        next_ready_check_delay_ms: i64,
        unknown_leader_tables: HashSet<Arc<PhysicalTablePath>>,
    ) -> Self {
        ReadyCheckResult {
            ready_nodes,
            next_ready_check_delay_ms,
            unknown_leader_tables,
        }
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::metadata::TablePath;
    use crate::row::{Datum, GenericRow};
    use crate::test_utils::{build_cluster, build_table_info};
    use std::sync::Arc;

    fn disabled_idempotence() -> Arc<IdempotenceManager> {
        Arc::new(IdempotenceManager::new(false, 5))
    }

    fn enabled_idempotence() -> Arc<IdempotenceManager> {
        Arc::new(IdempotenceManager::new(true, 5))
    }

    #[tokio::test]
    async fn re_enqueue_increments_attempts() -> Result<()> {
        let config = Config::default();
        let accumulator = RecordAccumulator::new(config, disabled_idempotence());
        let table_path = TablePath::new("db".to_string(), "tbl".to_string());
        let physical_table_path = Arc::new(PhysicalTablePath::of(Arc::new(table_path.clone())));
        let table_info = Arc::new(build_table_info(table_path.clone(), 1, 1));
        let cluster = Arc::new(build_cluster(&table_path, 1, 1));
        let row = GenericRow {
            values: vec![Datum::Int32(1)],
        };
        let record = WriteRecord::for_append(table_info, physical_table_path, 1, &row);

        accumulator.append(&record, 0, &cluster, false)?;

        let server = cluster.get_tablet_server(1).expect("server");
        let nodes = HashSet::from([server.clone()]);
        let mut batches = accumulator.drain(cluster.clone(), &nodes, 1024 * 1024)?;
        let mut drained = batches.remove(&1).expect("drained batches");
        let batch = drained.pop().expect("batch");
        assert_eq!(batch.write_batch.attempts(), 0);

        accumulator.re_enqueue(batch);

        let mut batches = accumulator.drain(cluster, &nodes, 1024 * 1024)?;
        let mut drained = batches.remove(&1).expect("drained batches");
        let batch = drained.pop().expect("batch");
        assert_eq!(batch.write_batch.attempts(), 1);
        Ok(())
    }

    #[tokio::test]
    async fn flush_counter_decremented_on_error() -> Result<()> {
        use crate::client::write::broadcast::BroadcastOnce;
        use std::sync::atomic::Ordering;

        let config = Config::default();
        let accumulator = RecordAccumulator::new(config, disabled_idempotence());

        accumulator.begin_flush();
        assert_eq!(accumulator.flushes_in_progress.load(Ordering::SeqCst), 1);

        // Create a failing batch by dropping the BroadcastOnce without broadcasting
        {
            let broadcast = BroadcastOnce::default();
            let receiver = broadcast.receiver();
            let handle = ResultHandle::new(receiver);
            let permit = accumulator.memory_limiter.acquire(1024).unwrap();
            accumulator
                .incomplete_batches
                .write()
                .insert(1, (handle, permit));
            // broadcast is dropped here, causing an error
        }

        // Await flush completion should fail but still decrement counter
        let result = accumulator.await_flush_completion().await;
        assert!(result.is_err());

        // Counter should still be decremented (this is the critical fix!)
        assert_eq!(accumulator.flushes_in_progress.load(Ordering::SeqCst), 0);
        assert!(!accumulator.flush_in_progress());

        Ok(())
    }

    fn append_and_drain(
        accumulator: &RecordAccumulator,
        cluster: &Arc<crate::cluster::Cluster>,
        table_path: &TablePath,
        bucket_id: i32,
    ) -> Result<ReadyWriteBatch> {
        let table_info = Arc::new(build_table_info(table_path.clone(), 1, 2));
        let physical_table_path = Arc::new(PhysicalTablePath::of(Arc::new(table_path.clone())));
        let row = GenericRow {
            values: vec![Datum::Int32(1)],
        };
        let record = WriteRecord::for_append(table_info, physical_table_path, 1, &row);
        accumulator.append(&record, bucket_id, cluster, false)?;
        let server = cluster.get_tablet_server(1).expect("server");
        let nodes = HashSet::from([server.clone()]);
        let mut batches = accumulator.drain(cluster.clone(), &nodes, 1024 * 1024)?;
        let mut drained = batches.remove(&1).expect("drained batches");
        Ok(drained.pop().expect("batch"))
    }

    #[test]
    fn test_should_stop_drain_for_fresh_batch_over_limit() {
        let idempotence = Arc::new(IdempotenceManager::new(true, 2));
        idempotence.set_writer_id(42);
        let config = Config::default();
        let accumulator = RecordAccumulator::new(config, Arc::clone(&idempotence));
        let table_path = TablePath::new("db".to_string(), "tbl".to_string());
        let cluster = Arc::new(build_cluster(&table_path, 1, 1));
        let table_info = Arc::new(build_table_info(table_path.clone(), 1, 1));
        let physical_table_path = Arc::new(PhysicalTablePath::of(Arc::new(table_path.clone())));
        let row = GenericRow {
            values: vec![Datum::Int32(1)],
        };
        let record = WriteRecord::for_append(table_info, physical_table_path, 1, &row);
        accumulator
            .append(&record, 0, &cluster, false)
            .expect("append");

        let table_bucket = TableBucket::new(1, 0);

        // Add 2 in-flight batches (reaching the max_in_flight=2)
        idempotence.add_in_flight_batch(&table_bucket, 0, 100);
        idempotence.add_in_flight_batch(&table_bucket, 1, 101);

        // Get the front batch from the deque
        let entry = accumulator
            .write_batches
            .get(&PhysicalTablePath::of(Arc::new(table_path)))
            .unwrap();
        let dq = entry.batches.get(&0).unwrap();
        let dq_guard = dq.lock();
        let first_batch = dq_guard.front().unwrap();

        // Fresh batch (no batch_sequence) with in-flight at limit → should stop
        assert!(!first_batch.has_batch_sequence());
        assert!(accumulator.should_stop_drain_batches_for_bucket(first_batch, &table_bucket));

        // Remove one in-flight → under limit → should not stop
        drop(dq_guard);
        idempotence.remove_in_flight_batch(&table_bucket, 101);
        let dq_guard = entry.batches.get(&0).unwrap().lock();
        let first_batch = dq_guard.front().unwrap();
        assert!(!accumulator.should_stop_drain_batches_for_bucket(first_batch, &table_bucket));
    }

    #[test]
    fn test_should_stop_drain_for_retry_not_first_inflight() {
        let idempotence = enabled_idempotence();
        idempotence.set_writer_id(42);
        let config = Config::default();
        let accumulator = RecordAccumulator::new(config, Arc::clone(&idempotence));
        let table_path = TablePath::new("db".to_string(), "tbl".to_string());
        let cluster = Arc::new(build_cluster(&table_path, 1, 1));

        // Drain two separate batches to get batch0(seq=0) and batch1(seq=1)
        let batch0 =
            append_and_drain(&accumulator, &cluster, &table_path, 0).expect("drain batch0");
        let batch1 =
            append_and_drain(&accumulator, &cluster, &table_path, 0).expect("drain batch1");

        assert_eq!(batch0.write_batch.batch_sequence(), 0);
        assert_eq!(batch1.write_batch.batch_sequence(), 1);

        let batch1_id = batch1.write_batch.batch_id();
        let table_bucket = batch0.table_bucket.clone();

        // Re-enqueue only batch1 (simulating batch0 still in-flight, batch1 got error)
        accumulator.re_enqueue(batch1);

        let entry = accumulator
            .write_batches
            .get(&PhysicalTablePath::of(Arc::new(table_path)))
            .unwrap();
        let dq = entry.batches.get(&0).unwrap();
        let dq_guard = dq.lock();
        let first_batch = dq_guard.front().unwrap();

        // Batch1 is re-enqueued with seq=1, but batch0 (seq=0) is the first in-flight.
        // batch1's batch_id != first in-flight batch_id → should stop.
        assert!(first_batch.has_batch_sequence());
        assert_eq!(first_batch.batch_id(), batch1_id);
        assert!(accumulator.should_stop_drain_batches_for_bucket(first_batch, &table_bucket));
    }

    #[tokio::test]
    async fn test_insert_in_sequence_order() -> Result<()> {
        let idempotence = enabled_idempotence();
        idempotence.set_writer_id(42);
        let config = Config::default();
        let accumulator = RecordAccumulator::new(config, Arc::clone(&idempotence));
        let table_path = TablePath::new("db".to_string(), "tbl".to_string());
        let cluster = Arc::new(build_cluster(&table_path, 1, 2));

        // Create and drain 3 batches to get them with sequences 0, 1, 2
        let batch0 = append_and_drain(&accumulator, &cluster, &table_path, 0)?;
        let batch1 = append_and_drain(&accumulator, &cluster, &table_path, 0)?;
        let batch2 = append_and_drain(&accumulator, &cluster, &table_path, 0)?;

        assert_eq!(batch0.write_batch.batch_sequence(), 0);
        assert_eq!(batch1.write_batch.batch_sequence(), 1);
        assert_eq!(batch2.write_batch.batch_sequence(), 2);

        let batch0_id = batch0.write_batch.batch_id();
        let batch1_id = batch1.write_batch.batch_id();
        let batch2_id = batch2.write_batch.batch_id();
        let table_bucket = batch0.table_bucket.clone();

        // Re-enqueue in reverse order: 2, 0, 1
        // insert_in_sequence_order should sort them as: 0, 1, 2
        accumulator.re_enqueue(batch2);
        accumulator.re_enqueue(batch0);
        accumulator.re_enqueue(batch1);

        // Verify the deque order directly
        let entry = accumulator
            .write_batches
            .get(&PhysicalTablePath::of(Arc::new(table_path)))
            .unwrap();
        let dq = entry.batches.get(&0).unwrap();
        let dq_guard = dq.lock();
        assert_eq!(dq_guard.len(), 3);
        // batch0 (seq=0) is the first in-flight, so it should be at front
        assert_eq!(dq_guard[0].batch_id(), batch0_id);
        assert_eq!(dq_guard[0].batch_sequence(), 0);
        assert_eq!(dq_guard[1].batch_id(), batch1_id);
        assert_eq!(dq_guard[1].batch_sequence(), 1);
        assert_eq!(dq_guard[2].batch_id(), batch2_id);
        assert_eq!(dq_guard[2].batch_sequence(), 2);
        drop(dq_guard);

        // Drain: first in-flight is seq=0, so batch0 passes should_stop check
        let server = cluster.get_tablet_server(1).expect("server");
        let nodes = HashSet::from([server.clone()]);
        let mut batches = accumulator.drain(cluster.clone(), &nodes, 1024 * 1024)?;
        let drained = batches.remove(&1).expect("drained batches");
        assert_eq!(drained.len(), 1);
        assert_eq!(drained[0].write_batch.batch_sequence(), 0);

        // Complete batch0 so batch1 becomes first in-flight
        idempotence.handle_completed_batch(&table_bucket, batch0_id, 42);

        let mut batches = accumulator.drain(cluster.clone(), &nodes, 1024 * 1024)?;
        let drained = batches.remove(&1).expect("drained");
        assert_eq!(drained[0].write_batch.batch_sequence(), 1);

        idempotence.handle_completed_batch(&table_bucket, batch1_id, 42);

        let mut batches = accumulator.drain(cluster, &nodes, 1024 * 1024)?;
        let drained = batches.remove(&1).expect("drained");
        assert_eq!(drained[0].write_batch.batch_sequence(), 2);

        Ok(())
    }

    #[tokio::test]
    async fn test_abort_batches() -> Result<()> {
        let idempotence = disabled_idempotence();
        let config = Config::default();
        let accumulator = RecordAccumulator::new(config, Arc::clone(&idempotence));
        let table_path = TablePath::new("db".to_string(), "tbl".to_string());
        let physical_table_path = Arc::new(PhysicalTablePath::of(Arc::new(table_path.clone())));
        let table_info = Arc::new(build_table_info(table_path.clone(), 1, 1));
        let cluster = Arc::new(build_cluster(&table_path, 1, 1));
        let row = GenericRow {
            values: vec![Datum::Int32(1)],
        };
        let record = WriteRecord::for_append(table_info, physical_table_path, 1, &row);

        let result = accumulator.append(&record, 0, &cluster, false)?;
        let handle = result.result_handle.expect("handle");
        assert!(accumulator.has_incomplete());

        accumulator.abort_batches(broadcast::Error::Client {
            message: "test abort".to_string(),
        });

        assert!(!accumulator.has_incomplete());
        assert!(!accumulator.has_undrained());

        // The handle should receive the error
        let batch_result = handle.wait().await?;
        assert!(matches!(
            batch_result,
            Err(broadcast::Error::Client { message }) if message == "test abort"
        ));
        Ok(())
    }

    #[tokio::test]
    async fn test_drain_skips_blocked_bucket_continues_others() -> Result<()> {
        // Use max_in_flight=1 so that one in-flight batch blocks further draining
        let idempotence = Arc::new(IdempotenceManager::new(true, 1));
        idempotence.set_writer_id(42);
        let config = Config::default();
        let accumulator = RecordAccumulator::new(config, Arc::clone(&idempotence));
        let table_path = TablePath::new("db".to_string(), "tbl".to_string());
        let cluster = Arc::new(build_cluster(&table_path, 1, 2));

        // Append to both buckets
        let table_info = Arc::new(build_table_info(table_path.clone(), 1, 2));
        let physical_table_path = Arc::new(PhysicalTablePath::of(Arc::new(table_path.clone())));
        let row = GenericRow {
            values: vec![Datum::Int32(1)],
        };

        // Append to bucket 0
        let record =
            WriteRecord::for_append(table_info.clone(), physical_table_path.clone(), 1, &row);
        accumulator.append(&record, 0, &cluster, false)?;

        // Append to bucket 1
        let record =
            WriteRecord::for_append(table_info.clone(), physical_table_path.clone(), 1, &row);
        accumulator.append(&record, 1, &cluster, false)?;

        // Drain once — both buckets get batches assigned with sequences
        let server = cluster.get_tablet_server(1).expect("server");
        let nodes = HashSet::from([server.clone()]);
        let batches = accumulator.drain(cluster.clone(), &nodes, 1024 * 1024)?;
        let drained = batches.get(&1).expect("drained");
        // Both buckets should produce batches
        assert_eq!(drained.len(), 2);

        // Now: both buckets have 1 in-flight each (added during drain).
        // Append another record to each bucket.
        let record =
            WriteRecord::for_append(table_info.clone(), physical_table_path.clone(), 1, &row);
        accumulator.append(&record, 0, &cluster, false)?;
        let record = WriteRecord::for_append(table_info, physical_table_path, 1, &row);
        accumulator.append(&record, 1, &cluster, false)?;

        // With max_in_flight=1, both buckets are at limit → should_stop returns true
        // for fresh batches. The drain should skip both (continue, not break).
        let batches2 = accumulator.drain(cluster.clone(), &nodes, 1024 * 1024)?;
        // No batches should be drained (both blocked)
        assert!(
            batches2.is_empty() || batches2.get(&1).is_none_or(|b| b.is_empty()),
            "Expected no batches when all buckets are blocked"
        );

        // Complete the in-flight for bucket 0
        let bucket0_batch = &drained[0];
        idempotence.handle_completed_batch(
            &bucket0_batch.table_bucket,
            bucket0_batch.write_batch.batch_id(),
            42,
        );

        // Now bucket 0 is unblocked but bucket 1 is still blocked
        let batches3 = accumulator.drain(cluster, &nodes, 1024 * 1024)?;
        let drained3 = batches3.get(&1).expect("some drained");
        // Only bucket 0 should produce a batch (continue skipped bucket 1)
        assert_eq!(drained3.len(), 1);
        assert_eq!(drained3[0].table_bucket.bucket_id(), 0);

        Ok(())
    }

    #[test]
    fn test_memory_limiter_acquire_release() {
        let limiter = Arc::new(MemoryLimiter::new(1024, Duration::from_secs(1)));

        let permit1 = limiter.acquire(512).unwrap();
        let permit2 = limiter.acquire(512).unwrap();

        // At capacity — verify used is 1024
        assert_eq!(*limiter.state.lock(), 1024);

        // Release one permit, verify used drops
        drop(permit1);
        assert_eq!(*limiter.state.lock(), 512);

        drop(permit2);
        assert_eq!(*limiter.state.lock(), 0);
    }

    #[test]
    fn test_memory_limiter_oversized_batch_fails_immediately() {
        let limiter = Arc::new(MemoryLimiter::new(1024, Duration::from_secs(60)));

        let result = limiter.acquire(2048);
        assert!(matches!(result.unwrap_err(), Error::IllegalArgument { .. }));
    }

    #[test]
    fn test_memory_limiter_blocks_then_unblocks() {
        let limiter = Arc::new(MemoryLimiter::new(1024, Duration::from_secs(5)));

        let permit = limiter.acquire(1024).unwrap();
        assert_eq!(*limiter.state.lock(), 1024);

        // Spawn a thread that tries to acquire — it should block
        let limiter2 = Arc::clone(&limiter);
        let handle = std::thread::spawn(move || limiter2.acquire(512));

        // Give the thread time to block
        std::thread::sleep(Duration::from_millis(50));
        // Still at capacity (thread is blocked)
        assert_eq!(*limiter.state.lock(), 1024);

        // Release the permit — thread should unblock
        drop(permit);

        let result = handle.join().unwrap();
        assert!(result.is_ok());
        let _permit2 = result.unwrap();
        assert_eq!(*limiter.state.lock(), 512);
    }

    #[test]
    fn test_memory_limiter_timeout() {
        let limiter = Arc::new(MemoryLimiter::new(1024, Duration::from_millis(100)));

        let _permit = limiter.acquire(1024).unwrap();

        // This should timeout
        let start = Instant::now();
        let result = limiter.acquire(512);
        let elapsed = start.elapsed();

        assert!(matches!(result.unwrap_err(), Error::BufferExhausted { .. }));
        assert!(elapsed >= Duration::from_millis(80)); // allow some timing slack
    }

    #[test]
    fn test_memory_limiter_close_fails_immediately() {
        let limiter = Arc::new(MemoryLimiter::new(1024, Duration::from_secs(60)));

        let _permit = limiter.acquire(512).unwrap();

        limiter.close();

        // New acquire should fail immediately, not block for 60s
        let start = Instant::now();
        let result = limiter.acquire(256);
        let elapsed = start.elapsed();

        assert!(matches!(result.unwrap_err(), Error::WriterClosed { .. }));
        assert!(elapsed < Duration::from_millis(50));
    }

    #[test]
    fn test_memory_limiter_close_unblocks_waiting_threads() {
        let limiter = Arc::new(MemoryLimiter::new(1024, Duration::from_secs(60)));

        // Fill the limiter completely
        let _permit = limiter.acquire(1024).unwrap();

        // Spawn a thread that blocks waiting for memory
        let limiter2 = Arc::clone(&limiter);
        let handle = std::thread::spawn(move || {
            let start = Instant::now();
            let result = limiter2.acquire(512);
            (result, start.elapsed())
        });

        // Give the thread time to block
        std::thread::sleep(Duration::from_millis(50));
        assert_eq!(limiter.waiting_count.load(Ordering::Relaxed), 1);

        // Close the limiter — should unblock the waiting thread
        limiter.close();

        let (result, elapsed) = handle.join().unwrap();
        assert!(matches!(result.unwrap_err(), Error::WriterClosed { .. }));
        assert!(elapsed < Duration::from_secs(5)); // should not wait the full 60s
    }

    #[test]
    fn test_oversized_kv_record_does_not_panic() {
        use crate::client::write::write_format::WriteFormat;
        use crate::client::write::{RowBytes, WriteRecord};
        use bytes::Bytes;

        // Use a tiny batch size so the KV record exceeds it
        let config = Config {
            writer_batch_size: 64,
            writer_buffer_memory_size: 1024 * 1024,
            ..Config::default()
        };

        let accumulator = RecordAccumulator::new(config, disabled_idempotence());
        let table_path = TablePath::new("db".to_string(), "tbl".to_string());
        let table_info = Arc::new(build_table_info(table_path.clone(), 1, 1));
        let physical_table_path = Arc::new(PhysicalTablePath::of(Arc::new(table_path.clone())));
        let cluster = Arc::new(build_cluster(&table_path, 1, 1));

        // Create a KV record larger than batch_size (64 bytes)
        let key = Bytes::from(vec![0u8; 32]);
        let value = vec![0u8; 256];
        let record = WriteRecord::for_upsert(
            table_info,
            physical_table_path,
            1,
            key,
            None,
            WriteFormat::CompactedKv,
            None,
            Some(RowBytes::Owned(Bytes::from(value))),
        );

        // This used to panic with "must append to a new batch" because
        // the KV write limit was hardcoded to DEFAULT_WRITE_LIMIT (256 bytes)
        // instead of using alloc_size = max(batch_size, record_size).
        let result = accumulator.append(&record, 0, &cluster, false);
        assert!(result.is_ok(), "oversized KV record should not panic");
    }

    #[test]
    fn test_memory_permit_accounts_for_oversized_record() {
        use crate::client::write::write_format::WriteFormat;
        use crate::client::write::{RowBytes, WriteRecord};
        use bytes::Bytes;

        let config = Config {
            writer_batch_size: 64,
            writer_buffer_memory_size: 1024 * 1024,
            ..Config::default()
        };

        let accumulator = RecordAccumulator::new(config, disabled_idempotence());
        let table_path = TablePath::new("db".to_string(), "tbl".to_string());
        let table_info = Arc::new(build_table_info(table_path.clone(), 1, 1));
        let physical_table_path = Arc::new(PhysicalTablePath::of(Arc::new(table_path.clone())));
        let cluster = Arc::new(build_cluster(&table_path, 1, 1));

        let key = Bytes::from(vec![0u8; 32]);
        let value = vec![0u8; 256];
        let record = WriteRecord::for_upsert(
            table_info,
            physical_table_path,
            1,
            key,
            None,
            WriteFormat::CompactedKv,
            None,
            Some(RowBytes::Owned(Bytes::from(value))),
        );

        // estimated_record_size includes batch header overhead
        let expected_alloc = record.estimated_record_size();
        assert!(expected_alloc > 64, "record should exceed batch_size=64");

        accumulator.append(&record, 0, &cluster, false).unwrap();

        // The permit should reserve max(batch_size, estimated_record_size) bytes.
        let used = *accumulator.memory_limiter.state.lock();
        assert_eq!(
            used, expected_alloc,
            "memory limiter should reserve max(batch_size, estimated_record_size)"
        );
    }

    #[tokio::test]
    async fn test_sender_wakeup_notifies() {
        let accumulator = RecordAccumulator::new(Config::default(), disabled_idempotence());

        // notified() should complete when wakeup_sender() is called
        let notified = accumulator.notified();
        accumulator.wakeup_sender();
        // If wakeup doesn't work, this would hang forever.
        tokio::time::timeout(Duration::from_millis(100), notified)
            .await
            .expect("notified should complete after wakeup_sender");
    }
}