risingwave_common/field_generator/
mod.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
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
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
// 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.

mod numeric;
mod timestamp;
mod varchar;

use std::time::Duration;

// TODO(error-handling): use a new error type
use anyhow::{anyhow, Result};
use chrono::{DateTime, FixedOffset};
pub use numeric::*;
use serde_json::Value;
pub use timestamp::*;
pub use varchar::*;

use crate::array::{ListValue, StructValue};
use crate::types::{DataType, Datum, ScalarImpl, Timestamp, Timestamptz};

pub const DEFAULT_MIN: i16 = i16::MIN;
pub const DEFAULT_MAX: i16 = i16::MAX;
pub const DEFAULT_START: i16 = 0;
pub const DEFAULT_END: i16 = i16::MAX;

/// default max past for `TimestampField` = 1 day
pub const DEFAULT_MAX_PAST: Duration = Duration::from_secs(60 * 60 * 24);

/// default length for `VarcharField` = 10
pub const DEFAULT_LENGTH: usize = 10;

/// fields that can be randomly generated impl this trait
pub trait NumericFieldRandomGenerator {
    fn new(min: Option<String>, max: Option<String>, seed: u64) -> Result<Self>
    where
        Self: Sized;

    fn generate(&mut self, offset: u64) -> Value;

    fn generate_datum(&mut self, offset: u64) -> Datum;
}

/// fields that can be continuously generated impl this trait
pub trait NumericFieldSequenceGenerator {
    fn new(
        start: Option<String>,
        end: Option<String>,
        offset: u64,
        step: u64,
        event_offset: u64,
    ) -> Result<Self>
    where
        Self: Sized;

    fn generate(&mut self) -> Value;

    fn generate_datum(&mut self) -> Datum;
}

/// the way that datagen create the field data. such as 'sequence' or 'random'.
#[derive(Default)]
pub enum FieldKind {
    Sequence,
    #[default]
    Random,
}

pub enum VarcharProperty {
    RandomVariableLength,
    RandomFixedLength(Option<usize>),
    Constant,
}

pub enum FieldGeneratorImpl {
    I16Sequence(I16SequenceField),
    I32Sequence(I32SequenceField),
    I64Sequence(I64SequenceField),
    F32Sequence(F32SequenceField),
    F64Sequence(F64SequenceField),
    I16Random(I16RandomField),
    I32Random(I32RandomField),
    I64Random(I64RandomField),
    F32Random(F32RandomField),
    F64Random(F64RandomField),
    VarcharRandomVariableLength(VarcharRandomVariableLengthField),
    VarcharRandomFixedLength(VarcharRandomFixedLengthField),
    VarcharConstant,
    Timestamp(ChronoField<Timestamp>),
    Timestamptz(ChronoField<Timestamptz>),
    Struct(Vec<(String, FieldGeneratorImpl)>),
    List(Box<FieldGeneratorImpl>, usize),
}

impl FieldGeneratorImpl {
    pub fn with_number_sequence(
        data_type: DataType,
        start: Option<String>,
        end: Option<String>,
        split_index: u64,
        split_num: u64,
        offset: u64,
    ) -> Result<Self> {
        match data_type {
            DataType::Int16 => Ok(FieldGeneratorImpl::I16Sequence(I16SequenceField::new(
                start,
                end,
                split_index,
                split_num,
                offset,
            )?)),
            DataType::Int32 => Ok(FieldGeneratorImpl::I32Sequence(I32SequenceField::new(
                start,
                end,
                split_index,
                split_num,
                offset,
            )?)),
            DataType::Int64 => Ok(FieldGeneratorImpl::I64Sequence(I64SequenceField::new(
                start,
                end,
                split_index,
                split_num,
                offset,
            )?)),
            DataType::Float32 => Ok(FieldGeneratorImpl::F32Sequence(F32SequenceField::new(
                start,
                end,
                split_index,
                split_num,
                offset,
            )?)),
            DataType::Float64 => Ok(FieldGeneratorImpl::F64Sequence(F64SequenceField::new(
                start,
                end,
                split_index,
                split_num,
                offset,
            )?)),
            _ => Err(anyhow!("unimplemented field generator {}", data_type)),
        }
    }

    pub fn with_number_random(
        data_type: DataType,
        min: Option<String>,
        max: Option<String>,
        seed: u64,
    ) -> Result<Self> {
        match data_type {
            DataType::Int16 => Ok(FieldGeneratorImpl::I16Random(I16RandomField::new(
                min, max, seed,
            )?)),
            DataType::Int32 => Ok(FieldGeneratorImpl::I32Random(I32RandomField::new(
                min, max, seed,
            )?)),
            DataType::Int64 => Ok(FieldGeneratorImpl::I64Random(I64RandomField::new(
                min, max, seed,
            )?)),
            DataType::Float32 => Ok(FieldGeneratorImpl::F32Random(F32RandomField::new(
                min, max, seed,
            )?)),
            DataType::Float64 => Ok(FieldGeneratorImpl::F64Random(F64RandomField::new(
                min, max, seed,
            )?)),
            _ => Err(anyhow!("unimplemented field generator {}", data_type)),
        }
    }

