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
#[macro_use]
extern crate serde_derive;
#[macro_use]
extern crate log;
#[macro_use]
extern crate lazy_static;
#[macro_use]
extern crate rlink_derive;
#[macro_use]
extern crate anyhow;

pub mod sink;
pub mod source;

pub(crate) mod state;

pub use sink::output_format::KafkaOutputFormat;
pub use source::input_format::KafkaInputFormat;

use std::collections::HashMap;

use rdkafka::ClientConfig;
use rlink::core::element::BufferReader;
use rlink::core::element::Record;

use crate::source::deserializer::{
    DefaultKafkaRecordDeserializer, DefaultKafkaRecordDeserializerBuilder,
    KafkaRecordDeserializerBuilder,
};

pub const BOOTSTRAP_SERVERS: &str = "bootstrap.servers";
pub const TOPICS: &str = "topics";
pub const GROUP_ID: &str = "group.id";

pub const SOURCE_CHANNEL_SIZE: usize = 50000;
pub const SINK_CHANNEL_SIZE: usize = 50000;

pub static KAFKA_DATA_TYPES: [u8; 6] = [
    // timestamp
    rlink::core::element::types::I64,
    // key
    rlink::core::element::types::BYTES,
    // payload
    rlink::core::element::types::BYTES,
    // topic
    rlink::core::element::types::BYTES,
    // partition
    rlink::core::element::types::I32,
    // offset
    rlink::core::element::types::I64,
];

pub fn build_kafka_record(
    timestamp: i64,
    key: &[u8],
    payload: &[u8],
    topic: &str,
    partition: i32,
    offset: i64,
) -> Result<Record, std::io::Error> {
    // 36 = 12(len(payload) + len(topic) + len(key)) +
    //      20(len(timestamp) + len(partition) + len(offset)) +
    //      4(place_holder)
    let capacity = payload.len() + topic.len() + key.len() + 36;
    let mut record = Record::with_capacity(capacity);
    let mut writer = record.as_writer(&KAFKA_DATA_TYPES);
    writer.set_i64(timestamp)?;
    writer.set_bytes(key)?;
    writer.set_bytes(payload)?;
    writer.set_str(topic)?;
    writer.set_i32(partition)?;
    writer.set_i64(offset)?;

    Ok(record)
}

pub struct KafkaRecord<'a, 'b> {
    reader: BufferReader<'a, 'b>,
}

impl<'a, 'b> KafkaRecord<'a, 'b> {
    pub fn new(record: &'a mut Record) -> Self {
        let reader = record.as_reader(&KAFKA_DATA_TYPES);
        KafkaRecord { reader }
    }

    pub fn get_kafka_timestamp(&self) -> Result<i64, std::io::Error> {
        self.reader.get_i64(0)
    }

    pub fn get_kafka_key(&self) -> Result<&[u8], std::io::Error> {
        self.reader.get_bytes(1)
    }

    pub fn get_kafka_payload(&self) -> Result<&[u8], std::io::Error> {
        self.reader.get_bytes(2)
    }

    pub fn get_kafka_topic(&self) -> Result<&str, std::io::Error> {
        self.reader.get_str(3)
    }

    pub fn get_kafka_partition(&self) -> Result<i32, std::io::Error> {
        self.reader.get_i32(4)
    }

    pub fn get_kafka_offset(&self) -> Result<i64, std::io::Error> {
        self.reader.get_i64(5)
    }
}

pub fn field(field_name: &str) -> String {
    format!("KAFKA_SOURCE.{}", field_name)
}

// pub fn create_input_format_from_prop(properties: &Properties) -> KafkaInputFormat {
//     let mut conf_map = HashMap::new();
//     conf_map.insert(
//         BOOTSTRAP_SERVERS.to_string(),
//         properties
//             .get_string(field(BOOTSTRAP_SERVERS).as_str())
//             .unwrap(),
//     );
//
//     let topics = properties.get_string(field(TOPICS).as_str()).unwrap();
//     let topics: Vec<&str> = topics.split(",").collect();
//     let topics = topics.iter().map(|topic| topic.to_string()).collect();
//
//     create_input_format(conf_map, topics)
// }

pub fn create_input_format(
    conf_map: HashMap<String, String>,
    topics: Vec<String>,
    buffer_size: Option<usize>,
    deserializer_builder: Option<Box<dyn KafkaRecordDeserializerBuilder>>,
) -> KafkaInputFormat {
    let mut client_config = ClientConfig::new();
    for (key, val) in conf_map {
        client_config.set(key.as_str(), val.as_str());
    }

    let buffer_size = buffer_size.unwrap_or(SOURCE_CHANNEL_SIZE);

    let deserializer_builder = deserializer_builder.unwrap_or_else(|| {
        let deserializer_builder: Box<dyn KafkaRecordDeserializerBuilder> =
            Box::new(DefaultKafkaRecordDeserializerBuilder::<
                DefaultKafkaRecordDeserializer,
            >::new());

        deserializer_builder
    });

    KafkaInputFormat::new(client_config, topics, buffer_size, deserializer_builder)
}

pub fn create_output_format(
    conf_map: HashMap<String, String>,
    topic: Option<String>,
    buffer_size: Option<usize>,
) -> KafkaOutputFormat {
    let mut client_config = ClientConfig::new();
    for (key, val) in conf_map {
        client_config.set(key.as_str(), val.as_str());
    }

    KafkaOutputFormat::new(
        client_config,
        topic,
        buffer_size.unwrap_or(SINK_CHANNEL_SIZE),
    )
}