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
use std::borrow::BorrowMut;
use std::str::FromStr;
use std::sync::Arc;
use std::time::Duration;

use clickhouse_rs::{ClientHandle, Options, Pool};
use rlink::channel::utils::handover::Handover;
use rlink::core::checkpoint::CheckpointFunction;
use rlink::core::element::{FnSchema, Record};
use rlink::core::function::{Context, NamedFunction, OutputFormat};
use rlink::utils::thread::{async_runtime, async_sleep, async_spawn};
use rlink::{core, utils};

pub type CkBlock = clickhouse_rs::Block;

pub trait ClickhouseConverter: Send + Sync {
    fn create_batch(&self, batch_size: usize) -> Box<dyn ClickhouseBatch>;
}

pub trait ClickhouseBatch: Send + Sync {
    fn append(&mut self, record: Record);
    fn flush(&mut self) -> CkBlock;
}

#[derive(NamedFunction)]
pub struct ClickhouseSink {
    url: String,
    table: String,
    batch_size: usize,
    batch_timeout: Duration,
    tasks: usize,
    converter: Arc<Box<dyn ClickhouseConverter>>,
    handover: Option<Handover>,
}

impl ClickhouseSink {
    pub fn new(
        url: &str,
        table: &str,
        batch_size: usize,
        batch_timeout: Duration,
        tasks: usize,
        builder: Box<dyn ClickhouseConverter>,
    ) -> Self {
        ClickhouseSink {
            url: url.to_string(),
            table: table.to_string(),
            batch_size,
            batch_timeout,
            tasks,
            converter: Arc::new(builder),
            handover: None,
        }
    }
}

impl OutputFormat for ClickhouseSink {
    fn open(&mut self, context: &Context) -> core::Result<()> {
        self.handover = Some(Handover::new(self.name(), context.task_id.to_tags(), 10000));

        let urls: Vec<&str> = self.url.split(",").collect();
        let url = if urls.len() > 1 {
            urls.get(context.task_id.task_number() as usize % urls.len())
                .unwrap()
                .to_string()
        } else {
            self.url.to_string()
        };
        info!("location clickhouse database url:{} from {}", url, self.url);

        let mut task = ClickhouseSinkTask::new(
            url.as_str(),
            self.table.clone(),
            self.batch_size,
            self.batch_timeout,
            self.converter.clone(),
            self.handover.as_ref().unwrap().clone(),
        );
        let tasks = self.tasks;
        utils::thread::spawn("clickhouse-sink-block", move || {
            async_runtime("ck_sink").block_on(async {
                task.run(tasks).await;
            });
        });

        Ok(())
    }

    fn write_record(&mut self, record: Record) {
        self.handover.as_ref().unwrap().produce(record).unwrap();
    }

    fn close(&mut self) -> core::Result<()> {
        Ok(())
    }

    fn schema(&self, _input_schema: FnSchema) -> FnSchema {
        FnSchema::Empty
    }
}

impl CheckpointFunction for ClickhouseSink {}

#[derive(Clone)]
pub struct ClickhouseSinkTask {
    pool: Pool,
    table: String,
    batch_size: usize,
    batch_timeout: Duration,
    converter: Arc<Box<dyn ClickhouseConverter>>,
    handover: Handover,
}

impl ClickhouseSinkTask {
    pub fn new(
        url: &str,
        table: String,
        batch_size: usize,
        batch_timeout: Duration,
        builder: Arc<Box<dyn ClickhouseConverter>>,
        handover: Handover,
    ) -> Self {
        let opts = Options::from_str(url).expect("parse clickhouse url error");
        let pool = Pool::new(opts);
        ClickhouseSinkTask {
            pool,
            table,
            batch_size,
            batch_timeout,
            converter: builder,
            handover,
        }
    }

    pub async fn run(&mut self, tasks: usize) {
        let mut join_handlers = Vec::new();
        for _ in 0..tasks {
            let mut self_clone = self.clone();

            let handler = async_spawn(async move {
                match self_clone.run0().await {
                    Ok(_) => {}
                    Err(e) => {
                        error!("run task error. {}", e);
                    }
                }
            });

            join_handlers.push(handler);
        }

        for handler in join_handlers {
            handler.await.unwrap();
        }
    }

    pub async fn run0(&mut self) -> anyhow::Result<()> {
        let mut client = self.pool.get_handle().await?;
        loop {
            match self.batch_send(client.borrow_mut()).await {
                Ok(len) => {
                    if len == 0 {
                        async_sleep(Duration::from_secs(1)).await;
                    }
                }
                Err(e) => {
                    error!("write clickhouse error. {}", e);

                    // todo reconnection
                    self.reconnection(client.borrow_mut()).await?;
                }
            }
        }
    }

    async fn reconnection(&mut self, client: &mut ClientHandle) -> anyhow::Result<()> {
        let mut err = None;
        for _ in 0..180 {
            async_sleep(Duration::from_secs(1)).await;
            match client.check_connection().await {
                Ok(_) => {
                    err = None;
                    break;
                }
                Err(e) => {
                    error!("reconnection error. {:?}", e);
                    err = Some(e)
                }
            }
        }

        match err {
            Some(e) => Err(anyhow::Error::from(e)),
            None => Ok(()),
        }
    }

    async fn batch_send(&mut self, client: &mut ClientHandle) -> anyhow::Result<usize> {
        let mut batch_block = self.converter.create_batch(self.batch_size);
        let begin_timestamp = utils::date_time::current_timestamp();
        let mut size = 0;
        for n in 0..self.batch_size {
            match self.handover.try_poll_next() {
                Ok(record) => {
                    batch_block.append(record);
                    size = n;
                }
                Err(_e) => {
                    async_sleep(Duration::from_millis(100)).await;
                    let current_timestamp = utils::date_time::current_timestamp();
                    if current_timestamp - begin_timestamp > self.batch_timeout {
                        break;
                    }
                }
            }
        }

        if size > 0 {
            let block = batch_block.flush();
            client.insert(self.table.as_str(), block).await?;
        }

        Ok(size)
    }
}

#[cfg(test)]
mod tests {
    use clickhouse_rs::Options;
    use std::str::FromStr;

    #[test]
    pub fn options_test() {
        let opt = Options::from_str(
            "tcp://rlink:123456@localhost:9000?keepalive=10s&connection_timeout=10s",
        )
        .unwrap();
        println!("{:?}", opt);
    }
}