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
use std::borrow::BorrowMut;
use std::collections::HashMap;
use std::fmt::Debug;
use std::sync::Arc;
use std::time::Duration;

use anyhow::anyhow;
use elasticsearch::http::headers::HeaderMap;
use elasticsearch::http::request::JsonBody;
use elasticsearch::http::transport::{SingleNodeConnectionPool, TransportBuilder};
use elasticsearch::http::Url;
use elasticsearch::{BulkParts, Elasticsearch};
use rlink::api::checkpoint::CheckpointFunction;
use rlink::api::element::Record;
use rlink::api::function::{Context, NamedFunction, OutputFormat};
use rlink::channel::utils::handover::Handover;
use rlink::utils::thread::{async_runtime, async_sleep, async_spawn};
use rlink::{api, utils};
use serde_json::Value;
use thiserror::Error;

pub struct ElasticsearchModel {
    pub index: String,
    pub es_type: &'static str,
    pub body: Value,
}

#[derive(Clone, Serialize, Deserialize, Debug)]
pub struct Index {
    index: HashMap<String, String>,
}

impl Index {
    pub fn new() -> Self {
        Index {
            index: HashMap::new(),
        }
    }

    pub fn set_index(&mut self, index_value: String) {
        self.index.insert("_index".to_string(), index_value);
    }

    pub fn set_type(&mut self, type_value: String) {
        self.index.insert("_type".to_string(), type_value);
    }

    pub fn to_json(&self) -> Result<Value, serde_json::Error> {
        serde_json::to_value(self)
    }
}

pub trait ElasticsearchConverter: Debug + Send + Sync {
    fn to_json(&self, record: &mut Record) -> ElasticsearchModel;
}

#[derive(Debug, NamedFunction)]
pub struct ElasticsearchOutputFormat {
    address: String,
    headers: HashMap<String, String>,

    builder: Arc<Box<dyn ElasticsearchConverter>>,
    handover: Option<Handover>,
}

impl ElasticsearchOutputFormat {
    pub fn new(
        address: &str,
        headers: HashMap<String, String>,
        builder: Box<dyn ElasticsearchConverter>,
    ) -> Self {
        ElasticsearchOutputFormat {
            address: address.to_string(),
            headers,
            builder: Arc::new(builder),
            handover: None,
        }
    }
}

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

        let mut write_thead = ElasticsearchWriteThread::new(
            self.address.as_str(),
            self.headers.clone(),
            self.handover.as_ref().unwrap().clone(),
            3000,
        )
        .expect("build elasticsearch connection error");

        let convert = self.builder.clone();
        utils::thread::spawn("elastic-sink-block", move || {
            async_runtime("es_sink").block_on(async {
                write_thead.run(convert, 5).await;
            });
        });

        Ok(())
    }

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

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

impl CheckpointFunction for ElasticsearchOutputFormat {}

#[derive(Clone)]
pub struct ElasticsearchWriteThread {
    client: Elasticsearch,
    batch_size: usize,
    handover: Handover,
}

impl ElasticsearchWriteThread {
    pub fn new(
        address: &str,
        headers: HashMap<String, String>,
        handover: Handover,
        batch_size: usize,
    ) -> Result<Self, Box<dyn std::error::Error>> {
        let mut header_map = HeaderMap::new();
        if headers.contains_key("stoken") {
            let val = headers.get("stoken").unwrap();
            header_map.insert("stoken", val.as_str().parse().unwrap());
        }

        let url = Url::parse(address)?;
        let conn_pool = SingleNodeConnectionPool::new(url);
        let transport = TransportBuilder::new(conn_pool)
            .headers(header_map)
            .build()?;
        let client = Elasticsearch::new(transport);

        Ok(ElasticsearchWriteThread {
            client,
            batch_size,
            handover,
        })
    }

    pub async fn run(
        &mut self,
        converters: Arc<Box<dyn ElasticsearchConverter>>,
        parallelism: usize,
    ) {
        let mut join_handlers = Vec::new();
        for _ in 0..parallelism {
            let mut self_clone = self.clone();
            let converter = converters.clone();

            let handler = async_spawn(async move {
                self_clone.run0(converter).await;
            });

            join_handlers.push(handler);
        }

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

    pub async fn run0(&mut self, converter: Arc<Box<dyn ElasticsearchConverter>>) {
        loop {
            match self.batch_send(&converter).await {
                Ok(len) => {
                    if len == 0 {
                        async_sleep(Duration::from_secs(1)).await;
                    }
                }
                Err(e) => {
                    error!("write elasticsearch error. {}", e);
                    async_sleep(Duration::from_millis(100)).await;
                }
            }
        }
    }

    async fn batch_send(
        &self,
        converter: &Box<dyn ElasticsearchConverter>,
    ) -> Result<usize, Box<dyn std::error::Error + Send>> {
        let mut bulk_bodies = Vec::with_capacity(self.batch_size);
        for _ in 0..self.batch_size {
            match self.handover.try_poll_next() {
                Ok(mut record) => {
                    let ElasticsearchModel {
                        index,
                        es_type,
                        body,
                    } = converter.to_json(record.borrow_mut());

                    let mut index_model = Index::new();
                    index_model.set_index(index.clone());
                    index_model.set_type(es_type.to_string());
                    bulk_bodies.push(JsonBody::new(index_model.to_json().unwrap()));

                    bulk_bodies.push(JsonBody::new(body));
                }
                Err(_e) => {
                    break;
                }
            }
        }

        let len = bulk_bodies.len();
        self.flush(bulk_bodies).await.map_err(|e| {
            let err = std::io::Error::new(std::io::ErrorKind::Other, format!("{}", e));
            let source: Box<dyn std::error::Error + Send> = Box::new(err);
            source
        })?;

        Ok(len)
    }

    async fn flush(
        &self,
        body_bulk: Vec<JsonBody<Value>>,
    ) -> Result<bool, Box<dyn std::error::Error>> {
        if body_bulk.len() == 0 {
            return Ok(true);
        }
        let response = self
            .client
            .bulk(BulkParts::None)
            .body(body_bulk)
            .send()
            .await?;
        let response_body = response.json::<Value>().await?;
        let errors = response_body["errors"]
            .as_bool()
            .ok_or(anyhow!("no errors field in es response"))?;

        if errors {
            let err = std::io::Error::new(std::io::ErrorKind::Other, "");
            let source: Box<dyn std::error::Error + Send> = Box::new(err);
            Err(source)
        } else {
            Ok(true)
        }
    }
}

#[derive(Error, Debug)]
#[error("boxed source")]
pub struct BoxedSource {
    #[source]
    source: Box<dyn std::error::Error + Send + 'static>,
}