rustsim-io 0.0.1

Arrow batch builders, CSV bridge, and ClickHouse writer for rustsim
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
//! ClickHouse writer with bounded-channel backpressure.
//!
//! [`ClickHouseWriter`] runs a background thread that receives Arrow
//! [`RecordBatch`] values over a bounded
//! `sync_channel`, serializes them to Arrow IPC stream format, and POSTs
//! them to ClickHouse via HTTP.
//!
//! # Backpressure
//!
//! The bounded channel blocks the sender when the buffer is full, preventing
//! the simulation from outrunning the writer.
//!
//! # Delivery semantics
//!
//! Delivery is tracked in three stages:
//! - accepted into the writer channel
//! - successfully sent to ClickHouse
//! - dropped after all retries are exhausted
//!
//! The writer exposes both live snapshots and final shutdown statistics so
//! callers can distinguish enqueue success from durable delivery success.
//!
//! # Retry
//!
//! Failed HTTP requests are retried with exponential backoff and full jitter:
//!
//! ```text
//! delay = random(0, min(max_retry_delay, base_retry_delay x 2^(attempt-1)))
//! ```

use arrow_array::RecordBatch;
use arrow_ipc::writer::StreamWriter;
use rand::Rng;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::{mpsc, Arc, Mutex};
use std::thread::{self, JoinHandle};
use std::time::Duration;
use thiserror::Error;
use tracing::{debug, error, info, warn};

/// Errors that can occur in the ClickHouse writer.
#[derive(Debug, Error)]
pub enum ClickHouseError {
    /// Invalid writer configuration.
    #[error("invalid writer configuration: {0}")]
    Config(String),
    /// The writer channel was closed unexpectedly.
    #[error("writer channel closed: {0}")]
    ChannelClosed(String),
    /// An HTTP request to ClickHouse failed.
    #[error("HTTP request failed: {0}")]
    Http(String),
    /// Arrow serialization failed.
    #[error("Arrow serialization failed: {0}")]
    Arrow(#[from] arrow_schema::ArrowError),
}

/// Configuration for the ClickHouse writer.
#[derive(Debug, Clone)]
pub struct ClickHouseConfig {
    /// ClickHouse HTTP endpoint (e.g. `http://localhost:8123` or `https://...`).
    ///
    /// HTTPS requires the `tls` feature (enabled by default).
    pub url: String,
    /// Target table name for `INSERT INTO ... FORMAT ArrowStream`.
    pub table: String,
    /// Optional target database. When set, sent via `X-ClickHouse-Database`.
    pub database: Option<String>,
    /// Optional ClickHouse username. When set, sent via `X-ClickHouse-User`.
    pub username: Option<String>,
    /// Optional ClickHouse password/key. When set, sent via `X-ClickHouse-Key`.
    pub password: Option<String>,
    /// Advisory target batch size in rows.
    ///
    /// This field is an **advisory hint** for upstream batching code (such as
    /// [`CollectArrowBridge`](crate::bridge::CollectArrowBridge) or
    /// [`TelemetryPipeline`](https://docs.rs/rustsim)) that decide when to
    /// flush a partially-filled Arrow batch toward the writer. The writer
    /// itself does **not** split, merge, or reject batches based on this
    /// value — whatever `RecordBatch` is handed to
    /// [`ClickHouseWriter::send`] is serialized and sent as-is.
    ///
    /// Keep this in sync with the `batch_size` you pass to
    /// `CollectArrowBridge::new`, or read it back from the config to drive
    /// that call.
    pub max_batch_rows: usize,
    /// Maximum number of retry attempts per batch.
    pub max_retries: u32,
    /// Base delay for exponential backoff (default 100 ms).
    ///
    /// Retry N sleeps for `base_retry_delay x 2^(N-1)` plus jitter.
    pub base_retry_delay: Duration,
    /// Maximum backoff cap (default 10 s).
    ///
    /// The computed delay is clamped to this value before jitter is applied.
    pub max_retry_delay: Duration,
    /// TCP connect timeout (default 10 s).
    pub connect_timeout: Duration,
    /// HTTP read timeout per request (default 60 s).
    pub read_timeout: Duration,
    /// If true and the `gzip` feature is enabled, compress request bodies.
    ///
    /// Has no effect when the `gzip` feature is disabled.
    pub gzip: bool,
    /// Number of background writer threads draining the channel in parallel.
    ///
    /// Defaults to `1` (single-threaded writer preserving submission order).
    /// Values greater than `1` allow multiple concurrent HTTP requests to
    /// ClickHouse, which is useful when per-batch request latency is the
    /// bottleneck. Ordering between batches is **not** preserved with
    /// `workers > 1`; if downstream consumers rely on the submission order,
    /// keep this at `1`.
    pub workers: usize,
    /// Optional directory where batches that fail all retries are persisted
    /// to disk as Arrow IPC files.
    ///
    /// Requires the `spill` feature. When set, a batch that exhausts every
    /// retry is written to
    /// `spill_dir/batch-{unix_nanos}-{pid}-{seq}.arrows` instead of being
    /// silently dropped. Replay from disk is not automatic and is left to
    /// the operator (e.g. a separate ingest job).
    ///
    /// When the feature is disabled this field is ignored.
    pub spill_dir: Option<std::path::PathBuf>,
}

impl Default for ClickHouseConfig {
    fn default() -> Self {
        Self {
            url: "http://localhost:8123".to_string(),
            table: "agent_data".to_string(),
            database: None,
            username: None,
            password: None,
            max_batch_rows: 100_000,
            max_retries: 3,
            base_retry_delay: Duration::from_millis(100),
            max_retry_delay: Duration::from_secs(10),
            connect_timeout: Duration::from_secs(10),
            read_timeout: Duration::from_secs(60),
            gzip: false,
            workers: 1,
            spill_dir: None,
        }
    }
}

impl ClickHouseConfig {
    /// Create a config with the given endpoint URL and table name.
    ///
    /// All other settings use sensible defaults and can be overridden with the
    /// builder-style methods below.
    pub fn new(url: impl Into<String>, table: impl Into<String>) -> Self {
        Self {
            url: url.into(),
            table: table.into(),
            ..Self::default()
        }
    }

