fiddler 4.9.1

Data Stream processor written in rust
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
//! ClickHouse output module for batch data insertion.
//!
//! This module provides a batching output implementation that sends data
//! to a ClickHouse database using bulk INSERT operations.
//!
//! # Configuration
//!
//! ```yaml
//! output:
//!   clickhouse:
//!     url: "http://localhost:8123"        # Required: ClickHouse HTTP endpoint
//!     database: "default"                 # Optional: Database name (default: "default")
//!     table: "events"                     # Required: Table name
//!     username: "default"                 # Optional: Username for authentication
//!     password: ""                        # Optional: Password for authentication
//!     batch:                              # Optional: Batching policy
//!       size: 1000                        # Optional: Batch size (default: 500)
//!     create_table: false                 # Optional: Auto-create table (default: false)
//!     columns:                            # Optional: Column definitions for table creation
//!       - name: "timestamp"
//!         type: "DateTime64(3)"
//!       - name: "data"
//!         type: "String"
//! ```
//!
//! # Message Format
//!
//! Messages are expected to be valid JSON objects. Each JSON object is inserted
//! as a row in the target table. The JSON keys should match the column names
//! in the ClickHouse table.
//!
//! # Example
//!
//! Given a table:
//! ```sql
//! CREATE TABLE events (
//!     timestamp DateTime64(3),
//!     event_type String,
//!     data String
//! ) ENGINE = MergeTree() ORDER BY timestamp
//! ```
//!
//! And messages like:
//! ```json
//! {"timestamp": "2024-01-15 10:30:00.000", "event_type": "click", "data": "..."}
//! ```

use crate::config::register_plugin;
use crate::config::ItemType;
use crate::config::{ConfigSpec, ExecutionType};
use crate::modules::tls::ClientTlsConfig;
use crate::{BatchingPolicy, Closer, Error, MessageBatch, OutputBatch};
use async_trait::async_trait;
use fiddler_macros::fiddler_registration_func;
use flume::{bounded, Receiver, Sender};
use serde::Deserialize;
use serde_yaml::Value;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::Arc;
use std::time::Duration;
use tracing::{debug, error, warn};

const DEFAULT_DATABASE: &str = "default";

/// Column definition for table creation.
#[derive(Debug, Deserialize, Clone)]
pub struct ColumnDef {
    /// Column name.
    pub name: String,
    /// ClickHouse data type (e.g., "String", "DateTime64(3)", "UInt64").
    #[serde(rename = "type")]
    pub col_type: String,
}

/// ClickHouse output configuration.
#[derive(Deserialize, Clone)]
pub struct ClickHouseOutputConfig {
    /// ClickHouse HTTP endpoint URL (required).
    pub url: String,
    /// Database name (default: "default").
    #[serde(default = "default_database")]
    pub database: String,
    /// Table name (required).
    pub table: String,
    /// Username for authentication.
    pub username: Option<String>,
    /// Password for authentication.
    pub password: Option<String>,
    /// Batching policy configuration.
    #[serde(default)]
    pub batch: Option<BatchingPolicy>,
    /// Whether to auto-create the table if it doesn't exist (default: false).
    #[serde(default)]
    pub create_table: bool,
    /// Column definitions for table creation.
    #[serde(default)]
    pub columns: Vec<ColumnDef>,
    /// TLS configuration for custom CA certificates and client certificates.
    pub tls: Option<ClientTlsConfig>,
}

fn default_database() -> String {
    DEFAULT_DATABASE.to_string()
}

/// Validate a ClickHouse identifier (database, table, column name).
fn validate_identifier(identifier: &str) -> Result<(), Error> {
    if identifier.is_empty() {
        return Err(Error::Validation("Identifier cannot be empty".into()));
    }

    let first_char = identifier.chars().next().unwrap_or('0');
    if !first_char.is_alphabetic() && first_char != '_' {
        return Err(Error::Validation(format!(
            "Invalid identifier '{}': must start with letter or underscore",
            identifier
        )));
    }

    if !identifier.chars().all(|c| c.is_alphanumeric() || c == '_') {
        return Err(Error::Validation(format!(
            "Invalid identifier '{}': must contain only alphanumeric characters and underscores",
            identifier
        )));
    }

    Ok(())
}

