Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

refactor(types): spilt timestamp to timestamp and timestampnano #20192

Open
wants to merge 17 commits into
base: main
Choose a base branch
from
Next Next commit
save
xxhZs committed Jan 15, 2025

Verified

This commit was created on GitHub.com and signed with GitHub’s verified signature.
commit afd01eb9b06c8f00b1b2b499554d0ffc179e86c9
2 changes: 2 additions & 0 deletions proto/data.proto
Original file line number Diff line number Diff line change
@@ -53,6 +53,7 @@ message DataType {
SERIAL = 19;
INT256 = 20;
MAP = 21;
TIMESTAMP_NANO = 22;
}
TypeName type_name = 1;
// Data length for char.
@@ -104,6 +105,7 @@ enum ArrayType {
SERIAL = 17;
INT256 = 18;
MAP = 20;
TIMESTAMP_NANO = 21;
}

message Array {
50 changes: 50 additions & 0 deletions src/common/src/array/arrow/arrow_impl.rs
Original file line number Diff line number Diff line change
@@ -107,6 +107,7 @@ pub trait ToArrow {
ArrayImpl::Date(array) => self.date_to_arrow(array),
ArrayImpl::Time(array) => self.time_to_arrow(array),
ArrayImpl::Timestamp(array) => self.timestamp_to_arrow(array),
ArrayImpl::TimestampNano(array) => self.timestampnano_to_arrow(array),
ArrayImpl::Timestamptz(array) => self.timestamptz_to_arrow(array),
ArrayImpl::Interval(array) => self.interval_to_arrow(array),
ArrayImpl::Utf8(array) => self.utf8_to_arrow(array),
@@ -180,6 +181,16 @@ pub trait ToArrow {
)))
}

#[inline]
fn timestampnano_to_arrow(
&self,
array: &TimestampNanoArray,
) -> Result<arrow_array::ArrayRef, ArrayError> {
Ok(Arc::new(arrow_array::TimestampNanosecondArray::from(
array,
)))
}