    pub fn with_timestamp(
        base: Option<DateTime<FixedOffset>>,
        max_past: Option<String>,
        max_past_mode: Option<String>,
        seed: u64,
    ) -> Result<Self> {
        Ok(FieldGeneratorImpl::Timestamp(ChronoField::new(
            base,
            max_past,
            max_past_mode,
            seed,
        )?))
    }

    pub fn with_timestamptz(
        base: Option<DateTime<FixedOffset>>,
        max_past: Option<String>,
        max_past_mode: Option<String>,
        seed: u64,
    ) -> Result<Self> {
        Ok(FieldGeneratorImpl::Timestamptz(ChronoField::new(
            base,
            max_past,
            max_past_mode,
            seed,
        )?))
    }

    pub fn with_varchar(varchar_property: &VarcharProperty, seed: u64) -> Self {
        match varchar_property {
            VarcharProperty::RandomFixedLength(length_option) => {
                FieldGeneratorImpl::VarcharRandomFixedLength(VarcharRandomFixedLengthField::new(
                    length_option,
                    seed,
                ))
            }
            VarcharProperty::RandomVariableLength => {
                FieldGeneratorImpl::VarcharRandomVariableLength(
                    VarcharRandomVariableLengthField::new(seed),
                )
            }
            VarcharProperty::Constant => FieldGeneratorImpl::VarcharConstant,
        }
    }

    pub fn with_struct_fields(fields: Vec<(String, FieldGeneratorImpl)>) -> Result<Self> {
        Ok(FieldGeneratorImpl::Struct(fields))
    }

    pub fn with_list(field: FieldGeneratorImpl, length_option: Option<String>) -> Result<Self> {
        let list_length = if let Some(length_option) = length_option {
            length_option.parse::<usize>()?
        } else {
            DEFAULT_LENGTH
        };
        Ok(FieldGeneratorImpl::List(Box::new(field), list_length))
    }

    pub fn generate_json(&mut self, offset: u64) -> Value {
        match self {
            FieldGeneratorImpl::I16Sequence(f) => f.generate(),
            FieldGeneratorImpl::I32Sequence(f) => f.generate(),
            FieldGeneratorImpl::I64Sequence(f) => f.generate(),
            FieldGeneratorImpl::F32Sequence(f) => f.generate(),
            FieldGeneratorImpl::F64Sequence(f) => f.generate(),
            FieldGeneratorImpl::I16Random(f) => f.generate(offset),
            FieldGeneratorImpl::I32Random(f) => f.generate(offset),
            FieldGeneratorImpl::I64Random(f) => f.generate(offset),
            FieldGeneratorImpl::F32Random(f) => f.generate(offset),
            FieldGeneratorImpl::F64Random(f) => f.generate(offset),
            FieldGeneratorImpl::VarcharRandomFixedLength(f) => f.generate(offset),
            FieldGeneratorImpl::VarcharRandomVariableLength(f) => f.generate(offset),
            FieldGeneratorImpl::VarcharConstant => VarcharConstant::generate_json(),
            FieldGeneratorImpl::Timestamp(f) => f.generate(offset),
            FieldGeneratorImpl::Timestamptz(f) => f.generate(offset),
            FieldGeneratorImpl::Struct(fields) => {
                let map = fields
                    .iter_mut()
                    .map(|(name, gen)| (name.clone(), gen.generate_json(offset)))
                    .collect();
                Value::Object(map)
            }
            FieldGeneratorImpl::List(field, list_length) => {
                let vec = (0..*list_length)
                    .map(|_| field.generate_json(offset))
                    .collect::<Vec<_>>();
                Value::Array(vec)
            }
        }
    }

    pub fn generate_datum(&mut self, offset: u64) -> Datum {
        match self {
            FieldGeneratorImpl::I16Sequence(f) => f.generate_datum(),
            FieldGeneratorImpl::I32Sequence(f) => f.generate_datum(),
            FieldGeneratorImpl::I64Sequence(f) => f.generate_datum(),
            FieldGeneratorImpl::F32Sequence(f) => f.generate_datum(),
            FieldGeneratorImpl::F64Sequence(f) => f.generate_datum(),
            FieldGeneratorImpl::I16Random(f) => f.generate_datum(offset),
            FieldGeneratorImpl::I32Random(f) => f.generate_datum(offset),
            FieldGeneratorImpl::I64Random(f) => f.generate_datum(offset),
            FieldGeneratorImpl::F32Random(f) => f.generate_datum(offset),
            FieldGeneratorImpl::F64Random(f) => f.generate_datum(offset),
            FieldGeneratorImpl::VarcharRandomFixedLength(f) => f.generate_datum(offset),
            FieldGeneratorImpl::VarcharRandomVariableLength(f) => f.generate_datum(offset),
            FieldGeneratorImpl::VarcharConstant => VarcharConstant::generate_datum(),
            FieldGeneratorImpl::Timestamp(f) => f.generate_datum(offset),
            FieldGeneratorImpl::Timestamptz(f) => f.generate_datum(offset),
            FieldGeneratorImpl::Struct(fields) => {
                let data = fields
                    .iter_mut()
                    .map(|(_, gen)| gen.generate_datum(offset))
                    .collect();
                Some(ScalarImpl::Struct(StructValue::new(data)))
            }
            FieldGeneratorImpl::List(field, list_length) => {
                Some(ScalarImpl::List(ListValue::from_datum_iter(
                    &field.data_type(),
                    std::iter::repeat_with(|| field.generate_datum(offset)).take(*list_length),
                )))
            }
        }
    }

