opensrv_clickhouse/types/column/
string.rs

1// Copyright 2021 Datafuse Labs.
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use std::io::Write;
16use std::string::ToString;
17use std::sync::Arc;
18
19use super::column_data::BoxColumnData;
20use super::column_data::ColumnData;
21use super::ColumnFrom;
22use crate::binary::Encoder;
23use crate::binary::ReadEx;
24use crate::errors::Result;
25use crate::types::column::array::ArrayColumnData;
26use crate::types::column::list::List;
27use crate::types::column::nullable::NullableColumnData;
28use crate::types::column::ArcColumnWrapper;
29use crate::types::column::ColumnWrapper;
30use crate::types::column::Either;
31use crate::types::column::StringPool;
32use crate::types::Column;
33use crate::types::ColumnType;
34use crate::types::FromSql;
35use crate::types::SqlType;
36use crate::types::Value;
37use crate::types::ValueRef;
38
39pub struct StringColumnData {
40    pool: StringPool,
41}
42
43pub(crate) struct StringAdapter<K: ColumnType> {
44    pub(crate) column: Column<K>,
45}
46
47impl StringColumnData {
48    pub fn with_capacity(capacity: usize) -> Self {
49        Self {
50            pool: StringPool::with_capacity(capacity),
51        }
52    }
53
54    pub fn load<T: ReadEx>(reader: &mut T, size: usize) -> Result<Self> {
55        let mut data = Self::with_capacity(size);
56
57        for _ in 0..size {
58            reader.read_str_into_buffer(&mut data.pool)?;
59        }
60
61        Ok(data)
62    }
63}
64
65impl ColumnFrom for Vec<String> {
66    fn column_from<W: ColumnWrapper>(data: Self) -> W::Wrapper {
67        W::wrap(StringColumnData { pool: data.into() })
68    }
69}
70
71impl<'a> ColumnFrom for Vec<&'a str> {
72    fn column_from<W: ColumnWrapper>(source: Self) -> W::Wrapper {
73        let data: Vec<_> = source.iter().map(ToString::to_string).collect();
74        W::wrap(StringColumnData { pool: data.into() })
75    }
76}
77
78impl<'a> ColumnFrom for Vec<&'a [u8]> {
79    fn column_from<W: ColumnWrapper>(data: Self) -> W::Wrapper {
80        W::wrap(StringColumnData { pool: data.into() })
81    }
82}
83
84trait StringSource {
85    fn into_value(self) -> Value;
86}
87
88impl StringSource for String {
89    fn into_value(self) -> Value {
90        self.into()
91    }
92}
93
94impl StringSource for &str {
95    fn into_value(self) -> Value {
96        self.into()
97    }
98}
99
100impl StringSource for Vec<u8> {
101    fn into_value(self) -> Value {
102        Value::String(Arc::new(self))
103    }
104}
105
106impl ColumnFrom for Vec<Vec<String>> {
107    fn column_from<W: ColumnWrapper>(source: Self) -> <W as ColumnWrapper>::Wrapper {
108        make_array_of_array::<W, String>(source)
109    }
110}
111
112impl ColumnFrom for Vec<Vec<&str>> {
113    fn column_from<W: ColumnWrapper>(source: Self) -> <W as ColumnWrapper>::Wrapper {
114        make_array_of_array::<W, &str>(source)
115    }
116}
117
118fn make_array_of_array<W: ColumnWrapper, S: StringSource>(
119    source: Vec<Vec<S>>,
120) -> <W as ColumnWrapper>::Wrapper {
121    let fake: Vec<String> = Vec::with_capacity(source.len());
122    let inner = Vec::column_from::<ArcColumnWrapper>(fake);
123    let sql_type = inner.sql_type();
124
125    let mut data = ArrayColumnData {
126        inner,
127        offsets: List::with_capacity(source.len()),
128    };
129
130    for vs in source {
131        let mut inner = Vec::with_capacity(vs.len());
132        for v in vs {
133            let value: Value = v.into_value();
134            inner.push(value)
135        }
136        data.push(Value::Array(sql_type.clone().into(), Arc::new(inner)));
137    }
138
139    W::wrap(data)
140}
141
142impl ColumnFrom for Vec<Option<Vec<u8>>> {
143    fn column_from<W: ColumnWrapper>(source: Self) -> W::Wrapper {
144        make_opt_column::<W, Vec<u8>>(source)
145    }
146}
147
148impl ColumnFrom for Vec<Option<&str>> {
149    fn column_from<W: ColumnWrapper>(source: Self) -> W::Wrapper {
150        make_opt_column::<W, &str>(source)
151    }
152}
153
154impl ColumnFrom for Vec<Option<String>> {
155    fn column_from<W: ColumnWrapper>(source: Self) -> W::Wrapper {
156        make_opt_column::<W, String>(source)
157    }
158}
159
160fn make_opt_column<W: ColumnWrapper, S: StringSource>(source: Vec<Option<S>>) -> W::Wrapper {
161    let inner = Arc::new(StringColumnData::with_capacity(source.len()));
162
163    let mut data = NullableColumnData {
164        inner,
165        nulls: Vec::with_capacity(source.len()),
166    };
167
168    for value in source {
169        let item = if let Some(v) = value {
170            let inner = v.into_value();
171            Value::Nullable(Either::Right(Box::new(inner)))
172        } else {
173            Value::Nullable(Either::Left(SqlType::String.into()))
174        };
175        data.push(item);
176    }
177
178    W::wrap(data)
179}
180
181impl ColumnData for StringColumnData {
182    fn sql_type(&self) -> SqlType {
183        SqlType::String
184    }
185
186    fn save(&self, encoder: &mut Encoder, start: usize, end: usize) {
187        let strings = self.pool.strings();
188        for v in strings.skip(start).take(end - start) {
189            encoder.byte_string(v);
190        }
191    }
192
193    fn len(&self) -> usize {
194        self.pool.len()
195    }
196
197    fn push(&mut self, value: Value) {
198        let s: Vec<u8> = value.into();
199        let mut b = self.pool.allocate(s.len());
200        b.write_all(s.as_ref()).unwrap();
201    }
202
203    fn at(&self, index: usize) -> ValueRef {
204        let s = self.pool.get(index);
205        ValueRef::from(s)
206    }
207
208    fn clone_instance(&self) -> BoxColumnData {
209        Box::new(Self {
210            pool: self.pool.clone(),
211        })
212    }
213
214    unsafe fn get_internal(&self, pointers: &[*mut *const u8], level: u8) -> Result<()> {
215        assert_eq!(level, 0);
216        *pointers[0] = &self.pool as *const StringPool as *const u8;
217        *(pointers[1] as *mut usize) = self.len();
218        Ok(())
219    }
220}
221
222impl<K: ColumnType> ColumnData for StringAdapter<K> {
223    fn sql_type(&self) -> SqlType {
224        SqlType::String
225    }
226
227    fn save(&self, encoder: &mut Encoder, start: usize, end: usize) {
228        for index in start..end {
229            let buf: Vec<u8> = Vec::from_sql(self.column.at(index)).unwrap();
230            encoder.byte_string(buf);
231        }
232    }
233
234    fn len(&self) -> usize {
235        self.column.len()
236    }
237
238    fn push(&mut self, _value: Value) {
239        unimplemented!()
240    }
241
242    fn at(&self, index: usize) -> ValueRef {
243        self.column.at(index)
244    }
245
246    fn clone_instance(&self) -> BoxColumnData {
247        unimplemented!()
248    }
249}