    /// Override the advisory batch-row target (see
    /// [`max_batch_rows`](Self::max_batch_rows)).
    pub fn with_max_batch_rows(mut self, max_batch_rows: usize) -> Self {
        self.max_batch_rows = max_batch_rows;
        self
    }

    /// Override the maximum retry count.
    pub fn with_max_retries(mut self, max_retries: u32) -> Self {
        self.max_retries = max_retries;
        self
    }

    /// Override the base retry delay.
    pub fn with_base_retry_delay(mut self, base_retry_delay: Duration) -> Self {
        self.base_retry_delay = base_retry_delay;
        self
    }

    /// Override the maximum retry delay.
    pub fn with_max_retry_delay(mut self, max_retry_delay: Duration) -> Self {
        self.max_retry_delay = max_retry_delay;
        self
    }

    /// Set ClickHouse authentication credentials (sent via
    /// `X-ClickHouse-User` / `X-ClickHouse-Key` headers).
    pub fn with_auth(mut self, username: impl Into<String>, password: impl Into<String>) -> Self {
        self.username = Some(username.into());
        self.password = Some(password.into());
        self
    }

    /// Set the target database (sent via `X-ClickHouse-Database`).
    pub fn with_database(mut self, database: impl Into<String>) -> Self {
        self.database = Some(database.into());
        self
    }

    /// Set TCP connect and HTTP read timeouts.
    pub fn with_timeouts(mut self, connect: Duration, read: Duration) -> Self {
        self.connect_timeout = connect;
        self.read_timeout = read;
        self
    }

    /// Enable or disable gzip request-body compression.
    ///
    /// Requires the `gzip` Cargo feature to have an effect.
    pub fn with_gzip(mut self, enabled: bool) -> Self {
        self.gzip = enabled;
        self
    }

    /// Override the number of concurrent background writer threads.
    ///
    /// See [`workers`](Self::workers) for caveats about ordering.
    /// Values less than 1 are clamped to 1.
    pub fn with_workers(mut self, workers: usize) -> Self {
        self.workers = workers.max(1);
        self
    }