/// Request sent to the background handler task.
struct Request {
    messages: MessageBatch,
    response: Sender<Result<(), Error>>,
}

/// ClickHouse HTTP client for the background handler.
struct ClickHouseClient {
    client: reqwest::Client,
    url: String,
    database: String,
    table: String,
    username: Option<String>,
    password: Option<String>,
    table_created: AtomicBool,
    create_table_enabled: bool,
    columns: Vec<ColumnDef>,
}

impl ClickHouseClient {
    fn new(config: &ClickHouseOutputConfig) -> Result<Self, Error> {
        // Validate identifiers
        validate_identifier(&config.database)?;
        validate_identifier(&config.table)?;
        for col in &config.columns {
            validate_identifier(&col.name)?;
        }

        let mut builder = reqwest::Client::builder()
            .pool_max_idle_per_host(2)
            .pool_idle_timeout(Duration::from_secs(90))
            .timeout(Duration::from_secs(60)) // Longer timeout for batch inserts
            .connect_timeout(Duration::from_secs(10));

        if let Some(ref tls) = config.tls {
            builder = crate::modules::tls::configure_reqwest_tls(builder, tls)?;
        }

        let client = builder
            .build()
            .map_err(|e| Error::ExecutionError(format!("Failed to build HTTP client: {}", e)))?;

        Ok(Self {
            client,
            url: config.url.clone(),
            database: config.database.clone(),
            table: config.table.clone(),
            username: config.username.clone(),
            password: config.password.clone(),
            table_created: AtomicBool::new(false),
            create_table_enabled: config.create_table,
            columns: config.columns.clone(),
        })
    }

    async fn execute_query(&self, query: &str) -> Result<(), Error> {
        let url = format!("{}/?database={}", self.url, self.database);

        let mut request = self.client.post(&url).body(query.to_string());

        if let (Some(username), Some(password)) = (&self.username, &self.password) {
            request = request.basic_auth(username, Some(password));
        } else if let Some(username) = &self.username {
            request = request.basic_auth(username, None::<&str>);
        }

        let response = request
            .send()
            .await
            .map_err(|e| Error::OutputError(format!("ClickHouse request failed: {}", e)))?;

        if !response.status().is_success() {
            let status = response.status();
            let body = response.text().await.unwrap_or_default();
            if status.is_client_error() {
                return Err(Error::UnRetryable(format!(
                    "ClickHouse query failed with status {}: {}",
                    status, body
                )));
            }
            return Err(Error::OutputError(format!(
                "ClickHouse query failed with status {}: {}",
                status, body
            )));
        }

        Ok(())
    }

    async fn create_table_if_not_exists(&self) -> Result<(), Error> {
        if self.columns.is_empty() {
            return Err(Error::ConfigFailedValidation(
                "columns must be specified when create_table is true".into(),
            ));
        }

        let columns_def: String = self
            .columns
            .iter()
            .map(|c| format!("    {} {}", c.name, c.col_type))
            .collect::<Vec<_>>()
            .join(",\n");

        // Default to MergeTree engine with first column as order key
        let order_key = &self.columns[0].name;

        let create_query = format!(
            r#"CREATE TABLE IF NOT EXISTS {}.{} (
{}
) ENGINE = MergeTree()
ORDER BY ({})"#,
            self.database, self.table, columns_def, order_key
        );

