risingwave_connector/sink/elasticsearch_opensearch/
elasticsearch_opensearch_formatter.rs

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
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use anyhow::anyhow;
use risingwave_common::array::{Op, StreamChunk};
use risingwave_common::catalog::Schema;
use risingwave_common::row::Row;
use serde_json::{Map, Value};

use super::super::encoder::template::TemplateEncoder;
use super::super::encoder::{JsonEncoder, RowEncoder};
use super::super::SinkError;
use crate::sink::Result;

pub struct ElasticSearchOpenSearchFormatter {
    key_encoder: TemplateEncoder,
    value_encoder: JsonEncoder,
    index_column: Option<usize>,
    index: Option<String>,
    routing_column: Option<usize>,
}

pub struct BuildBulkPara {
    pub index: String,
    pub key: String,
    pub value: Option<Map<String, Value>>,
    pub mem_size_b: usize,
    pub routing_column: Option<String>,
}

impl ElasticSearchOpenSearchFormatter {
    pub fn new(
        pk_indices: Vec<usize>,
        schema: &Schema,
        delimiter: Option<String>,
        index_column: Option<usize>,
        index: Option<String>,
        routing_column: Option<usize>,
    ) -> Result<Self> {
        let key_format = if pk_indices.is_empty() {
            let name = &schema
                .fields()
                .get(0)
                .ok_or_else(|| {
                    SinkError::ElasticSearchOpenSearch(anyhow!(
                        "no value find in sink schema, index is 0"
                    ))
                })?
                .name;
            format!("{{{}}}", name)
        } else if pk_indices.len() == 1 {
            let index = *pk_indices.get(0).unwrap();
            let name = &schema
                .fields()
                .get(index)
                .ok_or_else(|| {
                    SinkError::ElasticSearchOpenSearch(anyhow!(
                        "no value find in sink schema, index is {:?}",
                        index
                    ))
                })?
                .name;
            format!("{{{}}}", name)
        } else {
            let delimiter = delimiter
                .as_ref()
                .ok_or_else(|| anyhow!("please set the separator in the with option, when there are multiple primary key values"))?
                .clone();
            let mut names = Vec::with_capacity(pk_indices.len());
            for index in &pk_indices {
                names.push(format!(
                    "{{{}}}",
                    schema
                        .fields()
                        .get(*index)
                        .ok_or_else(|| {
                            SinkError::ElasticSearchOpenSearch(anyhow!(
                                "no value find in sink schema, index is {:?}",
                                index
                            ))
                        })?
                        .name
                ));
            }
            names.join(&delimiter)
        };
        let col_indices = if let Some(index) = index_column {
            let mut col_indices: Vec<usize> = (0..schema.len()).collect();
            col_indices.remove(index);
            Some(col_indices)
        } else {
            None
        };
        let key_encoder = TemplateEncoder::new(schema.clone(), col_indices.clone(), key_format);
        let value_encoder = JsonEncoder::new_with_es(schema.clone(), col_indices.clone());
        Ok(Self {
            key_encoder,
            value_encoder,
            index_column,
            index,
            routing_column,
        })
    }

    pub fn convert_chunk(&self, chunk: StreamChunk) -> Result<Vec<BuildBulkPara>> {
        let mut result_vec = Vec::with_capacity(chunk.capacity());
        for (op, rows) in chunk.rows() {
            let index = if let Some(index_column) = self.index_column {
                rows.datum_at(index_column)
                    .ok_or_else(|| {
                        SinkError::ElasticSearchOpenSearch(anyhow!(
                            "no value find in sink schema, index is {:?}",
                            index_column
                        ))
                    })?
                    .into_utf8()
            } else {
                self.index.as_ref().unwrap()
            };
            let routing_column = self
                .routing_column
                .map(|routing_column| {
                    Ok::<String, SinkError>(
                        rows.datum_at(routing_column)
                            .ok_or_else(|| {
                                SinkError::ElasticSearchOpenSearch(anyhow!(
                                    "no value find in sink schema, index is {:?}",
                                    routing_column
                                ))
                            })?
                            .into_utf8()
                            .to_string(),
                    )
                })
                .transpose()?;
            match op {
                Op::Insert | Op::UpdateInsert => {
                    let key = self.key_encoder.encode(rows)?;
                    let value = self.value_encoder.encode(rows)?;
                    result_vec.push(BuildBulkPara {
                        index: index.to_string(),
                        key,
                        value: Some(value),
                        mem_size_b: rows.value_estimate_size(),
                        routing_column,
                    });
                }
                Op::Delete => {
                    let key = self.key_encoder.encode(rows)?;
                    let mem_size_b = std::mem::size_of_val(&key);
                    result_vec.push(BuildBulkPara {
                        index: index.to_string(),
                        key,
                        value: None,
                        mem_size_b,
                        routing_column,
                    });
                }
                Op::UpdateDelete => continue,
            }
        }
        Ok(result_vec)
    }
}