    /// Enable on-disk spill for undelivered batches.
    ///
    /// Requires the `spill` Cargo feature to have an effect. The directory
    /// is created on first spill if it does not already exist.
    pub fn with_spill_dir(mut self, dir: impl Into<std::path::PathBuf>) -> Self {
        self.spill_dir = Some(dir.into());
        self
    }
}

/// Message type for the writer channel.
pub enum WriterMessage {
    /// A batch to write to ClickHouse.
    Batch(RecordBatch),
    /// Signal the writer to shut down after draining remaining batches.
    Shutdown,
}

/// Background ClickHouse writer with bounded-channel backpressure.
///
/// Created via [`ClickHouseWriter::new`]. Batches are sent to the background
/// thread via [`send`](Self::send). Call [`shutdown`](Self::shutdown) to drain
/// the channel and wait for completion.
pub struct ClickHouseWriter {
    sender: mpsc::SyncSender<WriterMessage>,
    handles: Vec<JoinHandle<WriterStats>>,
    metrics: Arc<WriterMetricsInner>,
    workers: usize,
}

/// Statistics from the writer's lifetime.
#[derive(Debug, Default, Clone, Copy, PartialEq, Eq)]
pub struct WriterStats {
    /// Number of batches accepted by the background writer loop.
    pub batches_received: u64,
    /// Total number of rows accepted by the background writer loop.
    pub rows_received: u64,
    /// Number of batches successfully sent.
    pub batches_sent: u64,
    /// Total number of rows across all successful batches.
    pub rows_sent: u64,
    /// Total number of failed send attempts (including retries).
    pub errors: u64,
    /// Number of batches dropped after all retries were exhausted.
    pub batches_failed: u64,
    /// Total number of rows dropped after all retries were exhausted.
    pub rows_failed: u64,
    /// Number of batches that were persisted to the spill directory after all
    /// retries were exhausted. Always `0` when the `spill` feature or
    /// [`ClickHouseConfig::spill_dir`] is not configured. Spilled batches are
    /// counted in `batches_failed`/`rows_failed` as well.
    pub batches_spilled: u64,
    /// Total number of rows persisted to the spill directory.
    pub rows_spilled: u64,
    /// Maximum observed outstanding-batch count submitted to the writer.
    pub max_outstanding_batches: u64,
}

/// Live snapshot of background-writer metrics.
#[derive(Debug, Default, Clone, Copy, PartialEq, Eq)]
pub struct WriterMetricsSnapshot {
    /// Number of batches currently submitted to the writer but not yet dequeued.
    pub outstanding_batches: u64,
    /// Number of batches accepted by the background writer loop.
    pub batches_received: u64,
    /// Total number of rows accepted by the background writer loop.
    pub rows_received: u64,
    /// Number of batches successfully sent.
    pub batches_sent: u64,
    /// Total number of rows across all successful batches.
    pub rows_sent: u64,
    /// Total number of failed send attempts (including retries).
    pub errors: u64,
    /// Number of batches dropped after all retries were exhausted.
    pub batches_failed: u64,
    /// Total number of rows dropped after all retries were exhausted.
    pub rows_failed: u64,
    /// Number of batches written to the spill directory.
    pub batches_spilled: u64,
    /// Total number of rows written to the spill directory.
    pub rows_spilled: u64,
    /// Maximum observed outstanding-batch count submitted to the writer.
    pub max_outstanding_batches: u64,
}

#[derive(Default)]
struct WriterMetricsInner {
    outstanding_batches: AtomicU64,
    batches_received: AtomicU64,
    rows_received: AtomicU64,
    batches_sent: AtomicU64,
    rows_sent: AtomicU64,
    errors: AtomicU64,
    batches_failed: AtomicU64,
    rows_failed: AtomicU64,
    batches_spilled: AtomicU64,
    rows_spilled: AtomicU64,
    max_outstanding_batches: AtomicU64,
}

impl WriterMetricsInner {
    fn snapshot(&self) -> WriterMetricsSnapshot {
        WriterMetricsSnapshot {
            outstanding_batches: self.outstanding_batches.load(Ordering::Relaxed),
            batches_received: self.batches_received.load(Ordering::Relaxed),
            rows_received: self.rows_received.load(Ordering::Relaxed),
            batches_sent: self.batches_sent.load(Ordering::Relaxed),
            rows_sent: self.rows_sent.load(Ordering::Relaxed),
            errors: self.errors.load(Ordering::Relaxed),
            batches_failed: self.batches_failed.load(Ordering::Relaxed),
            rows_failed: self.rows_failed.load(Ordering::Relaxed),
            batches_spilled: self.batches_spilled.load(Ordering::Relaxed),
            rows_spilled: self.rows_spilled.load(Ordering::Relaxed),
            max_outstanding_batches: self.max_outstanding_batches.load(Ordering::Relaxed),
        }
    }