        self.execute_query(&create_query).await
    }

    async fn insert_batch(&self, messages: &MessageBatch) -> Result<(), Error> {
        if messages.is_empty() {
            return Ok(());
        }

        // Ensure table exists if configured
        if self.create_table_enabled && !self.table_created.load(Ordering::Relaxed) {
            if self.create_table_if_not_exists().await.is_ok() {
                self.table_created.store(true, Ordering::Relaxed);
            }
        }

        // Build JSONEachRow format for ClickHouse
        // Each line is a JSON object representing one row
        let mut json_rows = Vec::with_capacity(messages.len());

        for msg in messages {
            // Validate that message is valid JSON
            match serde_json::from_slice::<serde_json::Value>(&msg.bytes) {
                Ok(json) => {
                    if json.is_object() {
                        // Compact JSON without newlines for JSONEachRow format
                        json_rows.push(json.to_string());
                    } else {
                        warn!("Skipping non-object JSON message");
                    }
                }
                Err(e) => {
                    warn!(error = %e, "Skipping invalid JSON message");
                }
            }
        }

        if json_rows.is_empty() {
            return Ok(());
        }

        // Use JSONEachRow format which is efficient and flexible
        let url = format!(
            "{}/?database={}&query=INSERT%20INTO%20{}%20FORMAT%20JSONEachRow",
            self.url, self.database, self.table
        );

        let body = json_rows.join("\n");

        let mut request = self.client.post(&url).body(body);

        if let (Some(username), Some(password)) = (&self.username, &self.password) {
            request = request.basic_auth(username, Some(password));
        } else if let Some(username) = &self.username {
            request = request.basic_auth(username, None::<&str>);
        }

        let response = request
            .send()
            .await
            .map_err(|e| Error::OutputError(format!("ClickHouse insert failed: {}", e)))?;

        if !response.status().is_success() {
            let status = response.status();
            let body = response.text().await.unwrap_or_default();
            if status.is_client_error() {
                return Err(Error::UnRetryable(format!(
                    "ClickHouse insert failed with status {}: {}",
                    status, body
                )));
            }
            return Err(Error::OutputError(format!(
                "ClickHouse insert failed with status {}: {}",
                status, body
            )));
        }

        debug!(count = json_rows.len(), "Inserted batch to ClickHouse");
        Ok(())
    }
}

/// Background handler task that processes insert requests.
async fn clickhouse_handler(
    client: Arc<ClickHouseClient>,
    requests: Receiver<Request>,
) -> Result<(), Error> {
    while let Ok(req) = requests.recv_async().await {
        let result = client.insert_batch(&req.messages).await;

        if let Err(ref e) = result {
            error!(error = %e, "Failed to insert batch to ClickHouse");
        }

        // Send result back to caller
        if req.response.send_async(result).await.is_err() {
            warn!("Response channel closed before result could be sent");
        }
    }

    debug!("ClickHouse handler task exiting");
    Ok(())
}

/// ClickHouse batch output.
///
/// Collects messages into batches and inserts them into ClickHouse
/// using the JSONEachRow format for efficient bulk loading.
pub struct ClickHouseOutput {
    sender: Sender<Request>,
    batch_size: usize,
    interval: Duration,
    max_batch_bytes: usize,
}

impl ClickHouseOutput {
    /// Creates a new ClickHouse output from configuration.
    pub fn new(config: ClickHouseOutputConfig) -> Result<Self, Error> {
        let client = Arc::new(ClickHouseClient::new(&config)?);

        // Extract batch settings
        let batch_size = config.batch.as_ref().map_or(500, |b| b.effective_size());
        let interval = config
            .batch
            .as_ref()
            .map_or(Duration::from_secs(10), |b| b.effective_duration());
        let max_batch_bytes = config
            .batch
            .as_ref()
            .map_or(10_485_760, |b| b.effective_max_batch_bytes());

        // Create channel for requests
        let (sender, receiver) = bounded(0);

        // Spawn background handler
        tokio::spawn(clickhouse_handler(client, receiver));

        debug!(
            url = %config.url,
            database = %config.database,
            table = %config.table,
            batch_size = batch_size,
            interval_secs = interval.as_secs(),
            "ClickHouse output initialized"
        );

        Ok(Self {
            sender,
            batch_size,
            interval,
            max_batch_bytes,
        })
    }
}