    fn data_type(&self) -> DataType {
        match self {
            Self::I16Sequence(_) => DataType::Int16,
            Self::I32Sequence(_) => DataType::Int32,
            Self::I64Sequence(_) => DataType::Int64,
            Self::F32Sequence(_) => DataType::Float32,
            Self::F64Sequence(_) => DataType::Float64,
            Self::I16Random(_) => DataType::Int16,
            Self::I32Random(_) => DataType::Int32,
            Self::I64Random(_) => DataType::Int64,
            Self::F32Random(_) => DataType::Float32,
            Self::F64Random(_) => DataType::Float64,
            Self::VarcharRandomFixedLength(_) => DataType::Varchar,
            Self::VarcharRandomVariableLength(_) => DataType::Varchar,
            Self::VarcharConstant => DataType::Varchar,
            Self::Timestamp(_) => DataType::Timestamp,
            Self::Timestamptz(_) => DataType::Timestamptz,
            Self::Struct(_) => todo!("data_type for struct"),
            Self::List(inner, _) => DataType::List(Box::new(inner.data_type())),
        }
    }
}

#[cfg(test)]
mod tests {
    use super::*;

    #[test]
    fn test_partition_sequence() {
        let split_num = 4;
        let mut i32_fields = vec![];
        for split_index in 0..split_num {
            i32_fields.push(
                FieldGeneratorImpl::with_number_sequence(
                    DataType::Int32,
                    Some("1".to_string()),
                    Some("20".to_string()),
                    split_index,
                    split_num,
                    0,
                )
                .unwrap(),
            );
        }

        for step in 0..5 {
            for (index, i32_field) in i32_fields.iter_mut().enumerate() {
                let value = i32_field.generate_json(0);
                assert!(value.is_number());
                let num = value.as_u64();
                let expected_num = split_num * step + 1 + index as u64;
                assert_eq!(expected_num, num.unwrap());
            }
        }
    }

    #[test]
    fn test_random_generate() {
        let seed = 1234;
        for data_type in [
            DataType::Int16,
            DataType::Int32,
            DataType::Int64,
            DataType::Float32,
            DataType::Float64,
            DataType::Varchar,
            DataType::Timestamp,
            DataType::Timestamptz,
        ] {
            let mut generator = match data_type {
                DataType::Varchar => FieldGeneratorImpl::with_varchar(
                    &VarcharProperty::RandomFixedLength(None),
                    seed,
                ),
                DataType::Timestamp => {
                    FieldGeneratorImpl::with_timestamp(None, None, None, seed).unwrap()
                }
                DataType::Timestamptz => {
                    FieldGeneratorImpl::with_timestamptz(None, None, None, seed).unwrap()
                }
                _ => FieldGeneratorImpl::with_number_random(data_type, None, None, seed).unwrap(),
            };

            let val1 = generator.generate_json(1);
            let val2 = generator.generate_json(2);

            assert_ne!(val1, val2);

            let val1_new = generator.generate_json(1);
            let val2_new = generator.generate_json(2);

            assert_eq!(val1_new, val1);
            assert_eq!(val2_new, val2);

            let datum1 = generator.generate_datum(5);
            let datum2 = generator.generate_datum(7);

            assert_ne!(datum1, datum2);

            let datum1_new = generator.generate_datum(5);
            let datum2_new = generator.generate_datum(7);

            assert_eq!(datum1_new, datum1);
            assert_eq!(datum2_new, datum2);
        }
    }

    #[test]
    fn test_deterministic_timestamp() {
        let seed = 1234;
        let base_time: DateTime<FixedOffset> =
            DateTime::parse_from_rfc3339("2020-01-01T00:00:00+00:00").unwrap();
        let mut generator =
            FieldGeneratorImpl::with_timestamp(Some(base_time), None, None, seed).unwrap();
        let val1 = generator.generate_json(1);
        let val2 = generator.generate_json(1);
        assert_eq!(val1, val2);
    }
}