    fn update_max_outstanding(&self, value: u64) {
        let mut current = self.max_outstanding_batches.load(Ordering::Relaxed);
        while value > current {
            match self.max_outstanding_batches.compare_exchange_weak(
                current,
                value,
                Ordering::Relaxed,
                Ordering::Relaxed,
            ) {
                Ok(_) => break,
                Err(observed) => current = observed,
            }
        }
    }
}

impl ClickHouseWriter {
    /// Create a new writer with bounded channel of `capacity` batches for backpressure.
    ///
    /// Spawns a background thread that drains the channel and sends batches
    /// to ClickHouse.
    pub fn new(config: ClickHouseConfig, capacity: usize) -> Result<Self, ClickHouseError> {
        if capacity == 0 {
            return Err(ClickHouseError::Config(
                "channel capacity must be positive".to_string(),
            ));
        }
        if config.max_retries == 0 {
            return Err(ClickHouseError::Config(
                "max_retries must be positive".to_string(),
            ));
        }
        if config.url.trim().is_empty() {
            return Err(ClickHouseError::Config("url must not be empty".to_string()));
        }
        if config.table.trim().is_empty() {
            return Err(ClickHouseError::Config(
                "table must not be empty".to_string(),
            ));
        }

        let (sender, receiver) = mpsc::sync_channel::<WriterMessage>(capacity);
        let metrics = Arc::new(WriterMetricsInner::default());
        let workers = config.workers.max(1);
        let receiver = Arc::new(Mutex::new(receiver));

        let mut handles = Vec::with_capacity(workers);
        for worker_id in 0..workers {
            let metrics_for_thread = Arc::clone(&metrics);
            let receiver_for_thread = Arc::clone(&receiver);
            let config_for_thread = config.clone();
            let handle = thread::spawn(move || {
                let mut stats = WriterStats::default();
                writer_loop(
                    worker_id,
                    &config_for_thread,
                    &receiver_for_thread,
                    &mut stats,
                    &metrics_for_thread,
                );
                stats
            });
            handles.push(handle);
        }

        Ok(Self {
            sender,
            handles,
            metrics,
            workers,
        })
    }

    /// Send a batch to the background writer.
    ///
    /// Blocks if the channel is full (backpressure).
    pub fn send(&self, batch: RecordBatch) -> Result<(), ClickHouseError> {
        let outstanding = self
            .metrics
            .outstanding_batches
            .fetch_add(1, Ordering::Relaxed)
            + 1;
        self.metrics.update_max_outstanding(outstanding);

        if let Err(e) = self.sender.send(WriterMessage::Batch(batch)) {
            self.metrics
                .outstanding_batches
                .fetch_sub(1, Ordering::Relaxed);
            return Err(ClickHouseError::ChannelClosed(e.to_string()));
        }

        Ok(())
    }

    /// Snapshot current writer metrics without shutting the writer down.
    pub fn metrics(&self) -> WriterMetricsSnapshot {
        self.metrics.snapshot()
    }

