arkflow-plugin 0.1.0

High-performance Rust flow processing engine
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
//! Kafka output component
//!
//! Send the processed data to the Kafka topic

use serde::{Deserialize, Serialize};

use arkflow_core::output::{register_output_builder, Output, OutputBuilder};
use arkflow_core::{Content, Error, MessageBatch};

use async_trait::async_trait;
use rdkafka::config::ClientConfig;
use rdkafka::error::KafkaResult;
use rdkafka::message::ToBytes;
use rdkafka::producer::future_producer::OwnedDeliveryResult;
use rdkafka::producer::{FutureProducer, FutureRecord, Producer};
use rdkafka::util::Timeout;
use std::sync::Arc;
use std::time::Duration;
use tokio::sync::RwLock;

#[derive(Debug, Clone, Copy, Serialize, Deserialize)]
#[serde(rename_all = "lowercase")]
pub enum CompressionType {
    None,
    Gzip,
    Snappy,
    Lz4,
}

impl std::fmt::Display for CompressionType {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        match self {
            CompressionType::None => write!(f, "none"),
            CompressionType::Gzip => write!(f, "gzip"),
            CompressionType::Snappy => write!(f, "snappy"),
            CompressionType::Lz4 => write!(f, "lz4"),
        }
    }
}

/// Kafka output configuration
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct KafkaOutputConfig {
    /// List of Kafka server addresses
    pub brokers: Vec<String>,
    /// Target topic
    pub topic: String,
    /// Partition key (optional)
    pub key: Option<String>,
    /// Client ID
    pub client_id: Option<String>,
    /// Compression type
    pub compression: Option<CompressionType>,
    /// Acknowledgment level (0=no acknowledgment, 1=leader acknowledgment, all=all replica acknowledgments)
    pub acks: Option<String>,
}

/// Kafka output component
struct KafkaOutput<T> {
    config: KafkaOutputConfig,
    producer: Arc<RwLock<Option<T>>>,
}

impl<T: KafkaClient> KafkaOutput<T> {
    /// Create a new Kafka output component
    pub fn new(config: KafkaOutputConfig) -> Result<Self, Error> {
        Ok(Self {
            config,
            producer: Arc::new(RwLock::new(None)),
        })
    }
}

#[async_trait]
impl<T: KafkaClient> Output for KafkaOutput<T> {
    async fn connect(&self) -> Result<(), Error> {
        let mut client_config = ClientConfig::new();

        // Configure the Kafka server address
        client_config.set("bootstrap.servers", &self.config.brokers.join(","));

        // Set the client ID
        if let Some(client_id) = &self.config.client_id {
            client_config.set("client.id", client_id);
        }

        // Set the compression type
        if let Some(compression) = &self.config.compression {
            client_config.set("compression.type", compression.to_string().to_lowercase());
        }

        // Set the confirmation level
        if let Some(acks) = &self.config.acks {
            client_config.set("acks", acks);
        }

        // Create a producer
        let producer = T::create(&client_config)
            .map_err(|e| Error::Connection(format!("A Kafka producer cannot be created: {}", e)))?;

        // Save the producer instance
        let producer_arc = self.producer.clone();
        let mut producer_guard = producer_arc.write().await;
        *producer_guard = Some(producer);

        Ok(())
    }

    async fn write(&self, msg: &MessageBatch) -> Result<(), Error> {
        let producer_arc = self.producer.clone();
        let producer_guard = producer_arc.read().await;
        let producer = producer_guard.as_ref().ok_or_else(|| {
            Error::Connection("The Kafka producer is not initialized".to_string())
        })?;

        let payloads = msg.as_string()?;
        if payloads.is_empty() {
            return Ok(());
        }

        match &msg.content {
            Content::Arrow(_) => {
                return Err(Error::Process(
                    "The arrow format is not supported".to_string(),
                ))
            }
            Content::Binary(v) => {
                for x in v {
                    // Create record
                    let mut record = FutureRecord::to(&self.config.topic).payload(&x);

                    // Set partition key if available
                    if let Some(key) = &self.config.key {
                        record = record.key(key);
                    }

                    // Get the producer and send the message
                    producer
                        .send(record, Duration::from_secs(5))
                        .await
                        .map_err(|(e, _)| {
                            Error::Process(format!("Failed to send a Kafka message: {}", e))
                        })?;
                }
            }
        }
        Ok(())
    }