#[async_trait]
impl OutputBatch for ClickHouseOutput {
    async fn write_batch(&mut self, messages: MessageBatch) -> Result<(), Error> {
        debug!(count = messages.len(), "Sending batch to ClickHouse");

        let (tx, rx) = bounded(0);
        self.sender
            .send_async(Request {
                messages,
                response: tx,
            })
            .await
            .map_err(|e| Error::UnableToSendToChannel(format!("{}", e)))?;

        // Wait for result
        rx.recv_async().await.map_err(|e| {
            Error::OutputError(format!("Failed to receive response from handler: {}", e))
        })??;

        Ok(())
    }

    async fn batch_size(&self) -> usize {
        self.batch_size
    }

    async fn interval(&self) -> Duration {
        self.interval
    }

    async fn max_batch_bytes(&self) -> usize {
        self.max_batch_bytes
    }
}

#[async_trait]
impl Closer for ClickHouseOutput {
    async fn close(&mut self) -> Result<(), Error> {
        debug!("ClickHouse output closing");
        Ok(())
    }
}

#[fiddler_registration_func]
fn create_clickhouse_output(conf: Value) -> Result<ExecutionType, Error> {
    let config: ClickHouseOutputConfig = serde_yaml::from_value(conf)?;

    // Validate required fields
    if config.url.is_empty() {
        return Err(Error::ConfigFailedValidation("url is required".into()));
    }
    if config.table.is_empty() {
        return Err(Error::ConfigFailedValidation("table is required".into()));
    }

    // Validate auth configuration
    if config.username.is_some() && config.password.is_none() {
        return Err(Error::ConfigFailedValidation(
            "password is required when username is specified".into(),
        ));
    }

    // Validate create_table requires columns
    if config.create_table && config.columns.is_empty() {
        return Err(Error::ConfigFailedValidation(
            "columns must be specified when create_table is true".into(),
        ));
    }

    Ok(ExecutionType::OutputBatch(Box::new(ClickHouseOutput::new(
        config,
    )?)))
}

/// Registers the ClickHouse output plugin.
pub(crate) fn register_clickhouse() -> Result<(), Error> {
    let config = r#"type: object
required:
  - url
  - table
properties:
  url:
    type: string
    description: "ClickHouse HTTP endpoint URL"
  database:
    type: string
    default: "default"
    description: "Database name"
  table:
    type: string
    description: "Table name"
  username:
    type: string
    description: "Username for authentication"
  password:
    type: string
    description: "Password for authentication"
  batch:
    type: object
    properties:
      size:
        type: integer
        description: "Batch size (default: 500)"
      duration:
        type: string
        description: "Flush interval (default: 10s)"
      max_batch_bytes:
        type: integer
        default: 10485760
        description: "Maximum cumulative byte size per batch (default: 10MB)"
    description: "Batching policy configuration"
  create_table:
    type: boolean
    default: false
    description: "Auto-create table if not exists"
  columns:
    type: array
    items:
      type: object
      properties:
        name:
          type: string
        type:
          type: string
      required:
        - name
        - type
    description: "Column definitions for table creation"
  tls:
    type: object
    properties:
      ca:
        type: string
        description: "CA certificate — file path or inline PEM"
      cert:
        type: string
        description: "Client certificate for mTLS — file path or inline PEM"
      key:
        type: string
        description: "Client private key for mTLS — file path or inline PEM"
      skip_verify:
        type: boolean
        default: false
        description: "Skip server certificate verification"
    description: "TLS configuration for custom certificates"
"#;
    let conf_spec = ConfigSpec::from_schema(config)?;

    register_plugin(
        "clickhouse".into(),
        ItemType::OutputBatch,
        conf_spec,
        create_clickhouse_output,
    )
}

