Skip to content

Commit

Permalink
feat(datagen): support timestamptz (#13451)
Browse files Browse the repository at this point in the history
  • Loading branch information
xiangjinwu authored Nov 16, 2023
1 parent 53d1956 commit c6dffdf
Show file tree
Hide file tree
Showing 5 changed files with 73 additions and 10 deletions.
6 changes: 5 additions & 1 deletion e2e_test/source/basic/datagen.slt
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ statement ok
drop table s1;

statement ok
create table s1 (v1 struct<v2 int>, t1 timestamp, c1 varchar) with (
create table s1 (v1 struct<v2 int>, t1 timestamp, z1 timestamptz, c1 varchar) with (
connector = 'datagen',
fields.v1.v2.kind = 'random',
fields.v1.v2.min = '1',
Expand All @@ -112,6 +112,10 @@ create table s1 (v1 struct<v2 int>, t1 timestamp, c1 varchar) with (
fields.t1.max_past = '2h 37min',
fields.t1.max_past_mode = 'relative',
fields.t1.seed = '3',
fields.z1.kind = 'random',
fields.z1.max_past = '2h 37min',
fields.z1.max_past_mode = 'relative',
fields.z1.seed = '3',
fields.c1.kind = 'random',
fields.c1.length = '100',
fields.c1.seed = '3',
Expand Down
7 changes: 7 additions & 0 deletions src/common/src/array/data_chunk.rs
Original file line number Diff line number Diff line change
Expand Up @@ -919,6 +919,13 @@ impl DataChunkTestExt for DataChunk {
.generate_datum(offset);
array_builder.append(datum);
}
DataType::Timestamptz => {
let datum =
FieldGeneratorImpl::with_timestamptz(None, None, None, Self::SEED)
.expect("create timestamptz generator should succeed")
.generate_datum(offset);
array_builder.append(datum);
}
_ if data_type.is_numeric() => {
let mut data_gen = FieldGeneratorImpl::with_number_random(
data_type.clone(),
Expand Down
23 changes: 22 additions & 1 deletion src/common/src/field_generator/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ pub use timestamp::*;
pub use varchar::*;

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

pub const DEFAULT_MIN: i16 = i16::MIN;
pub const DEFAULT_MAX: i16 = i16::MAX;
Expand Down Expand Up @@ -96,6 +96,7 @@ pub enum FieldGeneratorImpl {
VarcharRandomFixedLength(VarcharRandomFixedLengthField),
VarcharConstant,
Timestamp(ChronoField<Timestamp>),
Timestamptz(ChronoField<Timestamptz>),
Struct(Vec<(String, FieldGeneratorImpl)>),
List(Box<FieldGeneratorImpl>, usize),
}
Expand Down Expand Up @@ -189,6 +190,20 @@ impl FieldGeneratorImpl {
)?))
}

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) => {
Expand Down Expand Up @@ -235,6 +250,7 @@ impl FieldGeneratorImpl {
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()
Expand Down Expand Up @@ -267,6 +283,7 @@ impl FieldGeneratorImpl {
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()
Expand Down Expand Up @@ -328,6 +345,7 @@ mod tests {
DataType::Float64,
DataType::Varchar,
DataType::Timestamp,
DataType::Timestamptz,
] {
let mut generator = match data_type {
DataType::Varchar => FieldGeneratorImpl::with_varchar(
Expand All @@ -337,6 +355,9 @@ mod tests {
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(),
};

Expand Down
24 changes: 23 additions & 1 deletion src/common/src/field_generator/timestamp.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ use serde_json::Value;
use tracing::debug;

use super::DEFAULT_MAX_PAST;
use crate::types::{Datum, Scalar, Timestamp};
use crate::types::{Datum, Scalar, Timestamp, Timestamptz};

pub struct ChronoField<T: ChronoFieldInner> {
max_past: Duration,
Expand Down Expand Up @@ -106,3 +106,25 @@ impl ChronoFieldInner for Timestamp {
Value::String(self.0.to_string())
}
}

impl ChronoFieldInner for Timestamptz {
fn from_now() -> Self {
Timestamptz::from(
Utc::now()
.duration_round(Duration::microseconds(1))
.unwrap(),
)
}

fn from_base(base: DateTime<FixedOffset>) -> Self {
Timestamptz::from(base)
}

fn minus(&self, duration: Duration) -> Self {
Timestamptz::from(self.to_datetime_utc() - duration)
}

fn to_json(&self) -> Value {
Value::String(self.to_string())
}
}
23 changes: 16 additions & 7 deletions src/connector/src/source/datagen/source/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -213,7 +213,7 @@ fn generator_from_data_type(
None => split_index,
};
match data_type {
DataType::Timestamp => {
ty @ (DataType::Timestamp | DataType::Timestamptz) => {
let max_past_key = format!("fields.{}.max_past", name);
let max_past_value = fields_option_map.get(&max_past_key).cloned();
let max_past_mode_key = format!("fields.{}.max_past_mode", name);
Expand All @@ -230,12 +230,21 @@ fn generator_from_data_type(
None => None,
};

FieldGeneratorImpl::with_timestamp(
basetime,
max_past_value,
max_past_mode_value,
random_seed,
)
if ty == DataType::Timestamptz {
FieldGeneratorImpl::with_timestamptz(
basetime,
max_past_value,
max_past_mode_value,
random_seed,
)
} else {
FieldGeneratorImpl::with_timestamp(
basetime,
max_past_value,
max_past_mode_value,
random_seed,
)
}
}
DataType::Varchar => {
let length_key = format!("fields.{}.length", name);
Expand Down

0 comments on commit c6dffdf

Please sign in to comment.