    async fn close(&self) -> Result<(), Error> {
        // Get the producer and close
        let producer_arc = self.producer.clone();
        let mut producer_guard = producer_arc.write().await;

        if let Some(producer) = producer_guard.take() {
            // Wait for all messages to be sent
            producer.flush(Duration::from_secs(30)).map_err(|e| {
                Error::Connection(format!(
                    "Failed to refresh the message when the Kafka producer is disabled: {}",
                    e
                ))
            })?;
        }
        Ok(())
    }
}

pub(crate) struct KafkaOutputBuilder;
impl OutputBuilder for KafkaOutputBuilder {
    fn build(&self, config: &Option<serde_json::Value>) -> Result<Arc<dyn Output>, Error> {
        if config.is_none() {
            return Err(Error::Config(
                "HTTP output configuration is missing".to_string(),
            ));
        }
        let config: KafkaOutputConfig = serde_json::from_value(config.clone().unwrap())?;

        Ok(Arc::new(KafkaOutput::<FutureProducer>::new(config)?))
    }
}

pub fn init() {
    register_output_builder("kafka", Arc::new(KafkaOutputBuilder));
}
#[async_trait]
trait KafkaClient: Send + Sync {
    fn create(config: &ClientConfig) -> KafkaResult<Self>
    where
        Self: Sized;

    async fn send<K, P, T>(
        &self,
        record: FutureRecord<'_, K, P>,
        queue_timeout: T,
    ) -> OwnedDeliveryResult
    where
        K: ToBytes + ?Sized + Sync,
        P: ToBytes + ?Sized + Sync,
        T: Into<Timeout> + Sync + Send;

    fn flush<T: Into<Timeout>>(&self, timeout: T) -> KafkaResult<()>;
}
#[async_trait]
impl KafkaClient for FutureProducer {
    fn create(config: &ClientConfig) -> KafkaResult<Self> {
        config.create()
    }
    async fn send<K, P, T>(
        &self,
        record: FutureRecord<'_, K, P>,
        queue_timeout: T,
    ) -> OwnedDeliveryResult
    where
        K: ToBytes + ?Sized + Sync,
        P: ToBytes + ?Sized + Sync,
        T: Into<Timeout> + Sync + Send,
    {
        FutureProducer::send(self, record, queue_timeout).await
    }