#[cfg(test)]
mod tests {
    use super::*;

    #[test]
    fn test_validate_identifier_valid() {
        assert!(validate_identifier("events").is_ok());
        assert!(validate_identifier("my_table").is_ok());
        assert!(validate_identifier("_private").is_ok());
        assert!(validate_identifier("Table123").is_ok());
    }

    #[test]
    fn test_validate_identifier_invalid() {
        assert!(validate_identifier("").is_err());
        assert!(validate_identifier("123invalid").is_err());
        assert!(validate_identifier("invalid-name").is_err());
        assert!(validate_identifier("invalid.name").is_err());
    }

    #[test]
    fn test_config_deserialization() {
        let yaml = r#"
url: "http://localhost:8123"
database: "mydb"
table: "events"
username: "user"
password: "pass"
batch:
  size: 1000
create_table: true
columns:
  - name: timestamp
    type: DateTime64(3)
  - name: data
    type: String
"#;
        let config: ClickHouseOutputConfig = serde_yaml::from_str(yaml).unwrap();
        assert_eq!(config.url, "http://localhost:8123");
        assert_eq!(config.database, "mydb");
        assert_eq!(config.table, "events");
        assert_eq!(config.username, Some("user".to_string()));
        assert_eq!(config.password, Some("pass".to_string()));
        assert!(config.create_table);
        assert_eq!(config.columns.len(), 2);
        assert_eq!(config.columns[0].name, "timestamp");
        assert_eq!(config.columns[0].col_type, "DateTime64(3)");
        assert_eq!(config.batch.as_ref().unwrap().effective_size(), 1000);
    }

    #[test]
    fn test_config_deserialization_with_duration() {
        let yaml = r#"
url: "http://localhost:8123"
database: "mydb"
table: "events"
username: "user"
password: "pass"
batch:
  size: 1000
  duration: 500ms
create_table: true
columns:
  - name: timestamp
    type: DateTime64(3)
  - name: data
    type: String
"#;
        let config: ClickHouseOutputConfig = serde_yaml::from_str(yaml).unwrap();
        assert_eq!(config.url, "http://localhost:8123");
        assert_eq!(config.database, "mydb");
        assert_eq!(config.table, "events");
        assert_eq!(config.username, Some("user".to_string()));
        assert_eq!(config.password, Some("pass".to_string()));
        assert!(config.create_table);
        assert_eq!(config.columns.len(), 2);
        assert_eq!(config.columns[0].name, "timestamp");
        assert_eq!(config.columns[0].col_type, "DateTime64(3)");
        assert_eq!(config.batch.as_ref().unwrap().effective_size(), 1000);
    }

    #[test]
    fn test_config_defaults() {
        let yaml = r#"
url: "http://localhost:8123"
table: "events"
"#;
        let config: ClickHouseOutputConfig = serde_yaml::from_str(yaml).unwrap();
        assert_eq!(config.database, "default");
        assert!(config.username.is_none());
        assert!(config.password.is_none());
        assert!(!config.create_table);
        assert!(config.columns.is_empty());
    }

    #[test]
    fn test_config_with_tls() {
        let yaml = r#"
url: "https://localhost:8443"
table: "events"
tls:
  ca: /etc/ssl/ca.crt
  skip_verify: false
"#;
        let config: ClickHouseOutputConfig = serde_yaml::from_str(yaml).unwrap();
        let tls = config.tls.as_ref().unwrap();
        assert_eq!(tls.ca.as_deref(), Some("/etc/ssl/ca.crt"));
        assert!(!tls.skip_verify);
        assert!(tls.cert.is_none());
    }

    #[test]
    fn test_register_clickhouse() {
        let result = register_clickhouse();
        assert!(result.is_ok() || matches!(result, Err(Error::DuplicateRegisteredName(_))));
    }
}