    /// Shutdown the writer, drain remaining batches, and return statistics.
    pub fn shutdown(mut self) -> WriterStats {
        // Send one Shutdown per worker so every worker thread unblocks.
        for _ in 0..self.workers {
            let _ = self.sender.send(WriterMessage::Shutdown);
        }
        let mut merged = WriterStats::default();
        for h in self.handles.drain(..) {
            if let Ok(stats) = h.join() {
                merged.batches_received += stats.batches_received;
                merged.rows_received += stats.rows_received;
                merged.batches_sent += stats.batches_sent;
                merged.rows_sent += stats.rows_sent;
                merged.errors += stats.errors;
                merged.batches_failed += stats.batches_failed;
                merged.rows_failed += stats.rows_failed;
                merged.batches_spilled += stats.batches_spilled;
                merged.rows_spilled += stats.rows_spilled;
                merged.max_outstanding_batches = merged
                    .max_outstanding_batches
                    .max(stats.max_outstanding_batches);
            }
        }
        merged
    }
}

fn writer_loop(
    worker_id: usize,
    config: &ClickHouseConfig,
    receiver: &Mutex<mpsc::Receiver<WriterMessage>>,
    stats: &mut WriterStats,
    metrics: &WriterMetricsInner,
) {
    info!(
        worker_id,
        workers = config.workers,
        table = %config.table,
        url = %config.url,
        database = ?config.database,
        auth = config.username.is_some(),
        gzip = config.gzip && cfg!(feature = "gzip"),
        "ClickHouse writer started"
    );

    let agent = ureq::AgentBuilder::new()
        .timeout_connect(config.connect_timeout)
        .timeout_read(config.read_timeout)
        .build();

    let mut rng = rand::thread_rng();

    loop {
        // Briefly lock the shared receiver to dequeue a single message.
        // HTTP send and retry backoff happen outside the lock so multiple
        // workers can fan out concurrently.
        let msg = match receiver.lock() {
            Ok(guard) => guard.recv(),
            Err(poisoned) => poisoned.into_inner().recv(),
        };
        let batch = match msg {
            Ok(WriterMessage::Batch(b)) => b,
            Ok(WriterMessage::Shutdown) => break,
            Err(_) => break,
        };

        metrics.outstanding_batches.fetch_sub(1, Ordering::Relaxed);
        let rows = batch.num_rows() as u64;
        stats.batches_received += 1;
        stats.rows_received += rows;
        metrics.batches_received.fetch_add(1, Ordering::Relaxed);
        metrics.rows_received.fetch_add(rows, Ordering::Relaxed);

        let mut success = false;
        for attempt in 1..=config.max_retries {
            match send_batch_http(&agent, config, &batch) {
                Ok(()) => {
                    debug!(worker_id, rows, attempt, "batch sent successfully");
                    success = true;
                    break;
                }
                Err(e) => {
                    stats.errors += 1;
                    metrics.errors.fetch_add(1, Ordering::Relaxed);

                    if attempt < config.max_retries {
                        let delay = backoff_with_jitter(
                            config.base_retry_delay,
                            config.max_retry_delay,
                            attempt,
                            &mut rng,
                        );
                        warn!(
                            worker_id,
                            attempt,
                            max_retries = config.max_retries,
                            delay_ms = delay.as_millis() as u64,
                            error = %e,
                            "batch send failed, retrying after backoff"
                        );
                        thread::sleep(delay);
                    } else {
                        warn!(
                            worker_id,
                            attempt,
                            max_retries = config.max_retries,
                            error = %e,
                            "batch send failed, no retries remaining"
                        );
                    }
                }
            }
        }
        if success {
            stats.batches_sent += 1;
            stats.rows_sent += rows;
            metrics.batches_sent.fetch_add(1, Ordering::Relaxed);
            metrics.rows_sent.fetch_add(rows, Ordering::Relaxed);
        } else {
            stats.batches_failed += 1;
            stats.rows_failed += rows;
            metrics.batches_failed.fetch_add(1, Ordering::Relaxed);
            metrics.rows_failed.fetch_add(rows, Ordering::Relaxed);

            #[cfg(feature = "spill")]
            if let Some(dir) = config.spill_dir.as_ref() {
                match spill_batch_to_disk(dir, worker_id, &batch) {
                    Ok(path) => {
                        stats.batches_spilled += 1;
                        stats.rows_spilled += rows;
                        metrics.batches_spilled.fetch_add(1, Ordering::Relaxed);
                        metrics.rows_spilled.fetch_add(rows, Ordering::Relaxed);
                        warn!(worker_id, rows, path = %path.display(), "batch spilled to disk after all retries exhausted");
                    }
                    Err(e) => {
                        error!(worker_id, rows, error = %e, "batch dropped: retries exhausted AND spill failed");
                    }
                }
            } else {
                error!(worker_id, rows, "batch dropped after all retries exhausted");
            }
            #[cfg(not(feature = "spill"))]
            error!(worker_id, rows, "batch dropped after all retries exhausted");
        }
    }

    stats.max_outstanding_batches = metrics.max_outstanding_batches.load(Ordering::Relaxed);

    info!(
        worker_id,
        batches_sent = stats.batches_sent,
        rows_sent = stats.rows_sent,
        errors = stats.errors,
        "ClickHouse writer shut down"
    );
}

/// Compute exponential backoff with full jitter.
///
/// delay = random(0, min(max_delay, base * 2^(attempt-1)))
fn backoff_with_jitter(
    base: Duration,
    max_delay: Duration,
    attempt: u32,
    rng: &mut impl Rng,
) -> Duration {
    let exp = base.saturating_mul(1u32 << (attempt - 1).min(30));
    let capped = exp.min(max_delay);
    let jitter_nanos = rng.gen_range(0..=capped.as_nanos().min(u64::MAX as u128) as u64);
    Duration::from_nanos(jitter_nanos)
}

/// Persist a RecordBatch to disk as an Arrow IPC stream file.
///
/// Filenames are globally unique per process: `batch-{nanos}-{pid}-{worker}-{seq}.arrows`.
/// Creates `dir` if it does not exist. Returns the full path on success.
#[cfg(feature = "spill")]
fn spill_batch_to_disk(
    dir: &std::path::Path,
    worker_id: usize,
    batch: &RecordBatch,
) -> Result<std::path::PathBuf, ClickHouseError> {
    use std::fs::{self, File};
    use std::io::BufWriter;
    use std::sync::atomic::AtomicU64;
    use std::time::{SystemTime, UNIX_EPOCH};

    static SPILL_SEQ: AtomicU64 = AtomicU64::new(0);

    fs::create_dir_all(dir)
        .map_err(|e| ClickHouseError::Config(format!("cannot create spill dir: {e}")))?;

    let nanos = SystemTime::now()
        .duration_since(UNIX_EPOCH)
        .map(|d| d.as_nanos())
        .unwrap_or(0);
    let pid = std::process::id();
    let seq = SPILL_SEQ.fetch_add(1, Ordering::Relaxed);
    let path = dir.join(format!("batch-{nanos}-{pid}-{worker_id}-{seq}.arrows"));

    let file = File::create(&path)
        .map_err(|e| ClickHouseError::Config(format!("cannot create spill file: {e}")))?;
    let mut bw = BufWriter::new(file);
    {
        let mut writer =
            StreamWriter::try_new(&mut bw, &batch.schema()).map_err(ClickHouseError::Arrow)?;
        writer.write(batch).map_err(ClickHouseError::Arrow)?;
        writer.finish().map_err(ClickHouseError::Arrow)?;
    }
    std::io::Write::flush(&mut bw)
        .map_err(|e| ClickHouseError::Config(format!("spill flush failed: {e}")))?;
    Ok(path)
}

/// Serialize a RecordBatch to Arrow IPC stream format and POST it to ClickHouse.
fn send_batch_http(
    agent: &ureq::Agent,
    config: &ClickHouseConfig,
    batch: &RecordBatch,
) -> Result<(), ClickHouseError> {
    // Serialize batch to Arrow IPC stream bytes
    let mut buf = Vec::new();
    {
        let mut writer =
            StreamWriter::try_new(&mut buf, &batch.schema()).map_err(ClickHouseError::Arrow)?;
        writer.write(batch).map_err(ClickHouseError::Arrow)?;
        writer.finish().map_err(ClickHouseError::Arrow)?;
    }

    let query = format!("INSERT INTO {} FORMAT ArrowStream", config.table);
    let url = format!("{}/?query={}", config.url, query);

    let mut request = agent
        .post(&url)
        .set("Content-Type", "application/octet-stream");

    if let Some(user) = &config.username {
        request = request.set("X-ClickHouse-User", user);
    }
    if let Some(pw) = &config.password {
        request = request.set("X-ClickHouse-Key", pw);
    }
    if let Some(db) = &config.database {
        request = request.set("X-ClickHouse-Database", db);
    }

    let body = compress_if_enabled(config, buf)?;
    if config.gzip && cfg!(feature = "gzip") {
        request = request.set("Content-Encoding", "gzip");
    }

    let response = request.send_bytes(&body);

    match response {
        Ok(resp) => {
            let status = resp.status();
            if status == 200 {
                Ok(())
            } else {
                let body = resp.into_string().unwrap_or_default();
                Err(ClickHouseError::Http(format!("HTTP {}: {}", status, body)))
            }
        }
        Err(e) => Err(ClickHouseError::Http(e.to_string())),
    }
}

#[cfg(feature = "gzip")]
fn compress_if_enabled(
    config: &ClickHouseConfig,
    buf: Vec<u8>,
) -> Result<Vec<u8>, ClickHouseError> {
    if !config.gzip {
        return Ok(buf);
    }
    use flate2::write::GzEncoder;
    use flate2::Compression;
    use std::io::Write;
    let mut encoder = GzEncoder::new(Vec::with_capacity(buf.len()), Compression::default());
    encoder
        .write_all(&buf)
        .map_err(|e| ClickHouseError::Http(format!("gzip encode: {e}")))?;
    encoder
        .finish()
        .map_err(|e| ClickHouseError::Http(format!("gzip finish: {e}")))
}

#[cfg(not(feature = "gzip"))]
fn compress_if_enabled(
    _config: &ClickHouseConfig,
    buf: Vec<u8>,
) -> Result<Vec<u8>, ClickHouseError> {
    Ok(buf)
}