#[inline]
fn timestamptz_to_arrow(
&self,
@@ -319,6 +330,7 @@ pub trait ToArrow {
DataType::Date => self.date_type_to_arrow(),
DataType::Time => self.time_type_to_arrow(),
DataType::Timestamp => self.timestamp_type_to_arrow(),
DataType::TimestampNano => self.timestampnano_type_to_arrow(),
DataType::Timestamptz => self.timestamptz_type_to_arrow(),
DataType::Interval => self.interval_type_to_arrow(),
DataType::Varchar => self.varchar_type_to_arrow(),
@@ -382,6 +394,10 @@ pub trait ToArrow {
fn timestamp_type_to_arrow(&self) -> arrow_schema::DataType {
arrow_schema::DataType::Timestamp(arrow_schema::TimeUnit::Microsecond, None)
}
#[inline]
fn timestampnano_type_to_arrow(&self) -> arrow_schema::DataType {
arrow_schema::DataType::Timestamp(arrow_schema::TimeUnit::Nanosecond, None)
}

#[inline]
fn timestamptz_type_to_arrow(&self) -> arrow_schema::DataType {
@@ -1046,6 +1062,11 @@ converts_with_timeunit!(TimestamptzArray, arrow_array::TimestampMillisecondArray
converts_with_timeunit!(TimestamptzArray, arrow_array::TimestampMicrosecondArray, TimeUnit::Microsecond, @map);
converts_with_timeunit!(TimestamptzArray, arrow_array::TimestampNanosecondArray, TimeUnit::Nanosecond, @map);

converts_with_timeunit!(TimestampNanoArray, arrow_array::TimestampSecondArray, TimeUnit::Second, @map);
converts_with_timeunit!(TimestampNanoArray, arrow_array::TimestampMillisecondArray, TimeUnit::Millisecond, @map);
converts_with_timeunit!(TimestampNanoArray, arrow_array::TimestampMicrosecondArray, TimeUnit::Microsecond, @map);
converts_with_timeunit!(TimestampNanoArray, arrow_array::TimestampNanosecondArray, TimeUnit::Nanosecond, @map);

/// Converts RisingWave value from and into Arrow value.
trait FromIntoArrow {
/// The corresponding element type in the Arrow array.
@@ -1162,6 +1183,35 @@ impl FromIntoArrowWithUnit for Timestamp {
}
}

impl FromIntoArrowWithUnit for TimestampNano {
type ArrowType = i64;
type TimestampType = TimeUnit;

fn from_arrow_with_unit(value: Self::ArrowType, time_unit: Self::TimestampType) -> Self {
match time_unit {
TimeUnit::Second => {
TimestampNano(DateTime::from_timestamp(value as _, 0).unwrap().naive_utc())
}
TimeUnit::Millisecond => {
TimestampNano(DateTime::from_timestamp_millis(value).unwrap().naive_utc())
}
TimeUnit::Microsecond => {
TimestampNano(DateTime::from_timestamp_micros(value).unwrap().naive_utc())
}
TimeUnit::Nanosecond => TimestampNano(DateTime::from_timestamp_nanos(value).naive_utc()),
}
}

fn into_arrow_with_unit(self, time_unit: Self::TimestampType) -> Self::ArrowType {
match time_unit {
TimeUnit::Second => self.0.and_utc().timestamp(),
TimeUnit::Millisecond => self.0.and_utc().timestamp_millis(),
TimeUnit::Microsecond => self.0.and_utc().timestamp_micros(),
TimeUnit::Nanosecond => self.0.and_utc().timestamp_nanos_opt().unwrap(),
}
}
}

impl FromIntoArrowWithUnit for Timestamptz {
type ArrowType = i64;
type TimestampType = TimeUnit;
4 changes: 3 additions & 1 deletion src/common/src/array/chrono_array.rs
Original file line number Diff line number Diff line change
@@ -12,17 +12,19 @@
// See the License for the specific language governing permissions and
// limitations under the License.

use super::{PrimitiveArray, PrimitiveArrayBuilder};
use super::{PrimitiveArray, PrimitiveArrayBuilder, TimestampNano};
use crate::types::{Date, Time, Timestamp, Timestamptz};

pub type DateArray = PrimitiveArray<Date>;
pub type TimeArray = PrimitiveArray<Time>;
pub type TimestampArray = PrimitiveArray<Timestamp>;
pub type TimestampNanoArray = PrimitiveArray<TimestampNano>;
pub type TimestamptzArray = PrimitiveArray<Timestamptz>;

pub type DateArrayBuilder = PrimitiveArrayBuilder<Date>;
pub type TimeArrayBuilder = PrimitiveArrayBuilder<Time>;
pub type TimestampArrayBuilder = PrimitiveArrayBuilder<Timestamp>;
pub type TimestampNanoArrayBuilder = PrimitiveArrayBuilder<TimestampNano>;
pub type TimestamptzArrayBuilder = PrimitiveArrayBuilder<Timestamptz>;

#[cfg(test)]
3 changes: 2 additions & 1 deletion src/common/src/array/mod.rs
Original file line number Diff line number Diff line change
@@ -45,7 +45,8 @@ pub use bool_array::{BoolArray, BoolArrayBuilder};
pub use bytes_array::*;
pub use chrono_array::{
DateArray, DateArrayBuilder, TimeArray, TimeArrayBuilder, TimestampArray,
TimestampArrayBuilder, TimestamptzArray, TimestamptzArrayBuilder,
TimestampArrayBuilder,TimestampNanoArray,
TimestampNanoArrayBuilder, TimestamptzArray, TimestamptzArrayBuilder,
};
pub use data_chunk::{DataChunk, DataChunkTestExt};
pub use data_chunk_iter::RowRef;
1 change: 1 addition & 0 deletions src/common/src/array/primitive_array.rs
Original file line number Diff line number Diff line change
@@ -129,6 +129,7 @@ impl_primitive_for_others! {
{ Date, Date, Date },
{ Time, Time, Time },
{ Timestamp, Timestamp, Timestamp },
{ TimestampNano, TimestampNano, TimestampNano },
{ Timestamptz, Timestamptz, Timestamptz }
}

1 change: 1 addition & 0 deletions src/common/src/array/proto_reader.rs
Original file line number Diff line number Diff line change
@@ -36,6 +36,7 @@ impl ArrayImpl {
PbArrayType::Date => read_primitive_array::<Date>(array, cardinality)?,
PbArrayType::Time => read_primitive_array::<Time>(array, cardinality)?,
PbArrayType::Timestamp => read_primitive_array::<Timestamp>(array, cardinality)?,
PbArrayType::TimestampNano => read_primitive_array::<TimestampNano>(array, cardinality)?,
PbArrayType::Timestamptz => read_primitive_array::<Timestamptz>(array, cardinality)?,
PbArrayType::Interval => read_primitive_array::<Interval>(array, cardinality)?,
PbArrayType::Jsonb => JsonbArray::from_protobuf(array)?,
22 changes: 20 additions & 2 deletions src/common/src/hash/key.rs
Original file line number Diff line number Diff line change
@@ -35,8 +35,7 @@ use static_assertions::const_assert_eq;

use crate::array::{ListValue, MapValue, StructValue};
use crate::types::{
DataType, Date, Decimal, Int256, Int256Ref, JsonbVal, Scalar, ScalarRef, ScalarRefImpl, Serial,
Time, Timestamp, Timestamptz, F32, F64,
DataType, Date, Decimal, Int256, Int256Ref, JsonbVal, Scalar, ScalarRef, ScalarRefImpl, Serial, Time, Timestamp, TimestampNano, Timestamptz, F32, F64
};
use crate::util::hash_util::{Crc32FastBuilder, XxHash64Builder};
use crate::util::sort_util::OrderType;
@@ -584,6 +583,25 @@ impl HashKeyDe for Timestamp {
}
}

impl HashKeySer<'_> for TimestampNano {
fn serialize_into(self, mut buf: impl BufMut) {
buf.put_i64_ne(self.0.and_utc().timestamp());
buf.put_u32_ne(self.0.and_utc().timestamp_subsec_nanos());
}

fn exact_size() -> Option<usize> {
Some(12)
}
}

impl HashKeyDe for TimestampNano {
fn deserialize(_data_type: &DataType, mut buf: impl Buf) -> Self {
let secs = buf.get_i64_ne();
let nsecs = buf.get_u32_ne();
TimestampNano::with_secs_nsecs(secs, nsecs).unwrap()
}
}

impl HashKeySer<'_> for Time {
fn serialize_into(self, mut buf: impl BufMut) {
buf.put_u32_ne(self.0.num_seconds_from_midnight());
9 changes: 7 additions & 2 deletions src/common/src/test_utils/rand_array.rs
Original file line number Diff line number Diff line change
@@ -26,8 +26,7 @@ use rand::{Rng, SeedableRng};

use crate::array::{Array, ArrayBuilder, ArrayRef, ListValue, MapValue, StructValue};
use crate::types::{
DataType, Date, Decimal, Int256, Interval, JsonbVal, MapType, NativeType, Scalar, Serial, Time,
Timestamp, Timestamptz,
DataType, Date, Decimal, Int256, Interval, JsonbVal, MapType, NativeType, Scalar, Serial, Time, Timestamp, TimestampNano, Timestamptz
};

pub trait RandValue {
@@ -106,6 +105,12 @@ impl RandValue for Timestamp {
}
}

impl RandValue for TimestampNano {
fn rand_value<R: Rng>(rand: &mut R) -> Self {
TimestampNano::new(Date::rand_value(rand).0.and_time(Time::rand_value(rand).0))
}
}

impl RandValue for Timestamptz {
fn rand_value<R: Rng>(rand: &mut R) -> Self {
Timestamptz::from_micros(rand.gen())
7 changes: 4 additions & 3 deletions src/common/src/test_utils/rand_chunk.rs
Original file line number Diff line number Diff line change
@@ -13,9 +13,7 @@
// limitations under the License.

use crate::array::{
BoolArray, DataChunk, DateArray, DecimalArray, F32Array, F64Array, I16Array, I32Array,
I64Array, Int256Array, IntervalArray, SerialArray, TimeArray, TimestampArray, TimestamptzArray,
Utf8Array,
BoolArray, DataChunk, DateArray, DecimalArray, F32Array, F64Array, I16Array, I32Array, I64Array, Int256Array, IntervalArray, SerialArray, TimeArray, TimestampArray, TimestampNanoArray, TimestamptzArray, Utf8Array
};
use crate::test_utils::rand_array::seed_rand_array_ref;
use crate::types::DataType;
@@ -38,6 +36,9 @@ pub fn gen_chunk(data_types: &[DataType], size: usize, seed: u64, null_ratio: f6
DataType::Time => seed_rand_array_ref::<TimeArray>(size, seed, null_ratio),
DataType::Serial => seed_rand_array_ref::<SerialArray>(size, seed, null_ratio),
DataType::Timestamp => seed_rand_array_ref::<TimestampArray>(size, seed, null_ratio),
DataType::TimestampNano => {
seed_rand_array_ref::<TimestampNanoArray>(size, seed, null_ratio)
}
DataType::Timestamptz => {
seed_rand_array_ref::<TimestamptzArray>(size, seed, null_ratio)
}
Loading