    fn flush<T: Into<Timeout>>(&self, timeout: T) -> KafkaResult<()> {
        Producer::flush(self, timeout)
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use rdkafka::Timestamp;
    use std::sync::atomic::{AtomicBool, Ordering};
    use std::sync::Arc;
    use tokio::sync::Mutex;

    // Mock Kafka client for testing
    struct MockKafkaClient {
        // Track if client is connected
        connected: Arc<AtomicBool>,
        // Store sent messages for verification
        sent_messages: Arc<Mutex<Vec<(String, Vec<u8>, Option<String>)>>>,
        // Flag to simulate errors
        should_fail: Arc<AtomicBool>,
    }

    impl MockKafkaClient {
        fn new() -> Self {
            Self {
                connected: Arc::new(AtomicBool::new(true)),
                sent_messages: Arc::new(Mutex::new(Vec::new())),
                should_fail: Arc::new(AtomicBool::new(false)),
            }
        }

        fn with_failure() -> Self {
            let client = Self::new();
            client.should_fail.store(true, Ordering::SeqCst);
            client
        }
    }

    #[async_trait]
    impl KafkaClient for MockKafkaClient {
        fn create(config: &ClientConfig) -> KafkaResult<Self> {
            // Simulate connection failure if bootstrap.servers is empty
            if config.get("bootstrap.servers").unwrap_or("") == "" {
                return Err(rdkafka::error::KafkaError::ClientCreation(
                    "Failed to create client".to_string(),
                ));
            }
            Ok(Self::new())
        }

        async fn send<K, P, T>(
            &self,
            record: FutureRecord<'_, K, P>,
            _queue_timeout: T,
        ) -> OwnedDeliveryResult
        where
            K: ToBytes + ?Sized + Sync,
            P: ToBytes + ?Sized + Sync,
            T: Into<Timeout> + Sync + Send,
        {
            // Check if we should simulate a failure
            if self.should_fail.load(Ordering::SeqCst) {
                let err = rdkafka::error::KafkaError::MessageProduction(
                    rdkafka::types::RDKafkaErrorCode::QueueFull,
                );
                // Create OwnedMessage instead of Vec<u8> for the error return
                let payload = rdkafka::message::OwnedMessage::new(
                    Some(record.payload.unwrap().to_bytes().to_vec()),
                    None,
                    record.topic.to_string(),
                    Timestamp::NotAvailable,
                    0,
                    0,
                    None,
                );
                return Err((err, payload));
            }

            // Store the message for later verification
            let mut messages = self.sent_messages.lock().await;
            messages.push((
                record.topic.to_string(),
                record.payload.unwrap().to_bytes().to_vec(),
                record
                    .key
                    .map(|k| String::from_utf8_lossy(k.to_bytes()).to_string()),
            ));

            // Return a successful delivery
            // Convert RDKafkaRespErr to i32 for the success case
            Ok((
                rdkafka::types::RDKafkaRespErr::RD_KAFKA_RESP_ERR_NO_ERROR as i32,
                0,
            ))
        }

        fn flush<T: Into<Timeout>>(&self, _timeout: T) -> KafkaResult<()> {
            // Immediately return error if should_fail is true, preventing any blocking
            if self.should_fail.load(Ordering::SeqCst) {
                return Err(rdkafka::error::KafkaError::Flush(
                    rdkafka::types::RDKafkaErrorCode::QueueFull,
                ));
            }
            Ok(())
        }
    }

    /// Test creating a new Kafka output component
    #[tokio::test]
    async fn test_kafka_output_new() {
        // Create a basic configuration
        let config = KafkaOutputConfig {
            brokers: vec!["localhost:9092".to_string()],
            topic: "test-topic".to_string(),
            key: None,
            client_id: None,
            compression: None,
            acks: None,
        };

        // Create a new Kafka output component
        let output = KafkaOutput::<MockKafkaClient>::new(config);
        assert!(output.is_ok(), "Failed to create Kafka output component");
    }

    /// Test connecting to Kafka
    #[tokio::test]
    async fn test_kafka_output_connect() {
        // Create a basic configuration
        let config = KafkaOutputConfig {
            brokers: vec!["localhost:9092".to_string()],
            topic: "test-topic".to_string(),
            key: None,
            client_id: None,
            compression: None,
            acks: None,
        };

        // Create and connect the Kafka output
        let output = KafkaOutput::<MockKafkaClient>::new(config).unwrap();
        let result = output.connect().await;
        assert!(result.is_ok(), "Failed to connect to Kafka");

        // Verify producer is initialized
        let producer_guard = output.producer.read().await;
        assert!(producer_guard.is_some(), "Kafka producer not initialized");
    }

    /// Test connection failure
    #[tokio::test]
    async fn test_kafka_output_connect_failure() {
        // Create a configuration with empty brokers to trigger failure
        let config = KafkaOutputConfig {
            brokers: vec![],
            topic: "test-topic".to_string(),
            key: None,
            client_id: None,
            compression: None,
            acks: None,
        };

        // Create and try to connect the Kafka output
        let output = KafkaOutput::<MockKafkaClient>::new(config).unwrap();
        let result = output.connect().await;
        assert!(result.is_err(), "Connection should fail with empty brokers");
    }

    /// Test writing messages to Kafka
    #[tokio::test]
    async fn test_kafka_output_write() {
        // Create a basic configuration
        let config = KafkaOutputConfig {
            brokers: vec!["localhost:9092".to_string()],
            topic: "test-topic".to_string(),
            key: None,
            client_id: None,
            compression: None,
            acks: None,
        };

        // Create and connect the Kafka output
        let output = KafkaOutput::<MockKafkaClient>::new(config).unwrap();
        output.connect().await.unwrap();

        // Create a test message
        let msg = MessageBatch::from_string("test message");
        let result = output.write(&msg).await;
        assert!(result.is_ok(), "Failed to write message to Kafka");

        // Verify the message was sent
        let producer_guard = output.producer.read().await;
        let producer = producer_guard.as_ref().unwrap();
        let messages = producer.sent_messages.lock().await;
        assert_eq!(messages.len(), 1, "Message not sent to Kafka");
        assert_eq!(messages[0].0, "test-topic", "Wrong topic");
        assert_eq!(messages[0].1, b"test message", "Wrong message content");
        assert_eq!(messages[0].2, None, "Key should be None");
    }

    /// Test writing messages with a partition key
    #[tokio::test]
    async fn test_kafka_output_write_with_key() {
        // Create a configuration with a partition key
        let config = KafkaOutputConfig {
            brokers: vec!["localhost:9092".to_string()],
            topic: "test-topic".to_string(),
            key: Some("test-key".to_string()),
            client_id: None,
            compression: None,
            acks: None,
        };

        // Create and connect the Kafka output
        let output = KafkaOutput::<MockKafkaClient>::new(config).unwrap();
        output.connect().await.unwrap();

        // Create a test message
        let msg = MessageBatch::from_string("test message");
        let result = output.write(&msg).await;
        assert!(result.is_ok(), "Failed to write message to Kafka");

        // Verify the message was sent with the key
        let producer_guard = output.producer.read().await;
        let producer = producer_guard.as_ref().unwrap();
        let messages = producer.sent_messages.lock().await;
        assert_eq!(messages.len(), 1, "Message not sent to Kafka");
        assert_eq!(messages[0].2, Some("test-key".to_string()), "Wrong key");
    }

    /// Test writing to Kafka without connecting first
    #[tokio::test]
    async fn test_kafka_output_write_without_connect() {
        // Create a basic configuration
        let config = KafkaOutputConfig {
            brokers: vec!["localhost:9092".to_string()],
            topic: "test-topic".to_string(),
            key: None,
            client_id: None,
            compression: None,
            acks: None,
        };

        // Create Kafka output without connecting
        let output = KafkaOutput::<MockKafkaClient>::new(config).unwrap();
        let msg = MessageBatch::from_string("test message");
        let result = output.write(&msg).await;

        // Should return connection error
        assert!(result.is_err(), "Write should fail when not connected");
        match result {
            Err(Error::Connection(_)) => {} // Expected error
            _ => panic!("Expected Connection error"),
        }
    }

    /// Test writing with send failure
    #[tokio::test]
    async fn test_kafka_output_write_failure() {
        // Create a basic configuration
        let config = KafkaOutputConfig {
            brokers: vec!["localhost:9092".to_string()],
            topic: "test-topic".to_string(),
            key: None,
            client_id: None,
            compression: None,
            acks: None,
        };

        // Create and connect the Kafka output
        let output = KafkaOutput::<MockKafkaClient>::new(config).unwrap();
        output.connect().await.unwrap();

        // Set the producer to fail
        let producer_guard = output.producer.read().await;
        let producer = producer_guard.as_ref().unwrap();
        producer.should_fail.store(true, Ordering::SeqCst);

        // Create a test message
        let msg = MessageBatch::from_string("test message");
        let result = output.write(&msg).await;
        assert!(result.is_err(), "Write should fail with producer error");
    }

    /// Test closing Kafka connection
    #[tokio::test]
    async fn test_kafka_output_close() {
        // Create a basic configuration
        let config = KafkaOutputConfig {
            brokers: vec!["localhost:9092".to_string()],
            topic: "test-topic".to_string(),
            key: None,
            client_id: None,
            compression: None,
            acks: None,
        };

        // Create and connect the Kafka output
        let output = KafkaOutput::<MockKafkaClient>::new(config).unwrap();
        output.connect().await.unwrap();

        // Close the connection
        let result = output.close().await;
        assert!(result.is_ok(), "Failed to close Kafka connection");

        // Verify producer is cleared
        let producer_guard = output.producer.read().await;
        assert!(producer_guard.is_none(), "Kafka producer not cleared");
    }

    /// Test closing with flush failure
    #[tokio::test]
    async fn test_kafka_output_close_failure() {
        // Create a basic configuration
        let config = KafkaOutputConfig {
            brokers: vec!["localhost:9092".to_string()],
            topic: "test-topic".to_string(),
            key: None,
            client_id: None,
            compression: None,
            acks: None,
        };

        // Create and connect the Kafka output
        let output = KafkaOutput::<MockKafkaClient>::new(config).unwrap();
        output.connect().await.unwrap();

        // Set the producer to fail before acquiring the write lock
        {
            let producer_guard = output.producer.read().await;
            let producer = producer_guard.as_ref().unwrap();
            producer.should_fail.store(true, Ordering::SeqCst);
        }

        // Close the connection
        let result = output.close().await;
        assert!(result.is_err(), "Close should fail with flush error");
    }
}