chore: delete datatypes based on arrow2

This commit is contained in:
Lei, HUANG
2022-12-06 15:01:02 +08:00
parent b48ae21b71
commit 0ccb8b4302
44 changed files with 0 additions and 10488 deletions

View File

@@ -1,24 +0,0 @@
[package]
name = "datatypes2"
version = "0.1.0"
edition = "2021"
license = "Apache-2.0"
[features]
default = []
test = []
[dependencies]
arrow = "26.0.0"
common-base = { path = "../common/base" }
common-error = { path = "../common/error" }
common-time = { path = "../common/time" }
datafusion-common = "14.0.0"
enum_dispatch = "0.3"
num = "0.4"
num-traits = "0.2"
ordered-float = { version = "3.0", features = ["serde"] }
paste = "1.0"
serde = { version = "1.0", features = ["derive"] }
serde_json = "1.0"
snafu = { version = "0.7", features = ["backtraces"] }

View File

@@ -1,216 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 arrow::array::{self, Array, ListArray, PrimitiveArray};
use arrow::datatypes::DataType;
use common_time::timestamp::Timestamp;
use snafu::OptionExt;
use crate::error::{ConversionSnafu, Result};
use crate::prelude::ConcreteDataType;
use crate::value::{ListValue, Value};
pub type BinaryArray = arrow::array::LargeBinaryArray;
pub type MutableBinaryArray = arrow::array::LargeBinaryBuilder;
pub type StringArray = arrow::array::StringArray;
pub type MutableStringArray = arrow::array::StringBuilder;
macro_rules! cast_array {
($arr: ident, $CastType: ty) => {
$arr.as_any()
.downcast_ref::<$CastType>()
.with_context(|| ConversionSnafu {
from: format!("{:?}", $arr.data_type()),
})?
};
}
pub fn arrow_array_get(array: &dyn Array, idx: usize) -> Result<Value> {
if array.is_null(idx) {
return Ok(Value::Null);
}
let result = match array.data_type() {
DataType::Null => Value::Null,
DataType::Boolean => Value::Boolean(cast_array!(array, array::BooleanArray).value(idx)),
DataType::Binary | DataType::LargeBinary => {
Value::Binary(cast_array!(array, BinaryArray).value(idx).into())
}
DataType::Int8 => Value::Int8(cast_array!(array, PrimitiveArray::<i8>).value(idx)),
DataType::Int16 => Value::Int16(cast_array!(array, PrimitiveArray::<i16>).value(idx)),
DataType::Int32 => Value::Int32(cast_array!(array, PrimitiveArray::<i32>).value(idx)),
DataType::Int64 => Value::Int64(cast_array!(array, PrimitiveArray::<i64>).value(idx)),
DataType::UInt8 => Value::UInt8(cast_array!(array, PrimitiveArray::<u8>).value(idx)),
DataType::UInt16 => Value::UInt16(cast_array!(array, PrimitiveArray::<u16>).value(idx)),
DataType::UInt32 => Value::UInt32(cast_array!(array, PrimitiveArray::<u32>).value(idx)),
DataType::UInt64 => Value::UInt64(cast_array!(array, PrimitiveArray::<u64>).value(idx)),
DataType::Float32 => {
Value::Float32(cast_array!(array, PrimitiveArray::<f32>).value(idx).into())
}
DataType::Float64 => {
Value::Float64(cast_array!(array, PrimitiveArray::<f64>).value(idx).into())
}
DataType::Utf8 | DataType::LargeUtf8 => {
Value::String(cast_array!(array, StringArray).value(idx).into())
}
DataType::Timestamp(t, _) => {
let value = cast_array!(array, PrimitiveArray::<i64>).value(idx);
let unit = match ConcreteDataType::from_arrow_time_unit(t) {
ConcreteDataType::Timestamp(t) => t.unit,
_ => unreachable!(),
};
Value::Timestamp(Timestamp::new(value, unit))
}
DataType::List(_) => {
let array = cast_array!(array, ListArray::<i32>).value(idx);
let inner_datatype = ConcreteDataType::try_from(array.data_type())?;
let values = (0..array.len())
.map(|i| arrow_array_get(&*array, i))
.collect::<Result<Vec<Value>>>()?;
Value::List(ListValue::new(Some(Box::new(values)), inner_datatype))
}
_ => unimplemented!("Arrow array datatype: {:?}", array.data_type()),
};
Ok(result)
}
#[cfg(test)]
mod test {
use arrow::array::{
BooleanArray, Float32Array, Float64Array, Int16Array, Int32Array, Int64Array, Int8Array,
MutableListArray, MutablePrimitiveArray, TryExtend, UInt16Array, UInt32Array, UInt64Array,
UInt8Array,
};
use arrow::buffer::Buffer;
use arrow::datatypes::{DataType, TimeUnit as ArrowTimeUnit};
use common_time::timestamp::{TimeUnit, Timestamp};
use super::*;
use crate::prelude::Vector;
use crate::vectors::TimestampVector;
#[test]
fn test_arrow_array_access() {
let array1 = BooleanArray::from_slice(vec![true, true, false, false]);
assert_eq!(Value::Boolean(true), arrow_array_get(&array1, 1).unwrap());
let array1 = Int8Array::from_vec(vec![1, 2, 3, 4]);
assert_eq!(Value::Int8(2), arrow_array_get(&array1, 1).unwrap());
let array1 = UInt8Array::from_vec(vec![1, 2, 3, 4]);
assert_eq!(Value::UInt8(2), arrow_array_get(&array1, 1).unwrap());
let array1 = Int16Array::from_vec(vec![1, 2, 3, 4]);
assert_eq!(Value::Int16(2), arrow_array_get(&array1, 1).unwrap());
let array1 = UInt16Array::from_vec(vec![1, 2, 3, 4]);
assert_eq!(Value::UInt16(2), arrow_array_get(&array1, 1).unwrap());
let array1 = Int32Array::from_vec(vec![1, 2, 3, 4]);
assert_eq!(Value::Int32(2), arrow_array_get(&array1, 1).unwrap());
let array1 = UInt32Array::from_vec(vec![1, 2, 3, 4]);
assert_eq!(Value::UInt32(2), arrow_array_get(&array1, 1).unwrap());
let array = Int64Array::from_vec(vec![1, 2, 3, 4]);
assert_eq!(Value::Int64(2), arrow_array_get(&array, 1).unwrap());
let array1 = UInt64Array::from_vec(vec![1, 2, 3, 4]);
assert_eq!(Value::UInt64(2), arrow_array_get(&array1, 1).unwrap());
let array1 = Float32Array::from_vec(vec![1f32, 2f32, 3f32, 4f32]);
assert_eq!(
Value::Float32(2f32.into()),
arrow_array_get(&array1, 1).unwrap()
);
let array1 = Float64Array::from_vec(vec![1f64, 2f64, 3f64, 4f64]);
assert_eq!(
Value::Float64(2f64.into()),
arrow_array_get(&array1, 1).unwrap()
);
let array2 = StringArray::from(vec![Some("hello"), None, Some("world")]);
assert_eq!(
Value::String("hello".into()),
arrow_array_get(&array2, 0).unwrap()
);
assert_eq!(Value::Null, arrow_array_get(&array2, 1).unwrap());
let array3 = super::BinaryArray::from(vec![
Some("hello".as_bytes()),
None,
Some("world".as_bytes()),
]);
assert_eq!(
Value::Binary("hello".as_bytes().into()),
arrow_array_get(&array3, 0).unwrap()
);
assert_eq!(Value::Null, arrow_array_get(&array3, 1).unwrap());
let vector = TimestampVector::new(Int64Array::from_vec(vec![1, 2, 3, 4]));
let array = vector.to_boxed_arrow_array();
let value = arrow_array_get(&*array, 1).unwrap();
assert_eq!(
value,
Value::Timestamp(Timestamp::new(2, TimeUnit::Millisecond))
);
let array4 = PrimitiveArray::<i64>::from_data(
DataType::Timestamp(ArrowTimeUnit::Millisecond, None),
Buffer::from_slice(&vec![1, 2, 3, 4]),
None,
);
assert_eq!(
Value::Timestamp(Timestamp::new(1, TimeUnit::Millisecond)),
arrow_array_get(&array4, 0).unwrap()
);
let array4 = PrimitiveArray::<i64>::from_data(
DataType::Timestamp(ArrowTimeUnit::Nanosecond, None),
Buffer::from_slice(&vec![1, 2, 3, 4]),
None,
);
assert_eq!(
Value::Timestamp(Timestamp::new(1, TimeUnit::Nanosecond)),
arrow_array_get(&array4, 0).unwrap()
);
// test list array
let data = vec![
Some(vec![Some(1i32), Some(2), Some(3)]),
None,
Some(vec![Some(4), None, Some(6)]),
];
let mut arrow_array = MutableListArray::<i32, MutablePrimitiveArray<i32>>::new();
arrow_array.try_extend(data).unwrap();
let arrow_array: ListArray<i32> = arrow_array.into();
let v0 = arrow_array_get(&arrow_array, 0).unwrap();
match v0 {
Value::List(list) => {
assert!(matches!(list.datatype(), ConcreteDataType::Int32(_)));
let items = list.items().as_ref().unwrap();
assert_eq!(
**items,
vec![Value::Int32(1), Value::Int32(2), Value::Int32(3)]
);
}
_ => unreachable!(),
}
assert_eq!(Value::Null, arrow_array_get(&arrow_array, 1).unwrap());
let v2 = arrow_array_get(&arrow_array, 2).unwrap();
match v2 {
Value::List(list) => {
assert!(matches!(list.datatype(), ConcreteDataType::Int32(_)));
let items = list.items().as_ref().unwrap();
assert_eq!(**items, vec![Value::Int32(4), Value::Null, Value::Int32(6)]);
}
_ => unreachable!(),
}
}
}

View File

@@ -1,380 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::sync::Arc;
use arrow::datatypes::DataType as ArrowDataType;
use common_time::timestamp::TimeUnit;
use paste::paste;
use serde::{Deserialize, Serialize};
use crate::error::{self, Error, Result};
use crate::type_id::LogicalTypeId;
use crate::types::{
BinaryType, BooleanType, DateTimeType, DateType, Float32Type, Float64Type, Int16Type,
Int32Type, Int64Type, Int8Type, ListType, NullType, StringType, TimestampType, UInt16Type,
UInt32Type, UInt64Type, UInt8Type,
};
use crate::value::Value;
use crate::vectors::MutableVector;
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)]
#[enum_dispatch::enum_dispatch(DataType)]
pub enum ConcreteDataType {
Null(NullType),
Boolean(BooleanType),
// Numeric types:
Int8(Int8Type),
Int16(Int16Type),
Int32(Int32Type),
Int64(Int64Type),
UInt8(UInt8Type),
UInt16(UInt16Type),
UInt32(UInt32Type),
UInt64(UInt64Type),
Float32(Float32Type),
Float64(Float64Type),
// String types
Binary(BinaryType),
String(StringType),
Date(DateType),
DateTime(DateTimeType),
Timestamp(TimestampType),
List(ListType),
}
impl ConcreteDataType {
pub fn is_float(&self) -> bool {
matches!(
self,
ConcreteDataType::Float64(_) | ConcreteDataType::Float32(_)
)
}
pub fn is_boolean(&self) -> bool {
matches!(self, ConcreteDataType::Boolean(_))
}
pub fn stringifiable(&self) -> bool {
matches!(
self,
ConcreteDataType::String(_)
| ConcreteDataType::Date(_)
| ConcreteDataType::DateTime(_)
| ConcreteDataType::Timestamp(_)
)
}
pub fn is_signed(&self) -> bool {
matches!(
self,
ConcreteDataType::Int8(_)
| ConcreteDataType::Int16(_)
| ConcreteDataType::Int32(_)
| ConcreteDataType::Int64(_)
| ConcreteDataType::Date(_)
| ConcreteDataType::DateTime(_)
| ConcreteDataType::Timestamp(_)
)
}
pub fn is_unsigned(&self) -> bool {
matches!(
self,
ConcreteDataType::UInt8(_)
| ConcreteDataType::UInt16(_)
| ConcreteDataType::UInt32(_)
| ConcreteDataType::UInt64(_)
)
}
pub fn is_timestamp(&self) -> bool {
matches!(
self,
ConcreteDataType::Timestamp(_) | ConcreteDataType::Int64(_)
)
}
pub fn numerics() -> Vec<ConcreteDataType> {
vec![
ConcreteDataType::int8_datatype(),
ConcreteDataType::int16_datatype(),
ConcreteDataType::int32_datatype(),
ConcreteDataType::int64_datatype(),
ConcreteDataType::uint8_datatype(),
ConcreteDataType::uint16_datatype(),
ConcreteDataType::uint32_datatype(),
ConcreteDataType::uint64_datatype(),
ConcreteDataType::float32_datatype(),
ConcreteDataType::float64_datatype(),
]
}
/// Convert arrow data type to [ConcreteDataType].
///
/// # Panics
/// Panic if given arrow data type is not supported.
pub fn from_arrow_type(dt: &ArrowDataType) -> Self {
ConcreteDataType::try_from(dt).expect("Unimplemented type")
}
pub fn is_null(&self) -> bool {
matches!(self, ConcreteDataType::Null(NullType))
}
}
impl TryFrom<&ArrowDataType> for ConcreteDataType {
type Error = Error;
fn try_from(dt: &ArrowDataType) -> Result<ConcreteDataType> {
let concrete_type = match dt {
ArrowDataType::Null => Self::null_datatype(),
ArrowDataType::Boolean => Self::boolean_datatype(),
ArrowDataType::UInt8 => Self::uint8_datatype(),
ArrowDataType::UInt16 => Self::uint16_datatype(),
ArrowDataType::UInt32 => Self::uint32_datatype(),
ArrowDataType::UInt64 => Self::uint64_datatype(),
ArrowDataType::Int8 => Self::int8_datatype(),
ArrowDataType::Int16 => Self::int16_datatype(),
ArrowDataType::Int32 => Self::int32_datatype(),
ArrowDataType::Int64 => Self::int64_datatype(),
ArrowDataType::Float32 => Self::float32_datatype(),
ArrowDataType::Float64 => Self::float64_datatype(),
ArrowDataType::Date32 => Self::date_datatype(),
ArrowDataType::Date64 => Self::datetime_datatype(),
ArrowDataType::Timestamp(u, _) => ConcreteDataType::from_arrow_time_unit(u),
ArrowDataType::Binary | ArrowDataType::LargeBinary => Self::binary_datatype(),
ArrowDataType::Utf8 | ArrowDataType::LargeUtf8 => Self::string_datatype(),
ArrowDataType::List(field) => Self::List(ListType::new(
ConcreteDataType::from_arrow_type(&field.data_type),
)),
_ => {
return error::UnsupportedArrowTypeSnafu {
arrow_type: dt.clone(),
}
.fail()
}
};
Ok(concrete_type)
}
}
macro_rules! impl_new_concrete_type_functions {
($($Type: ident), +) => {
paste! {
impl ConcreteDataType {
$(
pub fn [<$Type:lower _datatype>]() -> ConcreteDataType {
ConcreteDataType::$Type([<$Type Type>]::default())
}
)+
}
}
}
}
impl_new_concrete_type_functions!(
Null, Boolean, UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64,
Binary, String, Date, DateTime
);
impl ConcreteDataType {
pub fn list_datatype(inner_type: ConcreteDataType) -> ConcreteDataType {
ConcreteDataType::List(ListType::new(inner_type))
}
pub fn timestamp_datatype(unit: TimeUnit) -> Self {
ConcreteDataType::Timestamp(TimestampType::new(unit))
}
pub fn timestamp_millis_datatype() -> Self {
ConcreteDataType::Timestamp(TimestampType::new(TimeUnit::Millisecond))
}
/// Converts from arrow timestamp unit to
// TODO(hl): maybe impl From<ArrowTimestamp> for our timestamp ?
pub fn from_arrow_time_unit(t: &arrow::datatypes::TimeUnit) -> Self {
match t {
arrow::datatypes::TimeUnit::Second => Self::timestamp_datatype(TimeUnit::Second),
arrow::datatypes::TimeUnit::Millisecond => {
Self::timestamp_datatype(TimeUnit::Millisecond)
}
arrow::datatypes::TimeUnit::Microsecond => {
Self::timestamp_datatype(TimeUnit::Microsecond)
}
arrow::datatypes::TimeUnit::Nanosecond => {
Self::timestamp_datatype(TimeUnit::Nanosecond)
}
}
}
}
/// Data type abstraction.
#[enum_dispatch::enum_dispatch]
pub trait DataType: std::fmt::Debug + Send + Sync {
/// Name of this data type.
fn name(&self) -> &str;
/// Returns id of the Logical data type.
fn logical_type_id(&self) -> LogicalTypeId;
/// Returns the default value of this type.
fn default_value(&self) -> Value;
/// Convert this type as [arrow2::datatypes::DataType].
fn as_arrow_type(&self) -> ArrowDataType;
/// Create a mutable vector with given `capacity` of this type.
fn create_mutable_vector(&self, capacity: usize) -> Box<dyn MutableVector>;
}
pub type DataTypeRef = Arc<dyn DataType>;
#[cfg(test)]
mod tests {
use arrow::datatypes::Field;
use super::*;
#[test]
fn test_concrete_type_as_datatype_trait() {
let concrete_type = ConcreteDataType::boolean_datatype();
assert_eq!("Boolean", concrete_type.name());
assert_eq!(Value::Boolean(false), concrete_type.default_value());
assert_eq!(LogicalTypeId::Boolean, concrete_type.logical_type_id());
assert_eq!(ArrowDataType::Boolean, concrete_type.as_arrow_type());
}
#[test]
fn test_from_arrow_type() {
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Null),
ConcreteDataType::Null(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Boolean),
ConcreteDataType::Boolean(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Binary),
ConcreteDataType::Binary(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::LargeBinary),
ConcreteDataType::Binary(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Int8),
ConcreteDataType::Int8(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Int16),
ConcreteDataType::Int16(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Int32),
ConcreteDataType::Int32(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Int64),
ConcreteDataType::Int64(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::UInt8),
ConcreteDataType::UInt8(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::UInt16),
ConcreteDataType::UInt16(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::UInt32),
ConcreteDataType::UInt32(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::UInt64),
ConcreteDataType::UInt64(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Float32),
ConcreteDataType::Float32(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Float64),
ConcreteDataType::Float64(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Utf8),
ConcreteDataType::String(_)
));
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Utf8),
ConcreteDataType::String(_)
));
assert_eq!(
ConcreteDataType::from_arrow_type(&ArrowDataType::List(Box::new(Field::new(
"item",
ArrowDataType::Int32,
true,
)))),
ConcreteDataType::List(ListType::new(ConcreteDataType::int32_datatype()))
);
assert!(matches!(
ConcreteDataType::from_arrow_type(&ArrowDataType::Date32),
ConcreteDataType::Date(_)
));
}
#[test]
fn test_from_arrow_timestamp() {
assert_eq!(
ConcreteDataType::timestamp_millisecond_datatype(),
ConcreteDataType::from_arrow_time_unit(&arrow::datatypes::TimeUnit::Millisecond)
);
assert_eq!(
ConcreteDataType::timestamp_datatype(TimeUnit::Microsecond),
ConcreteDataType::from_arrow_time_unit(&arrow::datatypes::TimeUnit::Microsecond)
);
assert_eq!(
ConcreteDataType::timestamp_datatype(TimeUnit::Nanosecond),
ConcreteDataType::from_arrow_time_unit(&arrow::datatypes::TimeUnit::Nanosecond)
);
assert_eq!(
ConcreteDataType::timestamp_datatype(TimeUnit::Second),
ConcreteDataType::from_arrow_time_unit(&arrow::datatypes::TimeUnit::Second)
);
}
#[test]
fn test_is_timestamp() {
assert!(ConcreteDataType::timestamp_millis_datatype().is_timestamp());
assert!(ConcreteDataType::timestamp_datatype(TimeUnit::Second).is_timestamp());
assert!(ConcreteDataType::timestamp_datatype(TimeUnit::Millisecond).is_timestamp());
assert!(ConcreteDataType::timestamp_datatype(TimeUnit::Microsecond).is_timestamp());
assert!(ConcreteDataType::timestamp_datatype(TimeUnit::Nanosecond).is_timestamp());
assert!(ConcreteDataType::int64_datatype().is_timestamp());
}
#[test]
fn test_is_null() {
assert!(ConcreteDataType::null_datatype().is_null());
assert!(!ConcreteDataType::int32_datatype().is_null());
}
}

View File

@@ -1,144 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use common_error::prelude::{ErrorCompat, ErrorExt, Snafu, StatusCode};
use snafu::Backtrace;
#[derive(Debug, Snafu)]
#[snafu(visibility(pub))]
pub enum Error {
#[snafu(display("Failed to serialize data, source: {}", source))]
Serialize {
source: serde_json::Error,
backtrace: Backtrace,
},
#[snafu(display("Failed to deserialize data, source: {}, json: {}", source, json))]
Deserialize {
source: serde_json::Error,
backtrace: Backtrace,
json: String,
},
#[snafu(display("Failed to convert datafusion type: {}", from))]
Conversion { from: String, backtrace: Backtrace },
#[snafu(display("Bad array access, Index out of bounds: {}, size: {}", index, size))]
BadArrayAccess {
index: usize,
size: usize,
backtrace: Backtrace,
},
#[snafu(display("Unknown vector, {}", msg))]
UnknownVector { msg: String, backtrace: Backtrace },
#[snafu(display("Unsupported arrow data type, type: {:?}", arrow_type))]
UnsupportedArrowType {
arrow_type: arrow::datatypes::DataType,
backtrace: Backtrace,
},
#[snafu(display("Timestamp column {} not found", name,))]
TimestampNotFound { name: String, backtrace: Backtrace },
#[snafu(display(
"Failed to parse version in schema meta, value: {}, source: {}",
value,
source
))]
ParseSchemaVersion {
value: String,
source: std::num::ParseIntError,
backtrace: Backtrace,
},
#[snafu(display("Invalid timestamp index: {}", index))]
InvalidTimestampIndex { index: usize, backtrace: Backtrace },
#[snafu(display("Duplicate timestamp index, exists: {}, new: {}", exists, new))]
DuplicateTimestampIndex {
exists: usize,
new: usize,
backtrace: Backtrace,
},
#[snafu(display("{}", msg))]
CastType { msg: String, backtrace: Backtrace },
#[snafu(display("Arrow failed to compute, source: {}", source))]
ArrowCompute {
source: arrow::error::ArrowError,
backtrace: Backtrace,
},
#[snafu(display("Unsupported column default constraint expression: {}", expr))]
UnsupportedDefaultExpr { expr: String, backtrace: Backtrace },
#[snafu(display("Default value should not be null for non null column"))]
NullDefault { backtrace: Backtrace },
#[snafu(display("Incompatible default value type, reason: {}", reason))]
DefaultValueType {
reason: String,
backtrace: Backtrace,
},
#[snafu(display("Duplicated metadata for {}", key))]
DuplicateMeta { key: String, backtrace: Backtrace },
}
impl ErrorExt for Error {
fn status_code(&self) -> StatusCode {
// Inner encoding and decoding error should not be exposed to users.
StatusCode::Internal
}
fn backtrace_opt(&self) -> Option<&Backtrace> {
ErrorCompat::backtrace(self)
}
fn as_any(&self) -> &dyn Any {
self
}
}
pub type Result<T> = std::result::Result<T, Error>;
#[cfg(test)]
mod tests {
use std::collections::HashMap;
use snafu::ResultExt;
use super::*;
#[test]
pub fn test_error() {
let mut map = HashMap::new();
map.insert(true, 1);
map.insert(false, 2);
let result = serde_json::to_string(&map).context(SerializeSnafu);
assert!(result.is_err(), "serialize result is: {:?}", result);
let err = serde_json::to_string(&map)
.context(SerializeSnafu)
.err()
.unwrap();
assert!(err.backtrace_opt().is_some());
assert_eq!(StatusCode::Internal, err.status_code());
}
}

View File

@@ -1,32 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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.
#![feature(generic_associated_types)]
#![feature(assert_matches)]
pub mod arrow_array;
pub mod data_type;
pub mod error;
pub mod macros;
pub mod prelude;
mod scalars;
pub mod schema;
pub mod serialize;
pub mod type_id;
pub mod types;
pub mod value;
pub mod vectors;
pub use arrow;
pub use error::{Error, Result};

View File

@@ -1,80 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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.
///! Some helper macros for datatypes, copied from databend.
#[macro_export]
macro_rules! for_all_scalar_types {
($macro:tt $(, $x:tt)*) => {
$macro! {
[$($x),*],
{ i8 },
{ i16 },
{ i32 },
{ i64 },
{ u8 },
{ u16 },
{ u32 },
{ u64 },
{ f32 },
{ f64 },
{ bool },
}
};
}
#[macro_export]
macro_rules! for_all_primitive_types {
($macro:tt $(, $x:tt)*) => {
$macro! {
[$($x),*],
{ i8 },
{ i16 },
{ i32 },
{ i64 },
{ u8 },
{ u16 },
{ u32 },
{ u64 },
{ f32 },
{ f64 }
}
};
}
#[macro_export]
macro_rules! with_match_primitive_type_id {
($key_type:expr, | $_:tt $T:ident | $body:tt, $nbody:tt) => {{
macro_rules! __with_ty__ {
( $_ $T:ident ) => {
$body
};
}
use $crate::type_id::LogicalTypeId;
match $key_type {
LogicalTypeId::Int8 => __with_ty__! { i8 },
LogicalTypeId::Int16 => __with_ty__! { i16 },
LogicalTypeId::Int32 => __with_ty__! { i32 },
LogicalTypeId::Int64 => __with_ty__! { i64 },
LogicalTypeId::UInt8 => __with_ty__! { u8 },
LogicalTypeId::UInt16 => __with_ty__! { u16 },
LogicalTypeId::UInt32 => __with_ty__! { u32 },
LogicalTypeId::UInt64 => __with_ty__! { u64 },
LogicalTypeId::Float32 => __with_ty__! { f32 },
LogicalTypeId::Float64 => __with_ty__! { f64 },
_ => $nbody,
}
}};
}

View File

@@ -1,23 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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.
pub use crate::data_type::{ConcreteDataType, DataType, DataTypeRef};
pub use crate::macros::*;
pub use crate::scalars::{Scalar, ScalarRef, ScalarVector, ScalarVectorBuilder};
pub use crate::type_id::LogicalTypeId;
pub use crate::types::Primitive;
pub use crate::value::{Value, ValueRef};
pub use crate::vectors::{
Helper as VectorHelper, MutableVector, Validity, Vector, VectorBuilder, VectorRef,
};

View File

@@ -1,464 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use common_time::{Date, DateTime, Timestamp};
use crate::prelude::*;
use crate::value::{ListValue, ListValueRef};
use crate::vectors::*;
fn get_iter_capacity<T, I: Iterator<Item = T>>(iter: &I) -> usize {
match iter.size_hint() {
(_lower, Some(upper)) => upper,
(0, None) => 1024,
(lower, None) => lower,
}
}
/// Owned scalar value
/// primitive types, bool, Vec<u8> ...
pub trait Scalar: 'static + Sized + Default + Any
where
for<'a> Self::VectorType: ScalarVector<RefItem<'a> = Self::RefType<'a>>,
{
type VectorType: ScalarVector<OwnedItem = Self>;
type RefType<'a>: ScalarRef<'a, ScalarType = Self, VectorType = Self::VectorType>
where
Self: 'a;
/// Get a reference of the current value.
fn as_scalar_ref(&self) -> Self::RefType<'_>;
/// Upcast GAT type's lifetime.
fn upcast_gat<'short, 'long: 'short>(long: Self::RefType<'long>) -> Self::RefType<'short>;
}
pub trait ScalarRef<'a>: std::fmt::Debug + Clone + Copy + Send + 'a {
type VectorType: ScalarVector<RefItem<'a> = Self>;
/// The corresponding [`Scalar`] type.
type ScalarType: Scalar<RefType<'a> = Self>;
/// Convert the reference into an owned value.
fn to_owned_scalar(&self) -> Self::ScalarType;
}
/// A sub trait of Vector to add scalar operation support.
// This implementation refers to Datebend's [ScalarColumn](https://github.com/datafuselabs/databend/blob/main/common/datavalues/src/scalars/type_.rs)
// and skyzh's [type-exercise-in-rust](https://github.com/skyzh/type-exercise-in-rust).
pub trait ScalarVector: Vector + Send + Sync + Sized + 'static
where
for<'a> Self::OwnedItem: Scalar<RefType<'a> = Self::RefItem<'a>>,
{
type OwnedItem: Scalar<VectorType = Self>;
/// The reference item of this vector.
type RefItem<'a>: ScalarRef<'a, ScalarType = Self::OwnedItem, VectorType = Self>
where
Self: 'a;
/// Iterator type of this vector.
type Iter<'a>: Iterator<Item = Option<Self::RefItem<'a>>>
where
Self: 'a;
/// Builder type to build this vector.
type Builder: ScalarVectorBuilder<VectorType = Self>;
/// Returns the reference to an element at given position.
///
/// Note: `get()` has bad performance, avoid call this function inside loop.
///
/// # Panics
/// Panics if `idx >= self.len()`.
fn get_data(&self, idx: usize) -> Option<Self::RefItem<'_>>;
/// Returns iterator of current vector.
fn iter_data(&self) -> Self::Iter<'_>;
fn from_slice(data: &[Self::RefItem<'_>]) -> Self {
let mut builder = Self::Builder::with_capacity(data.len());
for item in data {
builder.push(Some(*item));
}
builder.finish()
}
fn from_iterator<'a>(it: impl Iterator<Item = Self::RefItem<'a>>) -> Self {
let mut builder = Self::Builder::with_capacity(get_iter_capacity(&it));
for item in it {
builder.push(Some(item));
}
builder.finish()
}
fn from_owned_iterator(it: impl Iterator<Item = Option<Self::OwnedItem>>) -> Self {
let mut builder = Self::Builder::with_capacity(get_iter_capacity(&it));
for item in it {
match item {
Some(item) => builder.push(Some(item.as_scalar_ref())),
None => builder.push(None),
}
}
builder.finish()
}
fn from_vec<I: Into<Self::OwnedItem>>(values: Vec<I>) -> Self {
let it = values.into_iter();
let mut builder = Self::Builder::with_capacity(get_iter_capacity(&it));
for item in it {
builder.push(Some(item.into().as_scalar_ref()));
}
builder.finish()
}
}
/// A trait over all vector builders.
pub trait ScalarVectorBuilder: MutableVector {
type VectorType: ScalarVector<Builder = Self>;
/// Create a new builder with initial `capacity`.
fn with_capacity(capacity: usize) -> Self;
/// Push a value into the builder.
fn push(&mut self, value: Option<<Self::VectorType as ScalarVector>::RefItem<'_>>);
/// Finish build and return a new vector.
fn finish(&mut self) -> Self::VectorType;
}
macro_rules! impl_primitive_scalar_type {
($native:ident) => {
impl Scalar for $native {
type VectorType = PrimitiveVector<$native>;
type RefType<'a> = $native;
#[inline]
fn as_scalar_ref(&self) -> $native {
*self
}
#[allow(clippy::needless_lifetimes)]
#[inline]
fn upcast_gat<'short, 'long: 'short>(long: $native) -> $native {
long
}
}
/// Implement [`ScalarRef`] for primitive types. Note that primitive types are both [`Scalar`] and [`ScalarRef`].
impl<'a> ScalarRef<'a> for $native {
type VectorType = PrimitiveVector<$native>;
type ScalarType = $native;
#[inline]
fn to_owned_scalar(&self) -> $native {
*self
}
}
};
}
impl_primitive_scalar_type!(u8);
impl_primitive_scalar_type!(u16);
impl_primitive_scalar_type!(u32);
impl_primitive_scalar_type!(u64);
impl_primitive_scalar_type!(i8);
impl_primitive_scalar_type!(i16);
impl_primitive_scalar_type!(i32);
impl_primitive_scalar_type!(i64);
impl_primitive_scalar_type!(f32);
impl_primitive_scalar_type!(f64);
impl Scalar for bool {
type VectorType = BooleanVector;
type RefType<'a> = bool;
#[inline]
fn as_scalar_ref(&self) -> bool {
*self
}
#[allow(clippy::needless_lifetimes)]
#[inline]
fn upcast_gat<'short, 'long: 'short>(long: bool) -> bool {
long
}
}
impl<'a> ScalarRef<'a> for bool {
type VectorType = BooleanVector;
type ScalarType = bool;
#[inline]
fn to_owned_scalar(&self) -> bool {
*self
}
}
impl Scalar for String {
type VectorType = StringVector;
type RefType<'a> = &'a str;
#[inline]
fn as_scalar_ref(&self) -> &str {
self
}
#[inline]
fn upcast_gat<'short, 'long: 'short>(long: &'long str) -> &'short str {
long
}
}
impl<'a> ScalarRef<'a> for &'a str {
type VectorType = StringVector;
type ScalarType = String;
#[inline]
fn to_owned_scalar(&self) -> String {
self.to_string()
}
}
impl Scalar for Vec<u8> {
type VectorType = BinaryVector;
type RefType<'a> = &'a [u8];
#[inline]
fn as_scalar_ref(&self) -> &[u8] {
self
}
#[inline]
fn upcast_gat<'short, 'long: 'short>(long: &'long [u8]) -> &'short [u8] {
long
}
}
impl<'a> ScalarRef<'a> for &'a [u8] {
type VectorType = BinaryVector;
type ScalarType = Vec<u8>;
#[inline]
fn to_owned_scalar(&self) -> Vec<u8> {
self.to_vec()
}
}
impl Scalar for Date {
type VectorType = DateVector;
type RefType<'a> = Date;
fn as_scalar_ref(&self) -> Self::RefType<'_> {
*self
}
fn upcast_gat<'short, 'long: 'short>(long: Self::RefType<'long>) -> Self::RefType<'short> {
long
}
}
impl<'a> ScalarRef<'a> for Date {
type VectorType = DateVector;
type ScalarType = Date;
fn to_owned_scalar(&self) -> Self::ScalarType {
*self
}
}
impl Scalar for DateTime {
type VectorType = DateTimeVector;
type RefType<'a> = DateTime;
fn as_scalar_ref(&self) -> Self::RefType<'_> {
*self
}
fn upcast_gat<'short, 'long: 'short>(long: Self::RefType<'long>) -> Self::RefType<'short> {
long
}
}
impl<'a> ScalarRef<'a> for DateTime {
type VectorType = DateTimeVector;
type ScalarType = DateTime;
fn to_owned_scalar(&self) -> Self::ScalarType {
*self
}
}
impl Scalar for Timestamp {
type VectorType = TimestampVector;
type RefType<'a> = Timestamp;
fn as_scalar_ref(&self) -> Self::RefType<'_> {
*self
}
fn upcast_gat<'short, 'long: 'short>(long: Self::RefType<'long>) -> Self::RefType<'short> {
long
}
}
impl<'a> ScalarRef<'a> for Timestamp {
type VectorType = TimestampVector;
type ScalarType = Timestamp;
fn to_owned_scalar(&self) -> Self::ScalarType {
*self
}
}
impl Scalar for ListValue {
type VectorType = ListVector;
type RefType<'a> = ListValueRef<'a>;
fn as_scalar_ref(&self) -> Self::RefType<'_> {
ListValueRef::Ref { val: self }
}
fn upcast_gat<'short, 'long: 'short>(long: Self::RefType<'long>) -> Self::RefType<'short> {
long
}
}
impl<'a> ScalarRef<'a> for ListValueRef<'a> {
type VectorType = ListVector;
type ScalarType = ListValue;
fn to_owned_scalar(&self) -> Self::ScalarType {
match self {
ListValueRef::Indexed { vector, idx } => match vector.get(*idx) {
// Normally should not get `Value::Null` if the `ListValueRef` comes
// from the iterator of the ListVector, but we avoid panic and just
// returns a default list value in such case since `ListValueRef` may
// be constructed manually.
Value::Null => ListValue::default(),
Value::List(v) => v,
_ => unreachable!(),
},
ListValueRef::Ref { val } => (*val).clone(),
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::vectors::binary::BinaryVector;
use crate::vectors::primitive::Int32Vector;
fn build_vector_from_slice<T: ScalarVector>(items: &[Option<T::RefItem<'_>>]) -> T {
let mut builder = T::Builder::with_capacity(items.len());
for item in items {
builder.push(*item);
}
builder.finish()
}
fn assert_vector_eq<'a, T: ScalarVector>(expect: &[Option<T::RefItem<'a>>], vector: &'a T)
where
T::RefItem<'a>: PartialEq + std::fmt::Debug,
{
for (a, b) in expect.iter().zip(vector.iter_data()) {
assert_eq!(*a, b);
}
}
#[test]
fn test_build_i32_vector() {
let expect = vec![Some(1), Some(2), Some(3), None, Some(5)];
let vector: Int32Vector = build_vector_from_slice(&expect);
assert_vector_eq(&expect, &vector);
}
#[test]
fn test_build_binary_vector() {
let expect: Vec<Option<&'static [u8]>> = vec![
Some(b"a"),
Some(b"b"),
Some(b"c"),
None,
Some(b"e"),
Some(b""),
];
let vector: BinaryVector = build_vector_from_slice(&expect);
assert_vector_eq(&expect, &vector);
}
#[test]
fn test_build_date_vector() {
let expect: Vec<Option<Date>> = vec![
Some(Date::new(0)),
Some(Date::new(-1)),
None,
Some(Date::new(1)),
];
let vector: DateVector = build_vector_from_slice(&expect);
assert_vector_eq(&expect, &vector);
}
#[test]
fn test_date_scalar() {
let date = Date::new(1);
assert_eq!(date, date.as_scalar_ref());
assert_eq!(date, date.to_owned_scalar());
}
#[test]
fn test_datetime_scalar() {
let dt = DateTime::new(123);
assert_eq!(dt, dt.as_scalar_ref());
assert_eq!(dt, dt.to_owned_scalar());
}
#[test]
fn test_list_value_scalar() {
let list_value = ListValue::new(
Some(Box::new(vec![Value::Int32(123)])),
ConcreteDataType::int32_datatype(),
);
let list_ref = ListValueRef::Ref { val: &list_value };
assert_eq!(list_ref, list_value.as_scalar_ref());
assert_eq!(list_value, list_ref.to_owned_scalar());
let mut builder =
ListVectorBuilder::with_type_capacity(ConcreteDataType::int32_datatype(), 1);
builder.push(None);
builder.push(Some(list_value.as_scalar_ref()));
let vector = builder.finish();
let ref_on_vec = ListValueRef::Indexed {
vector: &vector,
idx: 0,
};
assert_eq!(ListValue::default(), ref_on_vec.to_owned_scalar());
let ref_on_vec = ListValueRef::Indexed {
vector: &vector,
idx: 1,
};
assert_eq!(list_value, ref_on_vec.to_owned_scalar());
}
#[test]
fn test_build_timestamp_vector() {
let expect: Vec<Option<Timestamp>> = vec![Some(10.into()), None, Some(42.into())];
let vector: TimestampVector = build_vector_from_slice(&expect);
assert_vector_eq(&expect, &vector);
let val = vector.get_data(0).unwrap();
assert_eq!(val, val.as_scalar_ref());
assert_eq!(10, val.to_owned_scalar().value());
}
}

View File

@@ -1,698 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 constraint;
mod raw;
use std::collections::HashMap;
use std::sync::Arc;
pub use arrow::datatypes::Metadata;
use arrow::datatypes::{Field, Schema as ArrowSchema};
use serde::{Deserialize, Serialize};
use snafu::{ensure, ResultExt};
use crate::data_type::{ConcreteDataType, DataType};
use crate::error::{self, DeserializeSnafu, Error, Result, SerializeSnafu};
pub use crate::schema::constraint::ColumnDefaultConstraint;
pub use crate::schema::raw::RawSchema;
use crate::vectors::VectorRef;
/// Key used to store whether the column is time index in arrow field's metadata.
const TIME_INDEX_KEY: &str = "greptime:time_index";
/// Key used to store version number of the schema in metadata.
const VERSION_KEY: &str = "greptime:version";
/// Key used to store default constraint in arrow field's metadata.
const ARROW_FIELD_DEFAULT_CONSTRAINT_KEY: &str = "greptime:default_constraint";
/// Schema of a column, used as an immutable struct.
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
pub struct ColumnSchema {
pub name: String,
pub data_type: ConcreteDataType,
is_nullable: bool,
is_time_index: bool,
default_constraint: Option<ColumnDefaultConstraint>,
metadata: Metadata,
}
impl ColumnSchema {
pub fn new<T: Into<String>>(
name: T,
data_type: ConcreteDataType,
is_nullable: bool,
) -> ColumnSchema {
ColumnSchema {
name: name.into(),
data_type,
is_nullable,
is_time_index: false,
default_constraint: None,
metadata: Metadata::new(),
}
}
#[inline]
pub fn is_time_index(&self) -> bool {
self.is_time_index
}
#[inline]
pub fn is_nullable(&self) -> bool {
self.is_nullable
}
#[inline]
pub fn default_constraint(&self) -> Option<&ColumnDefaultConstraint> {
self.default_constraint.as_ref()
}
#[inline]
pub fn metadata(&self) -> &Metadata {
&self.metadata
}
pub fn with_time_index(mut self, is_time_index: bool) -> Self {
self.is_time_index = is_time_index;
if is_time_index {
self.metadata
.insert(TIME_INDEX_KEY.to_string(), "true".to_string());
} else {
self.metadata.remove(TIME_INDEX_KEY);
}
self
}
pub fn with_default_constraint(
mut self,
default_constraint: Option<ColumnDefaultConstraint>,
) -> Result<Self> {
if let Some(constraint) = &default_constraint {
constraint.validate(&self.data_type, self.is_nullable)?;
}
self.default_constraint = default_constraint;
Ok(self)
}
/// Creates a new [`ColumnSchema`] with given metadata.
pub fn with_metadata(mut self, metadata: Metadata) -> Self {
self.metadata = metadata;
self
}
pub fn create_default_vector(&self, num_rows: usize) -> Result<Option<VectorRef>> {
match &self.default_constraint {
Some(c) => c
.create_default_vector(&self.data_type, self.is_nullable, num_rows)
.map(Some),
None => {
if self.is_nullable {
// No default constraint, use null as default value.
// TODO(yingwen): Use NullVector once it supports setting logical type.
ColumnDefaultConstraint::null_value()
.create_default_vector(&self.data_type, self.is_nullable, num_rows)
.map(Some)
} else {
Ok(None)
}
}
}
}
}
/// A common schema, should be immutable.
#[derive(Debug, Clone, PartialEq)]
pub struct Schema {
column_schemas: Vec<ColumnSchema>,
name_to_index: HashMap<String, usize>,
arrow_schema: Arc<ArrowSchema>,
/// Index of the timestamp key column.
///
/// Timestamp key column is the column holds the timestamp and forms part of
/// the primary key. None means there is no timestamp key column.
timestamp_index: Option<usize>,
/// Version of the schema.
///
/// Initial value is zero. The version should bump after altering schema.
version: u32,
}
impl Schema {
/// Initial version of the schema.
pub const INITIAL_VERSION: u32 = 0;
/// Create a schema from a vector of [ColumnSchema].
///
/// # Panics
/// Panics when ColumnSchema's `default_constraint` can't be serialized into json.
pub fn new(column_schemas: Vec<ColumnSchema>) -> Schema {
// Builder won't fail in this case
SchemaBuilder::try_from(column_schemas)
.unwrap()
.build()
.unwrap()
}
/// Try to Create a schema from a vector of [ColumnSchema].
pub fn try_new(column_schemas: Vec<ColumnSchema>) -> Result<Schema> {
SchemaBuilder::try_from(column_schemas)?.build()
}
#[inline]
pub fn arrow_schema(&self) -> &Arc<ArrowSchema> {
&self.arrow_schema
}
#[inline]
pub fn column_schemas(&self) -> &[ColumnSchema] {
&self.column_schemas
}
pub fn column_schema_by_name(&self, name: &str) -> Option<&ColumnSchema> {
self.name_to_index
.get(name)
.map(|index| &self.column_schemas[*index])
}
/// Retrieve the column's name by index
/// # Panics
/// This method **may** panic if the index is out of range of column schemas.
#[inline]
pub fn column_name_by_index(&self, idx: usize) -> &str {
&self.column_schemas[idx].name
}
#[inline]
pub fn column_index_by_name(&self, name: &str) -> Option<usize> {
self.name_to_index.get(name).copied()
}
#[inline]
pub fn contains_column(&self, name: &str) -> bool {
self.name_to_index.contains_key(name)
}
#[inline]
pub fn num_columns(&self) -> usize {
self.column_schemas.len()
}
#[inline]
pub fn is_empty(&self) -> bool {
self.column_schemas.is_empty()
}
/// Returns index of the timestamp key column.
#[inline]
pub fn timestamp_index(&self) -> Option<usize> {
self.timestamp_index
}
#[inline]
pub fn timestamp_column(&self) -> Option<&ColumnSchema> {
self.timestamp_index.map(|idx| &self.column_schemas[idx])
}
#[inline]
pub fn version(&self) -> u32 {
self.version
}
#[inline]
pub fn metadata(&self) -> &Metadata {
&self.arrow_schema.metadata
}
}
#[derive(Default)]
pub struct SchemaBuilder {
column_schemas: Vec<ColumnSchema>,
name_to_index: HashMap<String, usize>,
fields: Vec<Field>,
timestamp_index: Option<usize>,
version: u32,
metadata: Metadata,
}
impl TryFrom<Vec<ColumnSchema>> for SchemaBuilder {
type Error = Error;
fn try_from(column_schemas: Vec<ColumnSchema>) -> Result<SchemaBuilder> {
SchemaBuilder::try_from_columns(column_schemas)
}
}
impl SchemaBuilder {
pub fn try_from_columns(column_schemas: Vec<ColumnSchema>) -> Result<Self> {
let FieldsAndIndices {
fields,
name_to_index,
timestamp_index,
} = collect_fields(&column_schemas)?;
Ok(Self {
column_schemas,
name_to_index,
fields,
timestamp_index,
..Default::default()
})
}
pub fn version(mut self, version: u32) -> Self {
self.version = version;
self
}
/// Add key value pair to metadata.
///
/// Old metadata with same key would be overwritten.
pub fn add_metadata(mut self, key: impl Into<String>, value: impl Into<String>) -> Self {
self.metadata.insert(key.into(), value.into());
self
}
pub fn build(mut self) -> Result<Schema> {
if let Some(timestamp_index) = self.timestamp_index {
validate_timestamp_index(&self.column_schemas, timestamp_index)?;
}
self.metadata
.insert(VERSION_KEY.to_string(), self.version.to_string());
let arrow_schema = ArrowSchema::from(self.fields).with_metadata(self.metadata);
Ok(Schema {
column_schemas: self.column_schemas,
name_to_index: self.name_to_index,
arrow_schema: Arc::new(arrow_schema),
timestamp_index: self.timestamp_index,
version: self.version,
})
}
}
struct FieldsAndIndices {
fields: Vec<Field>,
name_to_index: HashMap<String, usize>,
timestamp_index: Option<usize>,
}
fn collect_fields(column_schemas: &[ColumnSchema]) -> Result<FieldsAndIndices> {
let mut fields = Vec::with_capacity(column_schemas.len());
let mut name_to_index = HashMap::with_capacity(column_schemas.len());
let mut timestamp_index = None;
for (index, column_schema) in column_schemas.iter().enumerate() {
if column_schema.is_time_index() {
ensure!(
timestamp_index.is_none(),
error::DuplicateTimestampIndexSnafu {
exists: timestamp_index.unwrap(),
new: index,
}
);
timestamp_index = Some(index);
}
let field = Field::try_from(column_schema)?;
fields.push(field);
name_to_index.insert(column_schema.name.clone(), index);
}
Ok(FieldsAndIndices {
fields,
name_to_index,
timestamp_index,
})
}
fn validate_timestamp_index(column_schemas: &[ColumnSchema], timestamp_index: usize) -> Result<()> {
ensure!(
timestamp_index < column_schemas.len(),
error::InvalidTimestampIndexSnafu {
index: timestamp_index,
}
);
let column_schema = &column_schemas[timestamp_index];
ensure!(
column_schema.data_type.is_timestamp(),
error::InvalidTimestampIndexSnafu {
index: timestamp_index,
}
);
ensure!(
column_schema.is_time_index(),
error::InvalidTimestampIndexSnafu {
index: timestamp_index,
}
);
Ok(())
}
pub type SchemaRef = Arc<Schema>;
impl TryFrom<&Field> for ColumnSchema {
type Error = Error;
fn try_from(field: &Field) -> Result<ColumnSchema> {
let data_type = ConcreteDataType::try_from(&field.data_type)?;
let mut metadata = field.metadata.clone();
let default_constraint = match metadata.remove(ARROW_FIELD_DEFAULT_CONSTRAINT_KEY) {
Some(json) => Some(serde_json::from_str(&json).context(DeserializeSnafu { json })?),
None => None,
};
let is_time_index = metadata.contains_key(TIME_INDEX_KEY);
Ok(ColumnSchema {
name: field.name.clone(),
data_type,
is_nullable: field.is_nullable,
is_time_index,
default_constraint,
metadata,
})
}
}
impl TryFrom<&ColumnSchema> for Field {
type Error = Error;
fn try_from(column_schema: &ColumnSchema) -> Result<Field> {
let mut metadata = column_schema.metadata.clone();
if let Some(value) = &column_schema.default_constraint {
// Adds an additional metadata to store the default constraint.
let old = metadata.insert(
ARROW_FIELD_DEFAULT_CONSTRAINT_KEY.to_string(),
serde_json::to_string(&value).context(SerializeSnafu)?,
);
ensure!(
old.is_none(),
error::DuplicateMetaSnafu {
key: ARROW_FIELD_DEFAULT_CONSTRAINT_KEY,
}
);
}
Ok(Field::new(
column_schema.name.clone(),
column_schema.data_type.as_arrow_type(),
column_schema.is_nullable(),
)
.with_metadata(metadata))
}
}
impl TryFrom<Arc<ArrowSchema>> for Schema {
type Error = Error;
fn try_from(arrow_schema: Arc<ArrowSchema>) -> Result<Schema> {
let mut column_schemas = Vec::with_capacity(arrow_schema.fields.len());
let mut name_to_index = HashMap::with_capacity(arrow_schema.fields.len());
for field in &arrow_schema.fields {
let column_schema = ColumnSchema::try_from(field)?;
name_to_index.insert(field.name.clone(), column_schemas.len());
column_schemas.push(column_schema);
}
let mut timestamp_index = None;
for (index, column_schema) in column_schemas.iter().enumerate() {
if column_schema.is_time_index() {
validate_timestamp_index(&column_schemas, index)?;
ensure!(
timestamp_index.is_none(),
error::DuplicateTimestampIndexSnafu {
exists: timestamp_index.unwrap(),
new: index,
}
);
timestamp_index = Some(index);
}
}
let version = try_parse_version(&arrow_schema.metadata, VERSION_KEY)?;
Ok(Self {
column_schemas,
name_to_index,
arrow_schema,
timestamp_index,
version,
})
}
}
impl TryFrom<ArrowSchema> for Schema {
type Error = Error;
fn try_from(arrow_schema: ArrowSchema) -> Result<Schema> {
let arrow_schema = Arc::new(arrow_schema);
Schema::try_from(arrow_schema)
}
}
fn try_parse_version(metadata: &Metadata, key: &str) -> Result<u32> {
if let Some(value) = metadata.get(key) {
let version = value
.parse()
.context(error::ParseSchemaVersionSnafu { value })?;
Ok(version)
} else {
Ok(Schema::INITIAL_VERSION)
}
}
#[cfg(test)]
mod tests {
use arrow::datatypes::DataType as ArrowDataType;
use super::*;
use crate::value::Value;
#[test]
fn test_column_schema() {
let column_schema = ColumnSchema::new("test", ConcreteDataType::int32_datatype(), true);
let field = Field::try_from(&column_schema).unwrap();
assert_eq!("test", field.name);
assert_eq!(ArrowDataType::Int32, field.data_type);
assert!(field.is_nullable);
let new_column_schema = ColumnSchema::try_from(&field).unwrap();
assert_eq!(column_schema, new_column_schema);
}
#[test]
fn test_column_schema_with_default_constraint() {
let column_schema = ColumnSchema::new("test", ConcreteDataType::int32_datatype(), true)
.with_default_constraint(Some(ColumnDefaultConstraint::Value(Value::from(99))))
.unwrap();
assert!(column_schema
.metadata()
.get(ARROW_FIELD_DEFAULT_CONSTRAINT_KEY)
.is_none());
let field = Field::try_from(&column_schema).unwrap();
assert_eq!("test", field.name);
assert_eq!(ArrowDataType::Int32, field.data_type);
assert!(field.is_nullable);
assert_eq!(
"{\"Value\":{\"Int32\":99}}",
field
.metadata
.get(ARROW_FIELD_DEFAULT_CONSTRAINT_KEY)
.unwrap()
);
let new_column_schema = ColumnSchema::try_from(&field).unwrap();
assert_eq!(column_schema, new_column_schema);
}
#[test]
fn test_column_schema_with_metadata() {
let mut metadata = Metadata::new();
metadata.insert("k1".to_string(), "v1".to_string());
let column_schema = ColumnSchema::new("test", ConcreteDataType::int32_datatype(), true)
.with_metadata(metadata)
.with_default_constraint(Some(ColumnDefaultConstraint::null_value()))
.unwrap();
assert_eq!("v1", column_schema.metadata().get("k1").unwrap());
assert!(column_schema
.metadata()
.get(ARROW_FIELD_DEFAULT_CONSTRAINT_KEY)
.is_none());
let field = Field::try_from(&column_schema).unwrap();
assert_eq!("v1", field.metadata.get("k1").unwrap());
assert!(field
.metadata
.get(ARROW_FIELD_DEFAULT_CONSTRAINT_KEY)
.is_some());
let new_column_schema = ColumnSchema::try_from(&field).unwrap();
assert_eq!(column_schema, new_column_schema);
}
#[test]
fn test_column_schema_with_duplicate_metadata() {
let mut metadata = Metadata::new();
metadata.insert(
ARROW_FIELD_DEFAULT_CONSTRAINT_KEY.to_string(),
"v1".to_string(),
);
let column_schema = ColumnSchema::new("test", ConcreteDataType::int32_datatype(), true)
.with_metadata(metadata)
.with_default_constraint(Some(ColumnDefaultConstraint::null_value()))
.unwrap();
Field::try_from(&column_schema).unwrap_err();
}
#[test]
fn test_column_schema_invalid_default_constraint() {
ColumnSchema::new("test", ConcreteDataType::int32_datatype(), false)
.with_default_constraint(Some(ColumnDefaultConstraint::null_value()))
.unwrap_err();
}
#[test]
fn test_column_default_constraint_try_into_from() {
let default_constraint = ColumnDefaultConstraint::Value(Value::from(42i64));
let bytes: Vec<u8> = default_constraint.clone().try_into().unwrap();
let from_value = ColumnDefaultConstraint::try_from(&bytes[..]).unwrap();
assert_eq!(default_constraint, from_value);
}
#[test]
fn test_column_schema_create_default_null() {
// Implicit default null.
let column_schema = ColumnSchema::new("test", ConcreteDataType::int32_datatype(), true);
let v = column_schema.create_default_vector(5).unwrap().unwrap();
assert_eq!(5, v.len());
assert!(v.only_null());
// Explicit default null.
let column_schema = ColumnSchema::new("test", ConcreteDataType::int32_datatype(), true)
.with_default_constraint(Some(ColumnDefaultConstraint::null_value()))
.unwrap();
let v = column_schema.create_default_vector(5).unwrap().unwrap();
assert_eq!(5, v.len());
assert!(v.only_null());
}
#[test]
fn test_column_schema_no_default() {
let column_schema = ColumnSchema::new("test", ConcreteDataType::int32_datatype(), false);
assert!(column_schema.create_default_vector(5).unwrap().is_none());
}
#[test]
fn test_build_empty_schema() {
let schema = SchemaBuilder::default().build().unwrap();
assert_eq!(0, schema.num_columns());
assert!(schema.is_empty());
}
#[test]
fn test_schema_no_timestamp() {
let column_schemas = vec![
ColumnSchema::new("col1", ConcreteDataType::int32_datatype(), false),
ColumnSchema::new("col2", ConcreteDataType::float64_datatype(), true),
];
let schema = Schema::new(column_schemas.clone());
assert_eq!(2, schema.num_columns());
assert!(!schema.is_empty());
assert!(schema.timestamp_index().is_none());
assert!(schema.timestamp_column().is_none());
assert_eq!(Schema::INITIAL_VERSION, schema.version());
for column_schema in &column_schemas {
let found = schema.column_schema_by_name(&column_schema.name).unwrap();
assert_eq!(column_schema, found);
}
assert!(schema.column_schema_by_name("col3").is_none());
let new_schema = Schema::try_from(schema.arrow_schema().clone()).unwrap();
assert_eq!(schema, new_schema);
assert_eq!(column_schemas, schema.column_schemas());
}
#[test]
fn test_metadata() {
let column_schemas = vec![ColumnSchema::new(
"col1",
ConcreteDataType::int32_datatype(),
false,
)];
let schema = SchemaBuilder::try_from(column_schemas)
.unwrap()
.add_metadata("k1", "v1")
.build()
.unwrap();
assert_eq!("v1", schema.metadata().get("k1").unwrap());
}
#[test]
fn test_schema_with_timestamp() {
let column_schemas = vec![
ColumnSchema::new("col1", ConcreteDataType::int32_datatype(), true),
ColumnSchema::new("ts", ConcreteDataType::timestamp_millis_datatype(), false)
.with_time_index(true),
];
let schema = SchemaBuilder::try_from(column_schemas.clone())
.unwrap()
.version(123)
.build()
.unwrap();
assert_eq!(1, schema.timestamp_index().unwrap());
assert_eq!(&column_schemas[1], schema.timestamp_column().unwrap());
assert_eq!(123, schema.version());
let new_schema = Schema::try_from(schema.arrow_schema().clone()).unwrap();
assert_eq!(1, schema.timestamp_index().unwrap());
assert_eq!(schema, new_schema);
}
#[test]
fn test_schema_wrong_timestamp() {
let column_schemas = vec![
ColumnSchema::new("col1", ConcreteDataType::int32_datatype(), true)
.with_time_index(true),
ColumnSchema::new("col2", ConcreteDataType::float64_datatype(), false),
];
assert!(SchemaBuilder::try_from(column_schemas)
.unwrap()
.build()
.is_err());
let column_schemas = vec![
ColumnSchema::new("col1", ConcreteDataType::int32_datatype(), true),
ColumnSchema::new("col2", ConcreteDataType::float64_datatype(), false)
.with_time_index(true),
];
assert!(SchemaBuilder::try_from(column_schemas)
.unwrap()
.build()
.is_err());
}
}

View File

@@ -1,304 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::fmt::{Display, Formatter};
use std::sync::Arc;
use common_time::util;
use serde::{Deserialize, Serialize};
use snafu::{ensure, ResultExt};
use crate::data_type::{ConcreteDataType, DataType};
use crate::error::{self, Result};
use crate::value::Value;
use crate::vectors::{Int64Vector, TimestampVector, VectorRef};
const CURRENT_TIMESTAMP: &str = "current_timestamp()";
/// Column's default constraint.
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)]
pub enum ColumnDefaultConstraint {
// A function invocation
// TODO(dennis): we save the function expression here, maybe use a struct in future.
Function(String),
// A value
Value(Value),
}
impl TryFrom<&[u8]> for ColumnDefaultConstraint {
type Error = error::Error;
fn try_from(bytes: &[u8]) -> Result<Self> {
let json = String::from_utf8_lossy(bytes);
serde_json::from_str(&json).context(error::DeserializeSnafu { json })
}
}
impl TryFrom<ColumnDefaultConstraint> for Vec<u8> {
type Error = error::Error;
fn try_from(value: ColumnDefaultConstraint) -> std::result::Result<Self, Self::Error> {
let s = serde_json::to_string(&value).context(error::SerializeSnafu)?;
Ok(s.into_bytes())
}
}
impl Display for ColumnDefaultConstraint {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
match self {
ColumnDefaultConstraint::Function(expr) => write!(f, "{}", expr),
ColumnDefaultConstraint::Value(v) => write!(f, "{}", v),
}
}
}
impl ColumnDefaultConstraint {
/// Returns a default null constraint.
pub fn null_value() -> ColumnDefaultConstraint {
ColumnDefaultConstraint::Value(Value::Null)
}
/// Check whether the constraint is valid for columns with given `data_type`
/// and `is_nullable` attributes.
pub fn validate(&self, data_type: &ConcreteDataType, is_nullable: bool) -> Result<()> {
ensure!(is_nullable || !self.maybe_null(), error::NullDefaultSnafu);
match self {
ColumnDefaultConstraint::Function(expr) => {
ensure!(
expr == CURRENT_TIMESTAMP,
error::UnsupportedDefaultExprSnafu { expr }
);
ensure!(
data_type.is_timestamp(),
error::DefaultValueTypeSnafu {
reason: "return value of the function must has timestamp type",
}
);
}
ColumnDefaultConstraint::Value(v) => {
if !v.is_null() {
// Whether the value could be nullable has been checked before, only need
// to check the type compatibility here.
ensure!(
data_type.logical_type_id() == v.logical_type_id(),
error::DefaultValueTypeSnafu {
reason: format!(
"column has type {:?} but default value has type {:?}",
data_type.logical_type_id(),
v.logical_type_id()
),
}
);
}
}
}
Ok(())
}
/// Create a vector that contains `num_rows` default values for given `data_type`.
///
/// If `is_nullable` is `true`, then this method would returns error if the created
/// default value is null.
///
/// # Panics
/// Panics if `num_rows == 0`.
pub fn create_default_vector(
&self,
data_type: &ConcreteDataType,
is_nullable: bool,
num_rows: usize,
) -> Result<VectorRef> {
assert!(num_rows > 0);
match self {
ColumnDefaultConstraint::Function(expr) => {
// Functions should also ensure its return value is not null when
// is_nullable is true.
match &expr[..] {
// TODO(dennis): we only supports current_timestamp right now,
// it's better to use a expression framework in future.
CURRENT_TIMESTAMP => create_current_timestamp_vector(data_type, num_rows),
_ => error::UnsupportedDefaultExprSnafu { expr }.fail(),
}
}
ColumnDefaultConstraint::Value(v) => {
ensure!(is_nullable || !v.is_null(), error::NullDefaultSnafu);
// TODO(yingwen):
// 1. For null value, we could use NullVector once it supports custom logical type.
// 2. For non null value, we could use ConstantVector, but it would cause all codes
// attempt to downcast the vector fail if they don't check whether the vector is const
// first.
let mut mutable_vector = data_type.create_mutable_vector(1);
mutable_vector.push_value_ref(v.as_value_ref())?;
let base_vector = mutable_vector.to_vector();
Ok(base_vector.replicate(&[num_rows]))
}
}
}
/// Returns true if this constraint might creates NULL.
fn maybe_null(&self) -> bool {
// Once we support more functions, we may return true if given function
// could return null.
matches!(self, ColumnDefaultConstraint::Value(Value::Null))
}
}
fn create_current_timestamp_vector(
data_type: &ConcreteDataType,
num_rows: usize,
) -> Result<VectorRef> {
match data_type {
ConcreteDataType::Timestamp(_) => Ok(Arc::new(TimestampVector::from_values(
std::iter::repeat(util::current_time_millis()).take(num_rows),
))),
ConcreteDataType::Int64(_) => Ok(Arc::new(Int64Vector::from_values(
std::iter::repeat(util::current_time_millis()).take(num_rows),
))),
_ => error::DefaultValueTypeSnafu {
reason: format!(
"Not support to assign current timestamp to {:?} type",
data_type
),
}
.fail(),
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::error::Error;
use crate::vectors::Int32Vector;
#[test]
fn test_null_default_constraint() {
let constraint = ColumnDefaultConstraint::null_value();
assert!(constraint.maybe_null());
let constraint = ColumnDefaultConstraint::Value(Value::Int32(10));
assert!(!constraint.maybe_null());
}
#[test]
fn test_validate_null_constraint() {
let constraint = ColumnDefaultConstraint::null_value();
let data_type = ConcreteDataType::int32_datatype();
constraint.validate(&data_type, false).unwrap_err();
constraint.validate(&data_type, true).unwrap();
}
#[test]
fn test_validate_value_constraint() {
let constraint = ColumnDefaultConstraint::Value(Value::Int32(10));
let data_type = ConcreteDataType::int32_datatype();
constraint.validate(&data_type, false).unwrap();
constraint.validate(&data_type, true).unwrap();
constraint
.validate(&ConcreteDataType::uint32_datatype(), true)
.unwrap_err();
}
#[test]
fn test_validate_function_constraint() {
let constraint = ColumnDefaultConstraint::Function(CURRENT_TIMESTAMP.to_string());
constraint
.validate(&ConcreteDataType::timestamp_millisecond_datatype(), false)
.unwrap();
constraint
.validate(&ConcreteDataType::boolean_datatype(), false)
.unwrap_err();
let constraint = ColumnDefaultConstraint::Function("hello()".to_string());
constraint
.validate(&ConcreteDataType::timestamp_millisecond_datatype(), false)
.unwrap_err();
}
#[test]
fn test_create_default_vector_by_null() {
let constraint = ColumnDefaultConstraint::null_value();
let data_type = ConcreteDataType::int32_datatype();
constraint
.create_default_vector(&data_type, false, 10)
.unwrap_err();
let constraint = ColumnDefaultConstraint::null_value();
let v = constraint
.create_default_vector(&data_type, true, 3)
.unwrap();
assert_eq!(3, v.len());
for i in 0..v.len() {
assert_eq!(Value::Null, v.get(i));
}
}
#[test]
fn test_create_default_vector_by_value() {
let constraint = ColumnDefaultConstraint::Value(Value::Int32(10));
let data_type = ConcreteDataType::int32_datatype();
let v = constraint
.create_default_vector(&data_type, false, 4)
.unwrap();
let expect: VectorRef = Arc::new(Int32Vector::from_values(vec![10; 4]));
assert_eq!(expect, v);
}
#[test]
fn test_create_default_vector_by_func() {
let constraint = ColumnDefaultConstraint::Function(CURRENT_TIMESTAMP.to_string());
// Timestamp type.
let data_type = ConcreteDataType::timestamp_millisecond_datatype();
let v = constraint
.create_default_vector(&data_type, false, 4)
.unwrap();
assert_eq!(4, v.len());
assert!(
matches!(v.get(0), Value::Timestamp(_)),
"v {:?} is not timestamp",
v.get(0)
);
// Int64 type.
let data_type = ConcreteDataType::int64_datatype();
let v = constraint
.create_default_vector(&data_type, false, 4)
.unwrap();
assert_eq!(4, v.len());
assert!(
matches!(v.get(0), Value::Int64(_)),
"v {:?} is not timestamp",
v.get(0)
);
let constraint = ColumnDefaultConstraint::Function("no".to_string());
let data_type = ConcreteDataType::timestamp_millisecond_datatype();
constraint
.create_default_vector(&data_type, false, 4)
.unwrap_err();
}
#[test]
fn test_create_by_func_and_invalid_type() {
let constraint = ColumnDefaultConstraint::Function(CURRENT_TIMESTAMP.to_string());
let data_type = ConcreteDataType::boolean_datatype();
let err = constraint
.create_default_vector(&data_type, false, 4)
.unwrap_err();
assert!(matches!(err, Error::DefaultValueType { .. }), "{:?}", err);
}
}

View File

@@ -1,77 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 serde::{Deserialize, Serialize};
use crate::error::{Error, Result};
use crate::schema::{ColumnSchema, Schema, SchemaBuilder};
/// Struct used to serialize and deserialize [`Schema`](crate::schema::Schema).
///
/// This struct only contains necessary data to recover the Schema.
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
pub struct RawSchema {
pub column_schemas: Vec<ColumnSchema>,
pub timestamp_index: Option<usize>,
pub version: u32,
}
impl TryFrom<RawSchema> for Schema {
type Error = Error;
fn try_from(raw: RawSchema) -> Result<Schema> {
SchemaBuilder::try_from(raw.column_schemas)?
.version(raw.version)
.build()
}
}
impl From<&Schema> for RawSchema {
fn from(schema: &Schema) -> RawSchema {
RawSchema {
column_schemas: schema.column_schemas.clone(),
timestamp_index: schema.timestamp_index,
version: schema.version,
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::data_type::ConcreteDataType;
#[test]
fn test_raw_convert() {
let column_schemas = vec![
ColumnSchema::new("col1", ConcreteDataType::int32_datatype(), true),
ColumnSchema::new(
"ts",
ConcreteDataType::timestamp_millisecond_datatype(),
false,
)
.with_time_index(true),
];
let schema = SchemaBuilder::try_from(column_schemas)
.unwrap()
.version(123)
.build()
.unwrap();
let raw = RawSchema::from(&schema);
let schema_new = Schema::try_from(raw).unwrap();
assert_eq!(schema, schema_new);
}
}

View File

@@ -1,20 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 crate::error::Result;
pub trait Serializable: Send + Sync {
/// Serialize a column of value with given type to JSON value
fn serialize_to_json(&self) -> Result<Vec<serde_json::Value>>;
}

View File

@@ -1,83 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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.
/// Unique identifier for logical data type.
#[derive(Clone, Copy, Debug, PartialEq, Eq)]
pub enum LogicalTypeId {
Null,
// Numeric types:
Boolean,
Int8,
Int16,
Int32,
Int64,
UInt8,
UInt16,
UInt32,
UInt64,
Float32,
Float64,
// String types:
String,
Binary,
// Date & Time types:
/// Date representing the elapsed time since UNIX epoch (1970-01-01)
/// in days (32 bits).
Date,
/// Datetime representing the elapsed time since UNIX epoch (1970-01-01) in
/// seconds/milliseconds/microseconds/nanoseconds, determined by precision.
DateTime,
Timestamp,
List,
}
impl LogicalTypeId {
/// Create ConcreteDataType based on this id. This method is for test only as it
/// would lost some info.
///
/// # Panics
/// Panics if data type is not supported.
#[cfg(any(test, feature = "test"))]
pub fn data_type(&self) -> crate::data_type::ConcreteDataType {
use crate::data_type::ConcreteDataType;
match self {
LogicalTypeId::Null => ConcreteDataType::null_datatype(),
LogicalTypeId::Boolean => ConcreteDataType::boolean_datatype(),
LogicalTypeId::Int8 => ConcreteDataType::int8_datatype(),
LogicalTypeId::Int16 => ConcreteDataType::int16_datatype(),
LogicalTypeId::Int32 => ConcreteDataType::int32_datatype(),
LogicalTypeId::Int64 => ConcreteDataType::int64_datatype(),
LogicalTypeId::UInt8 => ConcreteDataType::uint8_datatype(),
LogicalTypeId::UInt16 => ConcreteDataType::uint16_datatype(),
LogicalTypeId::UInt32 => ConcreteDataType::uint32_datatype(),
LogicalTypeId::UInt64 => ConcreteDataType::uint64_datatype(),
LogicalTypeId::Float32 => ConcreteDataType::float32_datatype(),
LogicalTypeId::Float64 => ConcreteDataType::float64_datatype(),
LogicalTypeId::String => ConcreteDataType::string_datatype(),
LogicalTypeId::Binary => ConcreteDataType::binary_datatype(),
LogicalTypeId::Date => ConcreteDataType::date_datatype(),
LogicalTypeId::DateTime => ConcreteDataType::datetime_datatype(),
LogicalTypeId::Timestamp => ConcreteDataType::timestamp_millisecond_datatype(), // to timestamp type with default time unit
LogicalTypeId::List => {
ConcreteDataType::list_datatype(ConcreteDataType::null_datatype())
}
}
}
}

View File

@@ -1,38 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 binary_type;
mod boolean_type;
mod date;
mod datetime;
mod list_type;
mod null_type;
mod primitive_traits;
mod primitive_type;
mod string_type;
mod timestamp;
pub use binary_type::BinaryType;
pub use boolean_type::BooleanType;
pub use date::DateType;
pub use datetime::DateTimeType;
pub use list_type::ListType;
pub use null_type::NullType;
pub use primitive_traits::{OrdPrimitive, Primitive};
pub use primitive_type::{
Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, PrimitiveElement,
PrimitiveType, UInt16Type, UInt32Type, UInt64Type, UInt8Type,
};
pub use string_type::StringType;
pub use timestamp::TimestampType;

View File

@@ -1,56 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::sync::Arc;
use arrow::datatypes::DataType as ArrowDataType;
use common_base::bytes::StringBytes;
use serde::{Deserialize, Serialize};
use crate::data_type::{DataType, DataTypeRef};
use crate::scalars::ScalarVectorBuilder;
use crate::type_id::LogicalTypeId;
use crate::value::Value;
use crate::vectors::{BinaryVectorBuilder, MutableVector};
#[derive(Debug, Default, Clone, PartialEq, Eq, Serialize, Deserialize)]
pub struct BinaryType;
impl BinaryType {
pub fn arc() -> DataTypeRef {
Arc::new(Self)
}
}
impl DataType for BinaryType {
fn name(&self) -> &str {
"Binary"
}
fn logical_type_id(&self) -> LogicalTypeId {
LogicalTypeId::Binary
}
fn default_value(&self) -> Value {
StringBytes::default().into()
}
fn as_arrow_type(&self) -> ArrowDataType {
ArrowDataType::LargeBinary
}
fn create_mutable_vector(&self, capacity: usize) -> Box<dyn MutableVector> {
Box::new(BinaryVectorBuilder::with_capacity(capacity))
}
}

View File

@@ -1,55 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::sync::Arc;
use arrow::datatypes::DataType as ArrowDataType;
use serde::{Deserialize, Serialize};
use crate::data_type::{DataType, DataTypeRef};
use crate::scalars::ScalarVectorBuilder;
use crate::type_id::LogicalTypeId;
use crate::value::Value;
use crate::vectors::{BooleanVectorBuilder, MutableVector};
#[derive(Debug, Default, Clone, PartialEq, Eq, Serialize, Deserialize)]
pub struct BooleanType;
impl BooleanType {
pub fn arc() -> DataTypeRef {
Arc::new(Self)
}
}
impl DataType for BooleanType {
fn name(&self) -> &str {
"Boolean"
}
fn logical_type_id(&self) -> LogicalTypeId {
LogicalTypeId::Boolean
}
fn default_value(&self) -> Value {
bool::default().into()
}
fn as_arrow_type(&self) -> ArrowDataType {
ArrowDataType::Boolean
}
fn create_mutable_vector(&self, capacity: usize) -> Box<dyn MutableVector> {
Box::new(BooleanVectorBuilder::with_capacity(capacity))
}
}

View File

@@ -1,54 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::sync::Arc;
use arrow::datatypes::DataType as ArrowDataType;
use serde::{Deserialize, Serialize};
use crate::data_type::DataType;
use crate::prelude::{DataTypeRef, LogicalTypeId, Value};
use crate::scalars::ScalarVectorBuilder;
use crate::vectors::{DateVectorBuilder, MutableVector};
#[derive(Debug, Default, Clone, PartialEq, Eq, Serialize, Deserialize)]
pub struct DateType;
impl DataType for DateType {
fn name(&self) -> &str {
"Date"
}
fn logical_type_id(&self) -> LogicalTypeId {
LogicalTypeId::Date
}
fn default_value(&self) -> Value {
Value::Date(Default::default())
}
fn as_arrow_type(&self) -> ArrowDataType {
ArrowDataType::Date32
}
fn create_mutable_vector(&self, capacity: usize) -> Box<dyn MutableVector> {
Box::new(DateVectorBuilder::with_capacity(capacity))
}
}
impl DateType {
pub fn arc() -> DataTypeRef {
Arc::new(Self)
}
}

View File

@@ -1,61 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::sync::Arc;
use arrow::datatypes::DataType as ArrowDataType;
use serde::{Deserialize, Serialize};
use crate::data_type::{DataType, DataTypeRef};
use crate::prelude::{LogicalTypeId, Value};
use crate::scalars::ScalarVectorBuilder;
use crate::vectors::{DateTimeVectorBuilder, MutableVector};
#[derive(Debug, Default, Clone, PartialEq, Eq, Serialize, Deserialize)]
pub struct DateTimeType;
const DATE_TIME_TYPE_NAME: &str = "DateTime";
/// [DateTimeType] represents the seconds elapsed since UNIX EPOCH.
impl DataType for DateTimeType {
fn name(&self) -> &str {
DATE_TIME_TYPE_NAME
}
fn logical_type_id(&self) -> LogicalTypeId {
LogicalTypeId::DateTime
}
fn default_value(&self) -> Value {
Value::DateTime(Default::default())
}
fn as_arrow_type(&self) -> ArrowDataType {
ArrowDataType::Date64
}
fn create_mutable_vector(&self, capacity: usize) -> Box<dyn MutableVector> {
Box::new(DateTimeVectorBuilder::with_capacity(capacity))
}
}
impl DateTimeType {
pub fn arc() -> DataTypeRef {
Arc::new(Self)
}
pub fn name() -> &'static str {
DATE_TIME_TYPE_NAME
}
}

View File

@@ -1,88 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 arrow::datatypes::{DataType as ArrowDataType, Field};
use serde::{Deserialize, Serialize};
use crate::prelude::*;
use crate::value::ListValue;
use crate::vectors::{ListVectorBuilder, MutableVector};
/// Used to represent the List datatype.
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
pub struct ListType {
/// The type of List's inner data.
inner: Box<ConcreteDataType>,
}
impl Default for ListType {
fn default() -> Self {
ListType::new(ConcreteDataType::null_datatype())
}
}
impl ListType {
pub fn new(datatype: ConcreteDataType) -> Self {
ListType {
inner: Box::new(datatype),
}
}
}
impl DataType for ListType {
fn name(&self) -> &str {
"List"
}
fn logical_type_id(&self) -> LogicalTypeId {
LogicalTypeId::List
}
fn default_value(&self) -> Value {
Value::List(ListValue::new(None, *self.inner.clone()))
}
fn as_arrow_type(&self) -> ArrowDataType {
let field = Box::new(Field::new("item", self.inner.as_arrow_type(), true));
ArrowDataType::List(field)
}
fn create_mutable_vector(&self, capacity: usize) -> Box<dyn MutableVector> {
Box::new(ListVectorBuilder::with_type_capacity(
*self.inner.clone(),
capacity,
))
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::value::ListValue;
#[test]
fn test_list_type() {
let t = ListType::new(ConcreteDataType::boolean_datatype());
assert_eq!("List", t.name());
assert_eq!(LogicalTypeId::List, t.logical_type_id());
assert_eq!(
Value::List(ListValue::new(None, ConcreteDataType::boolean_datatype())),
t.default_value()
);
assert_eq!(
ArrowDataType::List(Box::new(Field::new("item", ArrowDataType::Boolean, true))),
t.as_arrow_type()
);
}
}

View File

@@ -1,54 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::sync::Arc;
use arrow::datatypes::DataType as ArrowDataType;
use serde::{Deserialize, Serialize};
use crate::data_type::{DataType, DataTypeRef};
use crate::type_id::LogicalTypeId;
use crate::value::Value;
use crate::vectors::{MutableVector, NullVectorBuilder};
#[derive(Debug, Default, Clone, PartialEq, Eq, Serialize, Deserialize)]
pub struct NullType;
impl NullType {
pub fn arc() -> DataTypeRef {
Arc::new(Self)
}
}
impl DataType for NullType {
fn name(&self) -> &str {
"Null"
}
fn logical_type_id(&self) -> LogicalTypeId {
LogicalTypeId::Null
}
fn default_value(&self) -> Value {
Value::Null
}
fn as_arrow_type(&self) -> ArrowDataType {
ArrowDataType::Null
}
fn create_mutable_vector(&self, _capacity: usize) -> Box<dyn MutableVector> {
Box::new(NullVectorBuilder::default())
}
}

View File

@@ -1,136 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::cmp::Ordering;
use arrow::datatypes::ArrowPrimitiveType;
use num::NumCast;
use crate::prelude::Scalar;
use crate::value::{IntoValueRef, Value};
/// Primitive type.
pub trait Primitive:
PartialOrd
+ Default
+ Clone
+ Copy
+ Into<Value>
+ IntoValueRef<'static>
+ ArrowPrimitiveType
+ serde::Serialize
+ NumCast
+ Scalar
{
/// Largest numeric type this primitive type can be cast to.
type LargestType: Primitive;
}
macro_rules! impl_primitive {
($Type:ident, $LargestType: ident) => {
impl Primitive for $Type {
type LargestType = $LargestType;
}
};
}
impl_primitive!(u8, u64);
impl_primitive!(u16, u64);
impl_primitive!(u32, u64);
impl_primitive!(u64, u64);
impl_primitive!(i8, i64);
impl_primitive!(i16, i64);
impl_primitive!(i32, i64);
impl_primitive!(i64, i64);
impl_primitive!(f32, f64);
impl_primitive!(f64, f64);
/// A new type for [Primitive], complement the `Ord` feature for it. Wrapping not ordered
/// primitive types like `f32` and `f64` in `OrdPrimitive` can make them be used in places that
/// require `Ord`. For example, in `Median` or `Percentile` UDAFs.
#[derive(Debug, Clone, Copy, PartialEq)]
pub struct OrdPrimitive<T: Primitive>(pub T);
impl<T: Primitive> OrdPrimitive<T> {
pub fn as_primitive(&self) -> T {
self.0
}
}
impl<T: Primitive> Eq for OrdPrimitive<T> {}
impl<T: Primitive> PartialOrd for OrdPrimitive<T> {
fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
Some(self.cmp(other))
}
}
impl<T: Primitive> Ord for OrdPrimitive<T> {
fn cmp(&self, other: &Self) -> Ordering {
self.0.into().cmp(&other.0.into())
}
}
impl<T: Primitive> From<OrdPrimitive<T>> for Value {
fn from(p: OrdPrimitive<T>) -> Self {
p.0.into()
}
}
#[cfg(test)]
mod tests {
use std::collections::BinaryHeap;
use super::*;
#[test]
fn test_ord_primitive() {
struct Foo<T>
where
T: Primitive,
{
heap: BinaryHeap<OrdPrimitive<T>>,
}
impl<T> Foo<T>
where
T: Primitive,
{
fn push(&mut self, value: T) {
let value = OrdPrimitive::<T>(value);
self.heap.push(value);
}
}
macro_rules! test {
($Type:ident) => {
let mut foo = Foo::<$Type> {
heap: BinaryHeap::new(),
};
foo.push($Type::default());
};
}
test!(u8);
test!(u16);
test!(u32);
test!(u64);
test!(i8);
test!(i16);
test!(i32);
test!(i64);
test!(f32);
test!(f64);
}
}

View File

@@ -1,188 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::TypeId;
use std::marker::PhantomData;
use arrow::array::PrimitiveArray;
use arrow::datatypes::DataType as ArrowDataType;
use paste::paste;
use serde::{Deserialize, Serialize};
use snafu::OptionExt;
use crate::data_type::{ConcreteDataType, DataType};
use crate::error::{self, Result};
use crate::scalars::{Scalar, ScalarRef, ScalarVectorBuilder};
use crate::type_id::LogicalTypeId;
use crate::types::primitive_traits::Primitive;
use crate::value::{Value, ValueRef};
use crate::vectors::{MutableVector, PrimitiveVector, PrimitiveVectorBuilder, Vector};
#[derive(Clone, Serialize, Deserialize)]
pub struct PrimitiveType<T: Primitive> {
#[serde(skip)]
_phantom: PhantomData<T>,
}
impl<T: Primitive, U: Primitive> PartialEq<PrimitiveType<U>> for PrimitiveType<T> {
fn eq(&self, _other: &PrimitiveType<U>) -> bool {
TypeId::of::<T>() == TypeId::of::<U>()
}
}
impl<T: Primitive> Eq for PrimitiveType<T> {}
/// A trait that provide helper methods for a primitive type to implementing the [PrimitiveVector].
pub trait PrimitiveElement
where
for<'a> Self: Primitive
+ Scalar<VectorType = PrimitiveVector<Self>>
+ ScalarRef<'a, ScalarType = Self, VectorType = PrimitiveVector<Self>>
+ Scalar<RefType<'a> = Self>,
{
/// Construct the data type struct.
fn build_data_type() -> ConcreteDataType;
/// Returns the name of the type id.
fn type_name() -> String;
/// Dynamic cast the vector to the concrete vector type.
fn cast_vector(vector: &dyn Vector) -> Result<&PrimitiveArray<Self>>;
/// Cast value ref to the primitive type.
fn cast_value_ref(value: ValueRef) -> Result<Option<Self>>;
}
macro_rules! impl_primitive_element {
($Type:ident, $TypeId:ident) => {
paste::paste! {
impl PrimitiveElement for $Type {
fn build_data_type() -> ConcreteDataType {
ConcreteDataType::$TypeId(PrimitiveType::<$Type>::default())
}
fn type_name() -> String {
stringify!($TypeId).to_string()
}
fn cast_vector(vector: &dyn Vector) -> Result<&PrimitiveArray<$Type>> {
let primitive_vector = vector
.as_any()
.downcast_ref::<PrimitiveVector<$Type>>()
.with_context(|| error::CastTypeSnafu {
msg: format!(
"Failed to cast {} to vector of primitive type {}",
vector.vector_type_name(),
stringify!($TypeId)
),
})?;
Ok(&primitive_vector.array)
}
fn cast_value_ref(value: ValueRef) -> Result<Option<Self>> {
match value {
ValueRef::Null => Ok(None),
ValueRef::$TypeId(v) => Ok(Some(v.into())),
other => error::CastTypeSnafu {
msg: format!(
"Failed to cast value {:?} to primitive type {}",
other,
stringify!($TypeId),
),
}.fail(),
}
}
}
}
};
}
macro_rules! impl_numeric {
($Type:ident, $TypeId:ident) => {
impl DataType for PrimitiveType<$Type> {
fn name(&self) -> &str {
stringify!($TypeId)
}
fn logical_type_id(&self) -> LogicalTypeId {
LogicalTypeId::$TypeId
}
fn default_value(&self) -> Value {
$Type::default().into()
}
fn as_arrow_type(&self) -> ArrowDataType {
ArrowDataType::$TypeId
}
fn create_mutable_vector(&self, capacity: usize) -> Box<dyn MutableVector> {
Box::new(PrimitiveVectorBuilder::<$Type>::with_capacity(capacity))
}
}
impl std::fmt::Debug for PrimitiveType<$Type> {
fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
write!(f, "{}", self.name())
}
}
impl Default for PrimitiveType<$Type> {
fn default() -> Self {
Self {
_phantom: PhantomData,
}
}
}
impl_primitive_element!($Type, $TypeId);
paste! {
pub type [<$TypeId Type>]=PrimitiveType<$Type>;
}
};
}
impl_numeric!(u8, UInt8);
impl_numeric!(u16, UInt16);
impl_numeric!(u32, UInt32);
impl_numeric!(u64, UInt64);
impl_numeric!(i8, Int8);
impl_numeric!(i16, Int16);
impl_numeric!(i32, Int32);
impl_numeric!(i64, Int64);
impl_numeric!(f32, Float32);
impl_numeric!(f64, Float64);
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_eq() {
assert_eq!(UInt8Type::default(), UInt8Type::default());
assert_eq!(UInt16Type::default(), UInt16Type::default());
assert_eq!(UInt32Type::default(), UInt32Type::default());
assert_eq!(UInt64Type::default(), UInt64Type::default());
assert_eq!(Int8Type::default(), Int8Type::default());
assert_eq!(Int16Type::default(), Int16Type::default());
assert_eq!(Int32Type::default(), Int32Type::default());
assert_eq!(Int64Type::default(), Int64Type::default());
assert_eq!(Float32Type::default(), Float32Type::default());
assert_eq!(Float64Type::default(), Float64Type::default());
assert_ne!(Float32Type::default(), Float64Type::default());
assert_ne!(Float32Type::default(), Int32Type::default());
}
}

View File

@@ -1,55 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::sync::Arc;
use arrow::datatypes::DataType as ArrowDataType;
use common_base::bytes::StringBytes;
use serde::{Deserialize, Serialize};
use crate::data_type::DataType;
use crate::prelude::{DataTypeRef, LogicalTypeId, Value};
use crate::scalars::ScalarVectorBuilder;
use crate::vectors::{MutableVector, StringVectorBuilder};
#[derive(Debug, Default, Clone, PartialEq, Eq, Serialize, Deserialize)]
pub struct StringType;
impl StringType {
pub fn arc() -> DataTypeRef {
Arc::new(Self)
}
}
impl DataType for StringType {
fn name(&self) -> &str {
"String"
}
fn logical_type_id(&self) -> LogicalTypeId {
LogicalTypeId::String
}
fn default_value(&self) -> Value {
StringBytes::default().into()
}
fn as_arrow_type(&self) -> ArrowDataType {
ArrowDataType::Utf8
}
fn create_mutable_vector(&self, capacity: usize) -> Box<dyn MutableVector> {
Box::new(StringVectorBuilder::with_capacity(capacity))
}
}

View File

@@ -1,128 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 arrow::datatypes::{DataType as ArrowDataType, TimeUnit as ArrowTimeUnit};
use common_time::timestamp::{TimeUnit, Timestamp};
use serde::{Deserialize, Serialize};
use crate::data_type::DataType;
use crate::prelude::{LogicalTypeId, MutableVector, ScalarVectorBuilder, Value};
use crate::vectors::TimestampVectorBuilder;
#[derive(Debug, Default, Clone, PartialEq, Eq, Serialize, Deserialize)]
pub struct TimestampType {
pub unit: TimeUnit,
}
impl TimestampType {
pub fn new(unit: TimeUnit) -> Self {
Self { unit }
}
}
impl DataType for TimestampType {
fn name(&self) -> &str {
"Timestamp"
}
fn logical_type_id(&self) -> LogicalTypeId {
LogicalTypeId::Timestamp
}
fn default_value(&self) -> Value {
Value::Timestamp(Timestamp::new(0, self.unit))
}
fn as_arrow_type(&self) -> ArrowDataType {
match self.unit {
TimeUnit::Second => ArrowDataType::Timestamp(ArrowTimeUnit::Second, None),
TimeUnit::Millisecond => ArrowDataType::Timestamp(ArrowTimeUnit::Millisecond, None),
TimeUnit::Microsecond => ArrowDataType::Timestamp(ArrowTimeUnit::Microsecond, None),
TimeUnit::Nanosecond => ArrowDataType::Timestamp(ArrowTimeUnit::Nanosecond, None),
}
}
fn create_mutable_vector(&self, capacity: usize) -> Box<dyn MutableVector> {
Box::new(TimestampVectorBuilder::with_capacity(capacity))
}
}
#[cfg(test)]
mod tests {
use arrow::datatypes::TimeUnit as ArrowTimeUnit;
use common_time::timestamp::TimeUnit::Microsecond;
use super::*;
use crate::prelude::{ConcreteDataType, ValueRef};
#[test]
pub fn test_timestamp_type() {
assert_eq!(
LogicalTypeId::Timestamp,
TimestampType::new(TimeUnit::Microsecond).logical_type_id()
);
}
#[test]
pub fn test_as_arrow_type() {
assert_eq!(
ArrowDataType::Timestamp(ArrowTimeUnit::Nanosecond, None),
TimestampType::new(TimeUnit::Nanosecond).as_arrow_type()
);
assert_eq!(
ArrowDataType::Timestamp(ArrowTimeUnit::Microsecond, None),
TimestampType::new(TimeUnit::Microsecond).as_arrow_type()
);
assert_eq!(
ArrowDataType::Timestamp(ArrowTimeUnit::Millisecond, None),
TimestampType::new(TimeUnit::Millisecond).as_arrow_type()
);
assert_eq!(
ArrowDataType::Timestamp(ArrowTimeUnit::Second, None),
TimestampType::new(TimeUnit::Second).as_arrow_type()
);
}
#[test]
pub fn test_default_value() {
assert_eq!(
Value::Timestamp(Timestamp::new(0, Microsecond)),
TimestampType::new(TimeUnit::Microsecond).default_value()
);
}
#[test]
pub fn test_create_mutable_vector() {
let mut builder = TimestampType::new(TimeUnit::Microsecond).create_mutable_vector(10);
builder
.push_value_ref(ValueRef::Timestamp(Timestamp::new(
42,
TimeUnit::Millisecond,
)))
.unwrap();
builder.push_value_ref(ValueRef::Null).unwrap();
builder
.push_value_ref(ValueRef::Timestamp(Timestamp::new(96, TimeUnit::Second)))
.unwrap();
let v = builder.to_vector();
assert_eq!(
ConcreteDataType::timestamp_millisecond_datatype(),
v.data_type()
);
assert_eq!(Value::Timestamp(Timestamp::from_millis(42)), v.get(0));
assert_eq!(Value::Null, v.get(1));
// Push a timestamp with different unit will convert the value to value with time unit millisecond.
assert_eq!(Value::Timestamp(Timestamp::from_millis(96_000)), v.get(2));
}
}

File diff suppressed because it is too large Load Diff

View File

@@ -1,285 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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.
pub mod binary;
pub mod boolean;
mod builder;
pub mod constant;
pub mod date;
pub mod datetime;
mod eq;
mod helper;
mod list;
pub mod mutable;
pub mod null;
mod operations;
pub mod primitive;
mod string;
mod timestamp;
use std::any::Any;
use std::fmt::Debug;
use std::sync::Arc;
use arrow::array::{Array, ArrayRef};
use arrow::bitmap::Bitmap;
pub use binary::*;
pub use boolean::*;
pub use builder::VectorBuilder;
pub use constant::*;
pub use date::*;
pub use datetime::*;
pub use helper::Helper;
pub use list::*;
pub use mutable::MutableVector;
pub use null::*;
pub use operations::VectorOp;
pub use primitive::*;
use snafu::ensure;
pub use string::*;
pub use timestamp::*;
use crate::data_type::ConcreteDataType;
use crate::error::{self, Result};
use crate::serialize::Serializable;
use crate::value::{Value, ValueRef};
#[derive(Debug, PartialEq)]
pub enum Validity<'a> {
/// Whether the array slot is valid or not (null).
Slots(&'a Bitmap),
/// All slots are valid.
AllValid,
/// All slots are null.
AllNull,
}
impl<'a> Validity<'a> {
pub fn slots(&self) -> Option<&Bitmap> {
match self {
Validity::Slots(bitmap) => Some(bitmap),
_ => None,
}
}
}
/// Vector of data values.
pub trait Vector: Send + Sync + Serializable + Debug + VectorOp {
/// Returns the data type of the vector.
///
/// This may require heap allocation.
fn data_type(&self) -> ConcreteDataType;
fn vector_type_name(&self) -> String;
/// Returns the vector as [Any](std::any::Any) so that it can be
/// downcast to a specific implementation.
fn as_any(&self) -> &dyn Any;
/// Returns number of elements in the vector.
fn len(&self) -> usize;
/// Returns whether the vector is empty.
fn is_empty(&self) -> bool {
self.len() == 0
}
/// Convert this vector to a new arrow [ArrayRef].
fn to_arrow_array(&self) -> ArrayRef;
/// Convert this vector to a new boxed arrow [Array].
fn to_boxed_arrow_array(&self) -> Box<dyn Array>;
/// Returns the validity of the Array.
fn validity(&self) -> Validity;
/// Returns the memory size of vector.
fn memory_size(&self) -> usize;
/// The number of null slots on this [`Vector`].
/// # Implementation
/// This is `O(1)`.
fn null_count(&self) -> usize {
match self.validity() {
Validity::Slots(bitmap) => bitmap.null_count(),
Validity::AllValid => 0,
Validity::AllNull => self.len(),
}
}
/// Returns true when it's a ConstantColumn
fn is_const(&self) -> bool {
false
}
/// Returns whether row is null.
fn is_null(&self, row: usize) -> bool;
/// If the only value vector can contain is NULL.
fn only_null(&self) -> bool {
self.null_count() == self.len()
}
/// Slices the `Vector`, returning a new `VectorRef`.
///
/// # Panics
/// This function panics if `offset + length > self.len()`.
fn slice(&self, offset: usize, length: usize) -> VectorRef;
/// Returns the clone of value at `index`.
///
/// # Panics
/// Panic if `index` is out of bound.
fn get(&self, index: usize) -> Value;
/// Returns the clone of value at `index` or error if `index`
/// is out of bound.
fn try_get(&self, index: usize) -> Result<Value> {
ensure!(
index < self.len(),
error::BadArrayAccessSnafu {
index,
size: self.len()
}
);
Ok(self.get(index))
}
/// Returns the reference of value at `index`.
///
/// # Panics
/// Panic if `index` is out of bound.
fn get_ref(&self, index: usize) -> ValueRef;
}
pub type VectorRef = Arc<dyn Vector>;
/// Helper to define `try_from_arrow_array(array: arrow::array::ArrayRef)` function.
macro_rules! impl_try_from_arrow_array_for_vector {
($Array: ident, $Vector: ident) => {
impl $Vector {
pub fn try_from_arrow_array(
array: impl AsRef<dyn arrow::array::Array>,
) -> crate::error::Result<$Vector> {
Ok($Vector::from(
array
.as_ref()
.as_any()
.downcast_ref::<$Array>()
.with_context(|| crate::error::ConversionSnafu {
from: std::format!("{:?}", array.as_ref().data_type()),
})?
.clone(),
))
}
}
};
}
macro_rules! impl_validity_for_vector {
($array: expr) => {
match $array.validity() {
Some(bitmap) => Validity::Slots(bitmap),
None => Validity::AllValid,
}
};
}
macro_rules! impl_get_for_vector {
($array: expr, $index: ident) => {
if $array.is_valid($index) {
// Safety: The index have been checked by `is_valid()`.
unsafe { $array.value_unchecked($index).into() }
} else {
Value::Null
}
};
}
macro_rules! impl_get_ref_for_vector {
($array: expr, $index: ident) => {
if $array.is_valid($index) {
// Safety: The index have been checked by `is_valid()`.
unsafe { $array.value_unchecked($index).into() }
} else {
ValueRef::Null
}
};
}
macro_rules! impl_extend_for_builder {
($mutable_array: expr, $vector: ident, $VectorType: ident, $offset: ident, $length: ident) => {{
use snafu::OptionExt;
let concrete_vector = $vector
.as_any()
.downcast_ref::<$VectorType>()
.with_context(|| crate::error::CastTypeSnafu {
msg: format!(
"Failed to cast vector from {} to {}",
$vector.vector_type_name(),
stringify!($VectorType)
),
})?;
let slice = concrete_vector.array.slice($offset, $length);
$mutable_array.extend_trusted_len(slice.iter());
Ok(())
}};
}
pub(crate) use {
impl_extend_for_builder, impl_get_for_vector, impl_get_ref_for_vector,
impl_try_from_arrow_array_for_vector, impl_validity_for_vector,
};
#[cfg(test)]
pub mod tests {
use arrow::array::{Array, PrimitiveArray};
use serde_json;
use super::helper::Helper;
use super::*;
use crate::data_type::DataType;
use crate::types::PrimitiveElement;
#[test]
fn test_df_columns_to_vector() {
let df_column: Arc<dyn Array> = Arc::new(PrimitiveArray::from_slice(vec![1, 2, 3]));
let vector = Helper::try_into_vector(df_column).unwrap();
assert_eq!(
i32::build_data_type().as_arrow_type(),
vector.data_type().as_arrow_type()
);
}
#[test]
fn test_serialize_i32_vector() {
let df_column: Arc<dyn Array> = Arc::new(PrimitiveArray::<i32>::from_slice(vec![1, 2, 3]));
let json_value = Helper::try_into_vector(df_column)
.unwrap()
.serialize_to_json()
.unwrap();
assert_eq!("[1,2,3]", serde_json::to_string(&json_value).unwrap());
}
#[test]
fn test_serialize_i8_vector() {
let df_column: Arc<dyn Array> = Arc::new(PrimitiveArray::from_slice(vec![1u8, 2u8, 3u8]));
let json_value = Helper::try_into_vector(df_column)
.unwrap()
.serialize_to_json()
.unwrap();
assert_eq!("[1,2,3]", serde_json::to_string(&json_value).unwrap());
}
}

View File

@@ -1,323 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use std::sync::Arc;
use arrow::array::{Array, ArrayIter, ArrayRef, GenericByteArray};
use snafu::{OptionExt, ResultExt};
use crate::arrow_array::{BinaryArray, MutableBinaryArray};
use crate::data_type::ConcreteDataType;
use crate::error::{self, Result};
use crate::scalars::{ScalarVector, ScalarVectorBuilder};
use crate::serialize::Serializable;
use crate::value::{Value, ValueRef};
use crate::vectors::{self, MutableVector, Validity, Vector, VectorRef};
/// Vector of binary strings.
#[derive(Debug, PartialEq)]
pub struct BinaryVector {
array: BinaryArray,
}
impl BinaryVector {
pub(crate) fn as_arrow(&self) -> &dyn Array {
&self.array
}
}
impl From<BinaryArray> for BinaryVector {
fn from(array: BinaryArray) -> Self {
Self { array }
}
}
impl From<Vec<Option<Vec<u8>>>> for BinaryVector {
fn from(data: Vec<Option<Vec<u8>>>) -> Self {
Self {
array: BinaryArray::from(data),
}
}
}
impl Vector for BinaryVector {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::binary_datatype()
}
fn vector_type_name(&self) -> String {
"BinaryVector".to_string()
}
fn as_any(&self) -> &dyn Any {
self
}
fn len(&self) -> usize {
self.array.len()
}
fn to_arrow_array(&self) -> ArrayRef {
Arc::new(self.array.clone())
}
fn to_boxed_arrow_array(&self) -> Box<dyn Array> {
Box::new(self.array.clone())
}
fn validity(&self) -> Validity {
vectors::impl_validity_for_vector!(self.array)
}
fn memory_size(&self) -> usize {
self.array.values().len() + self.array.offsets().len() * std::mem::size_of::<i64>()
}
fn is_null(&self, row: usize) -> bool {
self.array.is_null(row)
}
fn slice(&self, offset: usize, length: usize) -> VectorRef {
Arc::new(Self::from(self.array.slice(offset, length)))
}
fn get(&self, index: usize) -> Value {
vectors::impl_get_for_vector!(self.array, index)
}
fn get_ref(&self, index: usize) -> ValueRef {
vectors::impl_get_ref_for_vector!(self.array, index)
}
}
impl ScalarVector for BinaryVector {
type OwnedItem = Vec<u8>;
type RefItem<'a> = &'a [u8];
type Iter<'a> = ArrayIter<&'a BinaryArray>;
type Builder = BinaryVectorBuilder;
fn get_data(&self, idx: usize) -> Option<Self::RefItem<'_>> {
if self.array.is_valid(idx) {
Some(self.array.value(idx))
} else {
None
}
}
fn iter_data(&self) -> Self::Iter<'_> {
self.array.iter()
}
}
pub struct BinaryVectorBuilder {
mutable_array: MutableBinaryArray,
}
impl MutableVector for BinaryVectorBuilder {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::binary_datatype()
}
fn len(&self) -> usize {
self.mutable_array.len()
}
fn as_any(&self) -> &dyn Any {
self
}
fn as_mut_any(&mut self) -> &mut dyn Any {
self
}
fn to_vector(&mut self) -> VectorRef {
Arc::new(self.finish())
}
fn push_value_ref(&mut self, value: ValueRef) -> Result<()> {
self.mutable_array.push(value.as_binary()?);
Ok(())
}
fn extend_slice_of(&mut self, vector: &dyn Vector, offset: usize, length: usize) -> Result<()> {
vectors::impl_extend_for_builder!(self.mutable_array, vector, BinaryVector, offset, length)
}
}
impl ScalarVectorBuilder for BinaryVectorBuilder {
type VectorType = BinaryVector;
fn with_capacity(capacity: usize) -> Self {
Self {
mutable_array: MutableBinaryArray::with_capacity(capacity),
}
}
fn push(&mut self, value: Option<<Self::VectorType as ScalarVector>::RefItem<'_>>) {
self.mutable_array.push(value);
}
fn finish(&mut self) -> Self::VectorType {
BinaryVector {
array: std::mem::take(&mut self.mutable_array).into(),
}
}
}
impl Serializable for BinaryVector {
fn serialize_to_json(&self) -> Result<Vec<serde_json::Value>> {
self.iter_data()
.map(|v| match v {
None => Ok(serde_json::Value::Null), // if binary vector not present, map to NULL
Some(vec) => serde_json::to_value(vec),
})
.collect::<serde_json::Result<_>>()
.context(error::SerializeSnafu)
}
}
vectors::impl_try_from_arrow_array_for_vector!(BinaryArray, BinaryVector);
#[cfg(test)]
mod tests {
use arrow::datatypes::DataType as ArrowDataType;
use common_base::bytes::Bytes;
use serde_json;
use super::*;
use crate::arrow_array::BinaryArray;
use crate::data_type::DataType;
use crate::serialize::Serializable;
use crate::types::BinaryType;
#[test]
fn test_binary_vector_misc() {
let v = BinaryVector::from(BinaryArray::from_slice(&[vec![1, 2, 3], vec![1, 2, 3]]));
assert_eq!(2, v.len());
assert_eq!("BinaryVector", v.vector_type_name());
assert!(!v.is_const());
assert_eq!(Validity::AllValid, v.validity());
assert!(!v.only_null());
assert_eq!(30, v.memory_size());
for i in 0..2 {
assert!(!v.is_null(i));
assert_eq!(Value::Binary(Bytes::from(vec![1, 2, 3])), v.get(i));
assert_eq!(ValueRef::Binary(&[1, 2, 3]), v.get_ref(i));
}
let arrow_arr = v.to_arrow_array();
assert_eq!(2, arrow_arr.len());
assert_eq!(&ArrowDataType::LargeBinary, arrow_arr.data_type());
}
#[test]
fn test_serialize_binary_vector_to_json() {
let vector = BinaryVector::from(BinaryArray::from_slice(&[vec![1, 2, 3], vec![1, 2, 3]]));
let json_value = vector.serialize_to_json().unwrap();
assert_eq!(
"[[1,2,3],[1,2,3]]",
serde_json::to_string(&json_value).unwrap()
);
}
#[test]
fn test_serialize_binary_vector_with_null_to_json() {
let mut builder = BinaryVectorBuilder::with_capacity(4);
builder.push(Some(&[1, 2, 3]));
builder.push(None);
builder.push(Some(&[4, 5, 6]));
let vector = builder.finish();
let json_value = vector.serialize_to_json().unwrap();
assert_eq!(
"[[1,2,3],null,[4,5,6]]",
serde_json::to_string(&json_value).unwrap()
);
}
#[test]
fn test_from_arrow_array() {
let arrow_array = BinaryArray::from_slice(&[vec![1, 2, 3], vec![1, 2, 3]]);
let original = arrow_array.clone();
let vector = BinaryVector::from(arrow_array);
assert_eq!(original, vector.array);
}
#[test]
fn test_binary_vector_build_get() {
let mut builder = BinaryVectorBuilder::with_capacity(4);
builder.push(Some(b"hello"));
builder.push(Some(b"happy"));
builder.push(Some(b"world"));
builder.push(None);
let vector = builder.finish();
assert_eq!(b"hello", vector.get_data(0).unwrap());
assert_eq!(None, vector.get_data(3));
assert_eq!(Value::Binary(b"hello".as_slice().into()), vector.get(0));
assert_eq!(Value::Null, vector.get(3));
let mut iter = vector.iter_data();
assert_eq!(b"hello", iter.next().unwrap().unwrap());
assert_eq!(b"happy", iter.next().unwrap().unwrap());
assert_eq!(b"world", iter.next().unwrap().unwrap());
assert_eq!(None, iter.next().unwrap());
assert_eq!(None, iter.next());
}
#[test]
fn test_binary_vector_validity() {
let mut builder = BinaryVectorBuilder::with_capacity(4);
builder.push(Some(b"hello"));
builder.push(Some(b"world"));
let vector = builder.finish();
assert_eq!(0, vector.null_count());
assert_eq!(Validity::AllValid, vector.validity());
let mut builder = BinaryVectorBuilder::with_capacity(3);
builder.push(Some(b"hello"));
builder.push(None);
builder.push(Some(b"world"));
let vector = builder.finish();
assert_eq!(1, vector.null_count());
let validity = vector.validity();
let slots = validity.slots().unwrap();
assert_eq!(1, slots.null_count());
assert!(!slots.get_bit(1));
}
#[test]
fn test_binary_vector_builder() {
let input = BinaryVector::from_slice(&[b"world", b"one", b"two"]);
let mut builder = BinaryType::default().create_mutable_vector(3);
builder
.push_value_ref(ValueRef::Binary("hello".as_bytes()))
.unwrap();
assert!(builder.push_value_ref(ValueRef::Int32(123)).is_err());
builder.extend_slice_of(&input, 1, 2).unwrap();
assert!(builder
.extend_slice_of(&crate::vectors::Int32Vector::from_slice(&[13]), 0, 1)
.is_err());
let vector = builder.to_vector();
let expect: VectorRef = Arc::new(BinaryVector::from_slice(&[b"hello", b"one", b"two"]));
assert_eq!(expect, vector);
}
}

View File

@@ -1,344 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use std::borrow::Borrow;
use std::sync::Arc;
use arrow::array::{Array, ArrayRef, BooleanArray, MutableArray, MutableBooleanArray};
use arrow::bitmap::utils::{BitmapIter, ZipValidity};
use snafu::{OptionExt, ResultExt};
use crate::data_type::ConcreteDataType;
use crate::error::Result;
use crate::scalars::{ScalarVector, ScalarVectorBuilder};
use crate::serialize::Serializable;
use crate::value::{Value, ValueRef};
use crate::vectors::{self, MutableVector, Validity, Vector, VectorRef};
/// Vector of boolean.
#[derive(Debug, PartialEq)]
pub struct BooleanVector {
array: BooleanArray,
}
impl BooleanVector {
pub(crate) fn as_arrow(&self) -> &dyn Array {
&self.array
}
pub(crate) fn as_boolean_array(&self) -> &BooleanArray {
&self.array
}
}
impl From<Vec<bool>> for BooleanVector {
fn from(data: Vec<bool>) -> Self {
BooleanVector {
array: BooleanArray::from_slice(&data),
}
}
}
impl From<BooleanArray> for BooleanVector {
fn from(array: BooleanArray) -> Self {
Self { array }
}
}
impl From<Vec<Option<bool>>> for BooleanVector {
fn from(data: Vec<Option<bool>>) -> Self {
BooleanVector {
array: BooleanArray::from(data),
}
}
}
impl<Ptr: Borrow<Option<bool>>> FromIterator<Ptr> for BooleanVector {
fn from_iter<I: IntoIterator<Item = Ptr>>(iter: I) -> Self {
BooleanVector {
array: BooleanArray::from_iter(iter),
}
}
}
impl Vector for BooleanVector {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::boolean_datatype()
}
fn vector_type_name(&self) -> String {
"BooleanVector".to_string()
}
fn as_any(&self) -> &dyn Any {
self
}
fn len(&self) -> usize {
self.array.len()
}
fn to_arrow_array(&self) -> ArrayRef {
Arc::new(self.array.clone())
}
fn to_boxed_arrow_array(&self) -> Box<dyn Array> {
Box::new(self.array.clone())
}
fn validity(&self) -> Validity {
vectors::impl_validity_for_vector!(self.array)
}
fn memory_size(&self) -> usize {
self.array.values().as_slice().0.len()
}
fn is_null(&self, row: usize) -> bool {
self.array.is_null(row)
}
fn slice(&self, offset: usize, length: usize) -> VectorRef {
Arc::new(Self::from(self.array.slice(offset, length)))
}
fn get(&self, index: usize) -> Value {
vectors::impl_get_for_vector!(self.array, index)
}
fn get_ref(&self, index: usize) -> ValueRef {
vectors::impl_get_ref_for_vector!(self.array, index)
}
}
impl ScalarVector for BooleanVector {
type OwnedItem = bool;
type RefItem<'a> = bool;
type Iter<'a> = ZipValidity<'a, bool, BitmapIter<'a>>;
type Builder = BooleanVectorBuilder;
fn get_data(&self, idx: usize) -> Option<Self::RefItem<'_>> {
if self.array.is_valid(idx) {
Some(self.array.value(idx))
} else {
None
}
}
fn iter_data(&self) -> Self::Iter<'_> {
self.array.iter()
}
}
pub struct BooleanVectorBuilder {
mutable_array: MutableBooleanArray,
}
impl MutableVector for BooleanVectorBuilder {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::boolean_datatype()
}
fn len(&self) -> usize {
self.mutable_array.len()
}
fn as_any(&self) -> &dyn Any {
self
}
fn as_mut_any(&mut self) -> &mut dyn Any {
self
}
fn to_vector(&mut self) -> VectorRef {
Arc::new(self.finish())
}
fn push_value_ref(&mut self, value: ValueRef) -> Result<()> {
self.mutable_array.push(value.as_boolean()?);
Ok(())
}
fn extend_slice_of(&mut self, vector: &dyn Vector, offset: usize, length: usize) -> Result<()> {
vectors::impl_extend_for_builder!(self.mutable_array, vector, BooleanVector, offset, length)
}
}
impl ScalarVectorBuilder for BooleanVectorBuilder {
type VectorType = BooleanVector;
fn with_capacity(capacity: usize) -> Self {
Self {
mutable_array: MutableBooleanArray::with_capacity(capacity),
}
}
fn push(&mut self, value: Option<<Self::VectorType as ScalarVector>::RefItem<'_>>) {
self.mutable_array.push(value);
}
fn finish(&mut self) -> Self::VectorType {
BooleanVector {
array: std::mem::take(&mut self.mutable_array).into(),
}
}
}
impl Serializable for BooleanVector {
fn serialize_to_json(&self) -> Result<Vec<serde_json::Value>> {
self.iter_data()
.map(serde_json::to_value)
.collect::<serde_json::Result<_>>()
.context(crate::error::SerializeSnafu)
}
}
vectors::impl_try_from_arrow_array_for_vector!(BooleanArray, BooleanVector);
#[cfg(test)]
mod tests {
use arrow::datatypes::DataType as ArrowDataType;
use serde_json;
use super::*;
use crate::data_type::DataType;
use crate::serialize::Serializable;
use crate::types::BooleanType;
#[test]
fn test_boolean_vector_misc() {
let bools = vec![true, false, true, true, false, false, true, true, false];
let v = BooleanVector::from(bools.clone());
assert_eq!(9, v.len());
assert_eq!("BooleanVector", v.vector_type_name());
assert!(!v.is_const());
assert_eq!(Validity::AllValid, v.validity());
assert!(!v.only_null());
assert_eq!(2, v.memory_size());
for (i, b) in bools.iter().enumerate() {
assert!(!v.is_null(i));
assert_eq!(Value::Boolean(*b), v.get(i));
assert_eq!(ValueRef::Boolean(*b), v.get_ref(i));
}
let arrow_arr = v.to_arrow_array();
assert_eq!(9, arrow_arr.len());
assert_eq!(&ArrowDataType::Boolean, arrow_arr.data_type());
}
#[test]
fn test_serialize_boolean_vector_to_json() {
let vector = BooleanVector::from(vec![true, false, true, true, false, false]);
let json_value = vector.serialize_to_json().unwrap();
assert_eq!(
"[true,false,true,true,false,false]",
serde_json::to_string(&json_value).unwrap(),
);
}
#[test]
fn test_serialize_boolean_vector_with_null_to_json() {
let vector = BooleanVector::from(vec![Some(true), None, Some(false)]);
let json_value = vector.serialize_to_json().unwrap();
assert_eq!(
"[true,null,false]",
serde_json::to_string(&json_value).unwrap(),
);
}
#[test]
fn test_boolean_vector_from_vec() {
let input = vec![false, true, false, true];
let vec = BooleanVector::from(input.clone());
assert_eq!(4, vec.len());
for (i, v) in input.into_iter().enumerate() {
assert_eq!(Some(v), vec.get_data(i), "failed at {}", i)
}
}
#[test]
fn test_boolean_vector_from_iter() {
let input = vec![Some(false), Some(true), Some(false), Some(true)];
let vec = input.iter().collect::<BooleanVector>();
assert_eq!(4, vec.len());
for (i, v) in input.into_iter().enumerate() {
assert_eq!(v, vec.get_data(i), "failed at {}", i)
}
}
#[test]
fn test_boolean_vector_from_vec_option() {
let input = vec![Some(false), Some(true), None, Some(true)];
let vec = BooleanVector::from(input.clone());
assert_eq!(4, vec.len());
for (i, v) in input.into_iter().enumerate() {
assert_eq!(v, vec.get_data(i), "failed at {}", i)
}
}
#[test]
fn test_boolean_vector_build_get() {
let input = [Some(true), None, Some(false)];
let mut builder = BooleanVectorBuilder::with_capacity(3);
for v in input {
builder.push(v);
}
let vector = builder.finish();
assert_eq!(input.len(), vector.len());
let res: Vec<_> = vector.iter_data().collect();
assert_eq!(input, &res[..]);
for (i, v) in input.into_iter().enumerate() {
assert_eq!(v, vector.get_data(i));
assert_eq!(Value::from(v), vector.get(i));
}
}
#[test]
fn test_boolean_vector_validity() {
let vector = BooleanVector::from(vec![Some(true), None, Some(false)]);
assert_eq!(1, vector.null_count());
let validity = vector.validity();
let slots = validity.slots().unwrap();
assert_eq!(1, slots.null_count());
assert!(!slots.get_bit(1));
let vector = BooleanVector::from(vec![true, false, false]);
assert_eq!(0, vector.null_count());
assert_eq!(Validity::AllValid, vector.validity());
}
#[test]
fn test_boolean_vector_builder() {
let input = BooleanVector::from_slice(&[true, false, true]);
let mut builder = BooleanType::default().create_mutable_vector(3);
builder.push_value_ref(ValueRef::Boolean(true)).unwrap();
assert!(builder.push_value_ref(ValueRef::Int32(123)).is_err());
builder.extend_slice_of(&input, 1, 2).unwrap();
assert!(builder
.extend_slice_of(&crate::vectors::Int32Vector::from_slice(&[13]), 0, 1)
.is_err());
let vector = builder.to_vector();
let expect: VectorRef = Arc::new(BooleanVector::from_slice(&[true, false, true]));
assert_eq!(expect, vector);
}
}

View File

@@ -1,494 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::sync::Arc;
use common_time::date::Date;
use common_time::datetime::DateTime;
use common_time::timestamp::Timestamp;
use crate::data_type::ConcreteDataType;
use crate::error::{self, Result};
use crate::prelude::ValueRef;
use crate::scalars::ScalarVectorBuilder;
use crate::value::Value;
use crate::vectors::date::DateVectorBuilder;
use crate::vectors::datetime::DateTimeVectorBuilder;
use crate::vectors::{
BinaryVectorBuilder, BooleanVectorBuilder, Float32VectorBuilder, Float64VectorBuilder,
Int16VectorBuilder, Int32VectorBuilder, Int64VectorBuilder, Int8VectorBuilder, MutableVector,
NullVector, StringVectorBuilder, TimestampVectorBuilder, UInt16VectorBuilder,
UInt32VectorBuilder, UInt64VectorBuilder, UInt8VectorBuilder, VectorRef,
};
pub enum VectorBuilder {
Null(usize),
// Numeric types:
Boolean(BooleanVectorBuilder),
UInt8(UInt8VectorBuilder),
UInt16(UInt16VectorBuilder),
UInt32(UInt32VectorBuilder),
UInt64(UInt64VectorBuilder),
Int8(Int8VectorBuilder),
Int16(Int16VectorBuilder),
Int32(Int32VectorBuilder),
Int64(Int64VectorBuilder),
Float32(Float32VectorBuilder),
Float64(Float64VectorBuilder),
// String types:
String(StringVectorBuilder),
Binary(BinaryVectorBuilder),
Date(DateVectorBuilder),
DateTime(DateTimeVectorBuilder),
Timestamp(TimestampVectorBuilder),
}
impl VectorBuilder {
pub fn new(data_type: ConcreteDataType) -> VectorBuilder {
VectorBuilder::with_capacity(data_type, 0)
}
pub fn with_capacity(data_type: ConcreteDataType, capacity: usize) -> VectorBuilder {
match data_type {
ConcreteDataType::Null(_) => VectorBuilder::Null(0),
ConcreteDataType::Boolean(_) => {
VectorBuilder::Boolean(BooleanVectorBuilder::with_capacity(capacity))
}
ConcreteDataType::UInt8(_) => {
VectorBuilder::UInt8(UInt8VectorBuilder::with_capacity(capacity))
}
ConcreteDataType::UInt16(_) => {
VectorBuilder::UInt16(UInt16VectorBuilder::with_capacity(capacity))
}
ConcreteDataType::UInt32(_) => {
VectorBuilder::UInt32(UInt32VectorBuilder::with_capacity(capacity))
}
ConcreteDataType::UInt64(_) => {
VectorBuilder::UInt64(UInt64VectorBuilder::with_capacity(capacity))
}
ConcreteDataType::Int8(_) => {
VectorBuilder::Int8(Int8VectorBuilder::with_capacity(capacity))
}
ConcreteDataType::Int16(_) => {
VectorBuilder::Int16(Int16VectorBuilder::with_capacity(capacity))
}
ConcreteDataType::Int32(_) => {
VectorBuilder::Int32(Int32VectorBuilder::with_capacity(capacity))
}
ConcreteDataType::Int64(_) => {
VectorBuilder::Int64(Int64VectorBuilder::with_capacity(capacity))
}
ConcreteDataType::Float32(_) => {
VectorBuilder::Float32(Float32VectorBuilder::with_capacity(capacity))
}
ConcreteDataType::Float64(_) => {
VectorBuilder::Float64(Float64VectorBuilder::with_capacity(capacity))
}
ConcreteDataType::String(_) => {
VectorBuilder::String(StringVectorBuilder::with_capacity(capacity))
}
ConcreteDataType::Binary(_) => {
VectorBuilder::Binary(BinaryVectorBuilder::with_capacity(capacity))
}
ConcreteDataType::Date(_) => {
VectorBuilder::Date(DateVectorBuilder::with_capacity(capacity))
}
ConcreteDataType::DateTime(_) => {
VectorBuilder::DateTime(DateTimeVectorBuilder::with_capacity(capacity))
}
ConcreteDataType::Timestamp(_) => {
VectorBuilder::Timestamp(TimestampVectorBuilder::with_capacity(capacity))
}
_ => unimplemented!(),
}
}
pub fn data_type(&self) -> ConcreteDataType {
match self {
VectorBuilder::Null(_) => ConcreteDataType::null_datatype(),
VectorBuilder::Boolean(b) => b.data_type(),
VectorBuilder::UInt8(b) => b.data_type(),
VectorBuilder::UInt16(b) => b.data_type(),
VectorBuilder::UInt32(b) => b.data_type(),
VectorBuilder::UInt64(b) => b.data_type(),
VectorBuilder::Int8(b) => b.data_type(),
VectorBuilder::Int16(b) => b.data_type(),
VectorBuilder::Int32(b) => b.data_type(),
VectorBuilder::Int64(b) => b.data_type(),
VectorBuilder::Float32(b) => b.data_type(),
VectorBuilder::Float64(b) => b.data_type(),
VectorBuilder::String(b) => b.data_type(),
VectorBuilder::Binary(b) => b.data_type(),
VectorBuilder::Date(b) => b.data_type(),
VectorBuilder::DateTime(b) => b.data_type(),
VectorBuilder::Timestamp(b) => b.data_type(),
}
}
pub fn push(&mut self, value: &Value) {
if value.is_null() {
self.push_null();
return;
}
match (&mut *self, value) {
(VectorBuilder::Boolean(b), Value::Boolean(v)) => b.push(Some(*v)),
(VectorBuilder::UInt8(b), Value::UInt8(v)) => b.push(Some(*v)),
(VectorBuilder::UInt16(b), Value::UInt16(v)) => b.push(Some(*v)),
(VectorBuilder::UInt32(b), Value::UInt32(v)) => b.push(Some(*v)),
(VectorBuilder::UInt64(b), Value::UInt64(v)) => b.push(Some(*v)),
(VectorBuilder::Int8(b), Value::Int8(v)) => b.push(Some(*v)),
(VectorBuilder::Int16(b), Value::Int16(v)) => b.push(Some(*v)),
(VectorBuilder::Int32(b), Value::Int32(v)) => b.push(Some(*v)),
(VectorBuilder::Int64(b), Value::Int64(v)) => b.push(Some(*v)),
(VectorBuilder::Float32(b), Value::Float32(v)) => b.push(Some(v.into_inner())),
(VectorBuilder::Float64(b), Value::Float64(v)) => b.push(Some(v.into_inner())),
(VectorBuilder::String(b), Value::String(v)) => b.push(Some(v.as_utf8())),
(VectorBuilder::Binary(b), Value::Binary(v)) => b.push(Some(v)),
(VectorBuilder::Date(b), Value::Date(v)) => b.push(Some(*v)),
(VectorBuilder::Date(b), Value::Int32(v)) => b.push(Some(Date::new(*v))),
(VectorBuilder::DateTime(b), Value::DateTime(v)) => b.push(Some(*v)),
(VectorBuilder::DateTime(b), Value::Int64(v)) => b.push(Some(DateTime::new(*v))),
(VectorBuilder::Timestamp(b), Value::Timestamp(t)) => b.push(Some(*t)),
(VectorBuilder::Timestamp(b), Value::Int64(v)) => {
b.push(Some(Timestamp::from_millis(*v)))
}
_ => panic!(
"Value {:?} does not match builder type {:?}",
value,
self.data_type()
),
}
}
pub fn try_push_ref(&mut self, value: ValueRef) -> Result<()> {
match &mut *self {
VectorBuilder::Null(b) => {
if !value.is_null() {
return error::CastTypeSnafu {
msg: "unable to accept non-null value in NullVectorBuilder",
}
.fail();
}
*b += 1;
Ok(())
}
VectorBuilder::Boolean(b) => b.push_value_ref(value),
VectorBuilder::UInt8(b) => b.push_value_ref(value),
VectorBuilder::UInt16(b) => b.push_value_ref(value),
VectorBuilder::UInt32(b) => b.push_value_ref(value),
VectorBuilder::UInt64(b) => b.push_value_ref(value),
VectorBuilder::Int8(b) => b.push_value_ref(value),
VectorBuilder::Int16(b) => b.push_value_ref(value),
VectorBuilder::Int32(b) => b.push_value_ref(value),
VectorBuilder::Int64(b) => b.push_value_ref(value),
VectorBuilder::Float32(b) => b.push_value_ref(value),
VectorBuilder::Float64(b) => b.push_value_ref(value),
VectorBuilder::String(b) => b.push_value_ref(value),
VectorBuilder::Binary(b) => b.push_value_ref(value),
VectorBuilder::Date(b) => b.push_value_ref(value),
VectorBuilder::DateTime(b) => b.push_value_ref(value),
VectorBuilder::Timestamp(b) => b.push_value_ref(value),
}
}
pub fn push_null(&mut self) {
match self {
VectorBuilder::Null(v) => *v += 1,
VectorBuilder::Boolean(b) => b.push(None),
VectorBuilder::UInt8(b) => b.push(None),
VectorBuilder::UInt16(b) => b.push(None),
VectorBuilder::UInt32(b) => b.push(None),
VectorBuilder::UInt64(b) => b.push(None),
VectorBuilder::Int8(b) => b.push(None),
VectorBuilder::Int16(b) => b.push(None),
VectorBuilder::Int32(b) => b.push(None),
VectorBuilder::Int64(b) => b.push(None),
VectorBuilder::Float32(b) => b.push(None),
VectorBuilder::Float64(b) => b.push(None),
VectorBuilder::String(b) => b.push(None),
VectorBuilder::Binary(b) => b.push(None),
VectorBuilder::Date(b) => b.push(None),
VectorBuilder::DateTime(b) => b.push(None),
VectorBuilder::Timestamp(b) => b.push(None),
}
}
pub fn finish(&mut self) -> VectorRef {
match self {
VectorBuilder::Null(v) => Arc::new(NullVector::new(*v)),
VectorBuilder::Boolean(b) => Arc::new(b.finish()),
VectorBuilder::UInt8(b) => Arc::new(b.finish()),
VectorBuilder::UInt16(b) => Arc::new(b.finish()),
VectorBuilder::UInt32(b) => Arc::new(b.finish()),
VectorBuilder::UInt64(b) => Arc::new(b.finish()),
VectorBuilder::Int8(b) => Arc::new(b.finish()),
VectorBuilder::Int16(b) => Arc::new(b.finish()),
VectorBuilder::Int32(b) => Arc::new(b.finish()),
VectorBuilder::Int64(b) => Arc::new(b.finish()),
VectorBuilder::Float32(b) => Arc::new(b.finish()),
VectorBuilder::Float64(b) => Arc::new(b.finish()),
VectorBuilder::String(b) => Arc::new(b.finish()),
VectorBuilder::Binary(b) => Arc::new(b.finish()),
VectorBuilder::Date(b) => Arc::new(b.finish()),
VectorBuilder::DateTime(b) => Arc::new(b.finish()),
VectorBuilder::Timestamp(b) => Arc::new(b.finish()),
}
}
}
#[cfg(test)]
mod tests {
use ordered_float::OrderedFloat;
use super::*;
use crate::prelude::Vector;
use crate::vectors::date::DateVector;
use crate::vectors::datetime::DateTimeVector;
macro_rules! impl_integer_builder_test {
($Type: ident, $datatype: ident) => {
let data_type = ConcreteDataType::$datatype();
let mut builder = VectorBuilder::with_capacity(data_type.clone(), 10);
assert_eq!(data_type, builder.data_type());
for i in 0..10 {
builder.push(&Value::$Type(i));
}
for i in 10..20 {
builder.try_push_ref(ValueRef::$Type(i)).unwrap();
}
let vector = builder.finish();
for i in 0..20 {
assert_eq!(Value::$Type(i), vector.get(i as usize));
}
let mut builder = VectorBuilder::new(ConcreteDataType::$datatype());
builder.push(&Value::Null);
builder.push(&Value::$Type(100));
builder.try_push_ref(ValueRef::Null).unwrap();
builder.try_push_ref(ValueRef::$Type(101)).unwrap();
let result = builder.try_push_ref(ValueRef::Boolean(true));
assert!(result.is_err());
assert_eq!(
result.unwrap_err().to_string(),
format!(
"Failed to cast value Boolean(true) to primitive type {}",
stringify!($Type)
),
);
let vector = builder.finish();
assert!(vector.is_null(0));
assert_eq!(Value::$Type(100), vector.get(1));
assert!(vector.is_null(2));
assert_eq!(Value::$Type(101), vector.get(3));
};
}
#[test]
fn test_null_vector_builder() {
let mut builder = VectorBuilder::new(ConcreteDataType::null_datatype());
assert_eq!(ConcreteDataType::null_datatype(), builder.data_type());
builder.push(&Value::Null);
let result = builder.try_push_ref(ValueRef::Boolean(true));
assert!(result.is_err());
assert_eq!(
result.unwrap_err().to_string(),
"unable to accept non-null value in NullVectorBuilder"
);
builder.try_push_ref(ValueRef::Null).unwrap();
let vector = builder.finish();
assert!(vector.is_null(0));
assert!(vector.is_null(1));
}
#[test]
fn test_integer_vector_builder() {
impl_integer_builder_test!(UInt8, uint8_datatype);
impl_integer_builder_test!(UInt16, uint16_datatype);
impl_integer_builder_test!(UInt32, uint32_datatype);
impl_integer_builder_test!(UInt64, uint64_datatype);
impl_integer_builder_test!(Int8, int8_datatype);
impl_integer_builder_test!(Int16, int16_datatype);
impl_integer_builder_test!(Int32, int32_datatype);
impl_integer_builder_test!(Int64, int64_datatype);
}
#[test]
fn test_float_vector_builder() {
let data_type = ConcreteDataType::float32_datatype();
let mut builder = VectorBuilder::new(data_type.clone());
assert_eq!(data_type, builder.data_type());
builder.push(&Value::Float32(OrderedFloat(1.0)));
let result = builder.try_push_ref(ValueRef::Boolean(true));
assert!(result.is_err());
assert_eq!(
result.unwrap_err().to_string(),
"Failed to cast value Boolean(true) to primitive type Float32"
);
builder
.try_push_ref(ValueRef::Float32(OrderedFloat(2.0)))
.unwrap();
builder.try_push_ref(ValueRef::Null).unwrap();
let vector = builder.finish();
assert_eq!(Value::Float32(OrderedFloat(1.0)), vector.get(0));
assert_eq!(Value::Float32(OrderedFloat(2.0)), vector.get(1));
assert_eq!(Value::Null, vector.get(2));
let mut builder = VectorBuilder::new(ConcreteDataType::float64_datatype());
builder.push(&Value::Float64(OrderedFloat(2.0)));
let result = builder.try_push_ref(ValueRef::Boolean(true));
assert!(result.is_err());
assert_eq!(
result.unwrap_err().to_string(),
"Failed to cast value Boolean(true) to primitive type Float64"
);
builder
.try_push_ref(ValueRef::Float64(OrderedFloat(3.0)))
.unwrap();
builder.try_push_ref(ValueRef::Null).unwrap();
let vector = builder.finish();
assert_eq!(Value::Float64(OrderedFloat(2.0)), vector.get(0));
assert_eq!(Value::Float64(OrderedFloat(3.0)), vector.get(1));
assert_eq!(Value::Null, vector.get(2));
}
#[test]
fn test_binary_vector_builder() {
let data_type = ConcreteDataType::binary_datatype();
let hello: &[u8] = b"hello";
let mut builder = VectorBuilder::new(data_type.clone());
assert_eq!(data_type, builder.data_type());
builder.push(&Value::Binary(hello.into()));
let result = builder.try_push_ref(ValueRef::Boolean(true));
assert!(result.is_err());
assert_eq!(
result.unwrap_err().to_string(),
"Failed to cast value ref Boolean(true) to Binary"
);
builder.try_push_ref(ValueRef::Binary(b"world")).unwrap();
builder.try_push_ref(ValueRef::Null).unwrap();
let vector = builder.finish();
assert_eq!(Value::Binary(hello.into()), vector.get(0));
assert_eq!(ValueRef::Binary(b"world"), vector.get_ref(1));
assert_eq!(Value::Null, vector.get(2));
}
#[test]
fn test_string_vector_builder() {
let data_type = ConcreteDataType::string_datatype();
let hello = "hello";
let mut builder = VectorBuilder::new(data_type.clone());
assert_eq!(data_type, builder.data_type());
builder.push(&Value::String(hello.into()));
let result = builder.try_push_ref(ValueRef::Boolean(true));
assert!(result.is_err());
assert_eq!(
result.unwrap_err().to_string(),
"Failed to cast value ref Boolean(true) to String"
);
builder.try_push_ref(ValueRef::String("world")).unwrap();
builder.try_push_ref(ValueRef::Null).unwrap();
let vector = builder.finish();
assert_eq!(Value::String(hello.into()), vector.get(0));
assert_eq!(ValueRef::String("world"), vector.get_ref(1));
assert_eq!(Value::Null, vector.get(2));
}
#[test]
pub fn test_date_vector_builder() {
let mut builder = VectorBuilder::with_capacity(ConcreteDataType::date_datatype(), 3);
assert_eq!(ConcreteDataType::date_datatype(), builder.data_type());
builder.push_null();
builder.push(&Value::Date(Date::new(123)));
let result = builder.try_push_ref(ValueRef::Boolean(true));
assert!(result.is_err());
assert_eq!(
result.unwrap_err().to_string(),
"Failed to cast value ref Boolean(true) to Date"
);
builder
.try_push_ref(ValueRef::Date(Date::new(456)))
.unwrap();
builder.try_push_ref(ValueRef::Null).unwrap();
let v = builder.finish();
let v = v.as_any().downcast_ref::<DateVector>().unwrap();
assert_eq!(Value::Null, v.get(0));
assert_eq!(Value::Date(Date::new(123)), v.get(1));
assert_eq!(ValueRef::Date(Date::new(456)), v.get_ref(2));
assert_eq!(ValueRef::Null, v.get_ref(3));
assert_eq!(
&arrow::datatypes::DataType::Date32,
v.to_arrow_array().data_type()
);
}
#[test]
pub fn test_datetime_vector_builder() {
let mut builder = VectorBuilder::with_capacity(ConcreteDataType::datetime_datatype(), 3);
assert_eq!(ConcreteDataType::datetime_datatype(), builder.data_type());
builder.push_null();
builder.push(&Value::DateTime(DateTime::new(123)));
let result = builder.try_push_ref(ValueRef::Boolean(true));
assert!(result.is_err());
assert_eq!(
result.unwrap_err().to_string(),
"Failed to cast value ref Boolean(true) to DateTime"
);
builder
.try_push_ref(ValueRef::DateTime(DateTime::new(456)))
.unwrap();
builder.try_push_ref(ValueRef::Null).unwrap();
let v = builder.finish();
let v = v.as_any().downcast_ref::<DateTimeVector>().unwrap();
assert_eq!(Value::Null, v.get(0));
assert_eq!(Value::DateTime(DateTime::new(123)), v.get(1));
assert_eq!(ValueRef::DateTime(DateTime::new(456)), v.get_ref(2));
assert_eq!(ValueRef::Null, v.get_ref(3));
assert_eq!(
&arrow::datatypes::DataType::Date64,
v.to_arrow_array().data_type()
);
}
}

View File

@@ -1,216 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use std::fmt;
use std::sync::Arc;
use arrow::array::{Array, ArrayRef};
use snafu::ResultExt;
use crate::data_type::ConcreteDataType;
use crate::error::{Result, SerializeSnafu};
use crate::serialize::Serializable;
use crate::value::{Value, ValueRef};
use crate::vectors::{BooleanVector, Helper, Validity, Vector, VectorRef};
#[derive(Clone)]
pub struct ConstantVector {
length: usize,
vector: VectorRef,
}
impl ConstantVector {
/// Create a new [ConstantVector].
///
/// # Panics
/// Panics if `vector.len() != 1`.
pub fn new(vector: VectorRef, length: usize) -> Self {
assert_eq!(1, vector.len());
// Avoid const recursion.
if vector.is_const() {
let vec: &ConstantVector = unsafe { Helper::static_cast(&vector) };
return Self::new(vec.inner().clone(), length);
}
Self { vector, length }
}
pub fn inner(&self) -> &VectorRef {
&self.vector
}
/// Returns the constant value.
pub fn get_constant_ref(&self) -> ValueRef {
self.vector.get_ref(0)
}
}
impl Vector for ConstantVector {
fn data_type(&self) -> ConcreteDataType {
self.vector.data_type()
}
fn vector_type_name(&self) -> String {
"ConstantVector".to_string()
}
fn as_any(&self) -> &dyn Any {
self
}
fn len(&self) -> usize {
self.length
}
fn to_arrow_array(&self) -> ArrayRef {
let v = self.vector.replicate(&[self.length]);
v.to_arrow_array()
}
fn to_boxed_arrow_array(&self) -> Box<dyn Array> {
let v = self.vector.replicate(&[self.length]);
v.to_boxed_arrow_array()
}
fn is_const(&self) -> bool {
true
}
fn validity(&self) -> Validity {
if self.vector.is_null(0) {
Validity::AllNull
} else {
Validity::AllValid
}
}
fn memory_size(&self) -> usize {
self.vector.memory_size()
}
fn is_null(&self, _row: usize) -> bool {
self.vector.is_null(0)
}
fn only_null(&self) -> bool {
self.vector.is_null(0)
}
fn slice(&self, _offset: usize, length: usize) -> VectorRef {
Arc::new(Self {
vector: self.vector.clone(),
length,
})
}
fn get(&self, _index: usize) -> Value {
self.vector.get(0)
}
fn get_ref(&self, _index: usize) -> ValueRef {
self.vector.get_ref(0)
}
}
impl fmt::Debug for ConstantVector {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "ConstantVector([{:?}; {}])", self.get(0), self.len())
}
}
impl Serializable for ConstantVector {
fn serialize_to_json(&self) -> Result<Vec<serde_json::Value>> {
std::iter::repeat(self.get(0))
.take(self.len())
.map(serde_json::Value::try_from)
.collect::<serde_json::Result<_>>()
.context(SerializeSnafu)
}
}
pub(crate) fn replicate_constant(vector: &ConstantVector, offsets: &[usize]) -> VectorRef {
assert_eq!(offsets.len(), vector.len());
if offsets.is_empty() {
return vector.slice(0, 0);
}
Arc::new(ConstantVector::new(
vector.vector.clone(),
*offsets.last().unwrap(),
))
}
pub(crate) fn filter_constant(
vector: &ConstantVector,
filter: &BooleanVector,
) -> Result<VectorRef> {
let length = filter.len() - filter.as_boolean_array().values().null_count();
if length == vector.len() {
return Ok(Arc::new(vector.clone()));
}
Ok(Arc::new(ConstantVector::new(
vector.inner().clone(),
length,
)))
}
#[cfg(test)]
mod tests {
use arrow::datatypes::DataType as ArrowDataType;
use super::*;
use crate::vectors::Int32Vector;
#[test]
fn test_constant_vector_misc() {
let a = Int32Vector::from_slice(vec![1]);
let c = ConstantVector::new(Arc::new(a), 10);
assert_eq!("ConstantVector", c.vector_type_name());
assert!(c.is_const());
assert_eq!(10, c.len());
assert_eq!(Validity::AllValid, c.validity());
assert!(!c.only_null());
assert_eq!(4, c.memory_size());
for i in 0..10 {
assert!(!c.is_null(i));
assert_eq!(Value::Int32(1), c.get(i));
}
let arrow_arr = c.to_arrow_array();
assert_eq!(10, arrow_arr.len());
assert_eq!(&ArrowDataType::Int32, arrow_arr.data_type());
}
#[test]
fn test_debug_null_array() {
let a = Int32Vector::from_slice(vec![1]);
let c = ConstantVector::new(Arc::new(a), 10);
let s = format!("{:?}", c);
assert_eq!(s, "ConstantVector([Int32(1); 10])");
}
#[test]
fn test_serialize_json() {
let a = Int32Vector::from_slice(vec![1]);
let c = ConstantVector::new(Arc::new(a), 10);
let s = serde_json::to_string(&c.serialize_to_json().unwrap()).unwrap();
assert_eq!(s, "[1,1,1,1,1,1,1,1,1,1]");
}
}

View File

@@ -1,327 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use std::sync::Arc;
use arrow::array::{Array, ArrayRef, PrimitiveArray};
use common_time::date::Date;
use snafu::OptionExt;
use crate::data_type::ConcreteDataType;
use crate::error::{self, Result};
use crate::prelude::*;
use crate::scalars::ScalarVector;
use crate::serialize::Serializable;
use crate::vectors::{MutableVector, PrimitiveIter, PrimitiveVector, PrimitiveVectorBuilder};
#[derive(Debug, Clone, PartialEq)]
pub struct DateVector {
array: PrimitiveVector<i32>,
}
impl DateVector {
pub fn new(array: PrimitiveArray<i32>) -> Self {
Self {
array: PrimitiveVector { array },
}
}
pub fn try_from_arrow_array(array: impl AsRef<dyn Array>) -> Result<Self> {
Ok(Self::new(
array
.as_ref()
.as_any()
.downcast_ref::<PrimitiveArray<i32>>()
.with_context(|| error::ConversionSnafu {
from: format!("{:?}", array.as_ref().data_type()),
})?
.clone(),
))
}
pub(crate) fn as_arrow(&self) -> &dyn Array {
self.array.as_arrow()
}
}
impl Vector for DateVector {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::date_datatype()
}
fn vector_type_name(&self) -> String {
"DateVector".to_string()
}
fn as_any(&self) -> &dyn Any {
self
}
fn len(&self) -> usize {
self.array.len()
}
fn to_arrow_array(&self) -> ArrayRef {
let validity = self.array.array.validity().cloned();
let buffer = self.array.array.values().clone();
Arc::new(PrimitiveArray::new(
arrow::datatypes::DataType::Date32,
buffer,
validity,
))
}
fn to_boxed_arrow_array(&self) -> Box<dyn Array> {
let validity = self.array.array.validity().cloned();
let buffer = self.array.array.values().clone();
Box::new(PrimitiveArray::new(
arrow::datatypes::DataType::Date32,
buffer,
validity,
))
}
fn validity(&self) -> Validity {
self.array.validity()
}
fn memory_size(&self) -> usize {
self.array.memory_size()
}
fn is_null(&self, row: usize) -> bool {
self.array.is_null(row)
}
fn slice(&self, offset: usize, length: usize) -> VectorRef {
Arc::new(Self {
array: PrimitiveVector::new(self.array.array.slice(offset, length)),
})
}
fn get(&self, index: usize) -> Value {
match self.array.get(index) {
Value::Int32(v) => Value::Date(Date::new(v)),
Value::Null => Value::Null,
_ => {
unreachable!()
}
}
}
fn get_ref(&self, index: usize) -> ValueRef {
match self.array.get(index) {
Value::Int32(v) => ValueRef::Date(Date::new(v)),
Value::Null => ValueRef::Null,
_ => {
unreachable!()
}
}
}
}
impl From<Vec<Option<i32>>> for DateVector {
fn from(data: Vec<Option<i32>>) -> Self {
Self {
array: PrimitiveVector::<i32>::from(data),
}
}
}
pub struct DateIter<'a> {
iter: PrimitiveIter<'a, i32>,
}
impl<'a> Iterator for DateIter<'a> {
type Item = Option<Date>;
fn next(&mut self) -> Option<Self::Item> {
self.iter.next().map(|v| v.map(Date::new))
}
}
impl ScalarVector for DateVector {
type OwnedItem = Date;
type RefItem<'a> = Date;
type Iter<'a> = DateIter<'a>;
type Builder = DateVectorBuilder;
fn get_data(&self, idx: usize) -> Option<Self::RefItem<'_>> {
self.array.get_data(idx).map(Date::new)
}
fn iter_data(&self) -> Self::Iter<'_> {
DateIter {
iter: self.array.iter_data(),
}
}
}
impl Serializable for DateVector {
fn serialize_to_json(&self) -> Result<Vec<serde_json::Value>> {
Ok(self
.array
.iter_data()
.map(|v| v.map(Date::new))
.map(|v| match v {
None => serde_json::Value::Null,
Some(v) => v.into(),
})
.collect::<Vec<_>>())
}
}
pub struct DateVectorBuilder {
buffer: PrimitiveVectorBuilder<i32>,
}
impl MutableVector for DateVectorBuilder {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::date_datatype()
}
fn len(&self) -> usize {
self.buffer.len()
}
fn as_any(&self) -> &dyn Any {
self
}
fn as_mut_any(&mut self) -> &mut dyn Any {
self
}
fn to_vector(&mut self) -> VectorRef {
Arc::new(self.finish())
}
fn push_value_ref(&mut self, value: ValueRef) -> Result<()> {
self.buffer.push(value.as_date()?.map(|d| d.val()));
Ok(())
}
fn extend_slice_of(&mut self, vector: &dyn Vector, offset: usize, length: usize) -> Result<()> {
let concrete_vector = vector
.as_any()
.downcast_ref::<DateVector>()
.with_context(|| error::CastTypeSnafu {
msg: format!(
"Failed to convert vector from {} to DateVector",
vector.vector_type_name()
),
})?;
self.buffer
.extend_slice_of(&concrete_vector.array, offset, length)?;
Ok(())
}
}
impl ScalarVectorBuilder for DateVectorBuilder {
type VectorType = DateVector;
fn with_capacity(capacity: usize) -> Self {
Self {
buffer: PrimitiveVectorBuilder::with_capacity(capacity),
}
}
fn push(&mut self, value: Option<<Self::VectorType as ScalarVector>::RefItem<'_>>) {
self.buffer.push(value.map(|d| d.val()))
}
fn finish(&mut self) -> Self::VectorType {
Self::VectorType {
array: self.buffer.finish(),
}
}
}
pub(crate) fn replicate_date(vector: &DateVector, offsets: &[usize]) -> VectorRef {
let array = crate::vectors::primitive::replicate_primitive_with_type(
&vector.array,
offsets,
vector.data_type(),
);
Arc::new(DateVector { array })
}
#[cfg(test)]
mod tests {
use super::*;
use crate::data_type::DataType;
use crate::types::DateType;
#[test]
fn test_build_date_vector() {
let mut builder = DateVectorBuilder::with_capacity(4);
builder.push(Some(Date::new(1)));
builder.push(None);
builder.push(Some(Date::new(-1)));
let vector = builder.finish();
assert_eq!(3, vector.len());
assert_eq!(Value::Date(Date::new(1)), vector.get(0));
assert_eq!(ValueRef::Date(Date::new(1)), vector.get_ref(0));
assert_eq!(Some(Date::new(1)), vector.get_data(0));
assert_eq!(None, vector.get_data(1));
assert_eq!(Value::Null, vector.get(1));
assert_eq!(ValueRef::Null, vector.get_ref(1));
assert_eq!(Some(Date::new(-1)), vector.get_data(2));
let mut iter = vector.iter_data();
assert_eq!(Some(Date::new(1)), iter.next().unwrap());
assert_eq!(None, iter.next().unwrap());
assert_eq!(Some(Date::new(-1)), iter.next().unwrap());
}
#[test]
fn test_date_scalar() {
let vector = DateVector::from_slice(&[Date::new(1), Date::new(2)]);
assert_eq!(2, vector.len());
assert_eq!(Some(Date::new(1)), vector.get_data(0));
assert_eq!(Some(Date::new(2)), vector.get_data(1));
}
#[test]
fn test_date_vector_builder() {
let input = DateVector::from_slice(&[Date::new(1), Date::new(2), Date::new(3)]);
let mut builder = DateType::default().create_mutable_vector(3);
builder
.push_value_ref(ValueRef::Date(Date::new(5)))
.unwrap();
assert!(builder.push_value_ref(ValueRef::Int32(123)).is_err());
builder.extend_slice_of(&input, 1, 2).unwrap();
assert!(builder
.extend_slice_of(&crate::vectors::Int32Vector::from_slice(&[13]), 0, 1)
.is_err());
let vector = builder.to_vector();
let expect: VectorRef = Arc::new(DateVector::from_slice(&[
Date::new(5),
Date::new(2),
Date::new(3),
]));
assert_eq!(expect, vector);
}
#[test]
fn test_date_from_arrow() {
let vector = DateVector::from_slice(&[Date::new(1), Date::new(2)]);
let arrow = vector.as_arrow().slice(0, vector.len());
let vector2 = DateVector::try_from_arrow_array(&arrow).unwrap();
assert_eq!(vector, vector2);
}
}

View File

@@ -1,346 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use std::sync::Arc;
use arrow::array::{Array, ArrayRef, PrimitiveArray};
use common_time::datetime::DateTime;
use snafu::OptionExt;
use crate::data_type::ConcreteDataType;
use crate::error::{self, Result};
use crate::prelude::{
MutableVector, ScalarVector, ScalarVectorBuilder, Validity, Value, ValueRef, Vector, VectorRef,
};
use crate::serialize::Serializable;
use crate::vectors::{PrimitiveIter, PrimitiveVector, PrimitiveVectorBuilder};
#[derive(Debug, Clone, PartialEq)]
pub struct DateTimeVector {
array: PrimitiveVector<i64>,
}
impl DateTimeVector {
pub fn new(array: PrimitiveArray<i64>) -> Self {
Self {
array: PrimitiveVector { array },
}
}
pub fn try_from_arrow_array(array: impl AsRef<dyn Array>) -> Result<Self> {
Ok(Self::new(
array
.as_ref()
.as_any()
.downcast_ref::<PrimitiveArray<i64>>()
.with_context(|| error::ConversionSnafu {
from: format!("{:?}", array.as_ref().data_type()),
})?
.clone(),
))
}
pub(crate) fn as_arrow(&self) -> &dyn Array {
self.array.as_arrow()
}
}
impl Vector for DateTimeVector {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::datetime_datatype()
}
fn vector_type_name(&self) -> String {
"DateTimeVector".to_string()
}
fn as_any(&self) -> &dyn Any {
self
}
fn len(&self) -> usize {
self.array.len()
}
fn to_arrow_array(&self) -> ArrayRef {
let validity = self.array.array.validity().cloned();
let buffer = self.array.array.values().clone();
Arc::new(PrimitiveArray::new(
arrow::datatypes::DataType::Date64,
buffer,
validity,
))
}
fn to_boxed_arrow_array(&self) -> Box<dyn Array> {
let validity = self.array.array.validity().cloned();
let buffer = self.array.array.values().clone();
Box::new(PrimitiveArray::new(
arrow::datatypes::DataType::Date64,
buffer,
validity,
))
}
fn validity(&self) -> Validity {
self.array.validity()
}
fn memory_size(&self) -> usize {
self.array.memory_size()
}
fn is_null(&self, row: usize) -> bool {
self.array.is_null(row)
}
fn slice(&self, offset: usize, length: usize) -> VectorRef {
Arc::new(Self {
array: PrimitiveVector::new(self.array.array.slice(offset, length)),
})
}
fn get(&self, index: usize) -> Value {
match self.array.get(index) {
Value::Int64(v) => Value::DateTime(DateTime::new(v)),
Value::Null => Value::Null,
_ => {
unreachable!()
}
}
}
fn get_ref(&self, index: usize) -> ValueRef {
match self.array.get(index) {
Value::Int64(v) => ValueRef::DateTime(DateTime::new(v)),
Value::Null => ValueRef::Null,
_ => {
unreachable!()
}
}
}
}
impl Serializable for DateTimeVector {
fn serialize_to_json(&self) -> crate::Result<Vec<serde_json::Value>> {
Ok(self
.array
.iter_data()
.map(|v| v.map(DateTime::new))
.map(|v| match v {
None => serde_json::Value::Null,
Some(v) => v.into(),
})
.collect::<Vec<_>>())
}
}
impl From<Vec<Option<i64>>> for DateTimeVector {
fn from(data: Vec<Option<i64>>) -> Self {
Self {
array: PrimitiveVector::<i64>::from(data),
}
}
}
pub struct DateTimeVectorBuilder {
buffer: PrimitiveVectorBuilder<i64>,
}
impl ScalarVectorBuilder for DateTimeVectorBuilder {
type VectorType = DateTimeVector;
fn with_capacity(capacity: usize) -> Self {
Self {
buffer: PrimitiveVectorBuilder::with_capacity(capacity),
}
}
fn push(&mut self, value: Option<<Self::VectorType as ScalarVector>::RefItem<'_>>) {
self.buffer.push(value.map(|d| d.val()))
}
fn finish(&mut self) -> Self::VectorType {
Self::VectorType {
array: self.buffer.finish(),
}
}
}
impl MutableVector for DateTimeVectorBuilder {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::datetime_datatype()
}
fn len(&self) -> usize {
self.buffer.len()
}
fn as_any(&self) -> &dyn Any {
self
}
fn as_mut_any(&mut self) -> &mut dyn Any {
self
}
fn to_vector(&mut self) -> VectorRef {
Arc::new(self.finish())
}
fn push_value_ref(&mut self, value: ValueRef) -> Result<()> {
self.buffer.push(value.as_datetime()?.map(|d| d.val()));
Ok(())
}
fn extend_slice_of(&mut self, vector: &dyn Vector, offset: usize, length: usize) -> Result<()> {
let concrete_vector = vector
.as_any()
.downcast_ref::<DateTimeVector>()
.with_context(|| error::CastTypeSnafu {
msg: format!(
"Failed to convert vector from {} to DateVector",
vector.vector_type_name()
),
})?;
self.buffer
.extend_slice_of(&concrete_vector.array, offset, length)?;
Ok(())
}
}
pub struct DateTimeIter<'a> {
iter: PrimitiveIter<'a, i64>,
}
impl<'a> Iterator for DateTimeIter<'a> {
type Item = Option<DateTime>;
fn next(&mut self) -> Option<Self::Item> {
self.iter.next().map(|v| v.map(DateTime::new))
}
}
impl ScalarVector for DateTimeVector {
type OwnedItem = DateTime;
type RefItem<'a> = DateTime;
type Iter<'a> = DateTimeIter<'a>;
type Builder = DateTimeVectorBuilder;
fn get_data(&self, idx: usize) -> Option<Self::RefItem<'_>> {
self.array.get_data(idx).map(DateTime::new)
}
fn iter_data(&self) -> Self::Iter<'_> {
DateTimeIter {
iter: self.array.iter_data(),
}
}
}
pub(crate) fn replicate_datetime(vector: &DateTimeVector, offsets: &[usize]) -> VectorRef {
let array = crate::vectors::primitive::replicate_primitive_with_type(
&vector.array,
offsets,
vector.data_type(),
);
Arc::new(DateTimeVector { array })
}
#[cfg(test)]
mod tests {
use std::assert_matches::assert_matches;
use super::*;
use crate::data_type::DataType;
use crate::types::DateTimeType;
#[test]
fn test_datetime_vector() {
let v = DateTimeVector::new(PrimitiveArray::from_vec(vec![1, 2, 3]));
assert_eq!(ConcreteDataType::datetime_datatype(), v.data_type());
assert_eq!(3, v.len());
assert_eq!("DateTimeVector", v.vector_type_name());
assert_eq!(
&arrow::datatypes::DataType::Date64,
v.to_arrow_array().data_type()
);
assert_eq!(Some(DateTime::new(1)), v.get_data(0));
assert_eq!(Value::DateTime(DateTime::new(1)), v.get(0));
assert_eq!(ValueRef::DateTime(DateTime::new(1)), v.get_ref(0));
let mut iter = v.iter_data();
assert_eq!(Some(DateTime::new(1)), iter.next().unwrap());
assert_eq!(Some(DateTime::new(2)), iter.next().unwrap());
assert_eq!(Some(DateTime::new(3)), iter.next().unwrap());
assert!(!v.is_null(0));
assert_eq!(24, v.memory_size()); // size of i64 * 3
assert_matches!(v.validity(), Validity::AllValid);
if let Value::DateTime(d) = v.get(0) {
assert_eq!(1, d.val());
} else {
unreachable!()
}
assert_eq!(
"[\"1970-01-01 00:00:01\",\"1970-01-01 00:00:02\",\"1970-01-01 00:00:03\"]",
serde_json::to_string(&v.serialize_to_json().unwrap()).unwrap()
);
}
#[test]
fn test_datetime_vector_builder() {
let mut builder = DateTimeVectorBuilder::with_capacity(3);
builder.push(Some(DateTime::new(1)));
builder.push(None);
builder.push(Some(DateTime::new(-1)));
let v = builder.finish();
assert_eq!(ConcreteDataType::datetime_datatype(), v.data_type());
assert_eq!(Value::DateTime(DateTime::new(1)), v.get(0));
assert_eq!(Value::Null, v.get(1));
assert_eq!(Value::DateTime(DateTime::new(-1)), v.get(2));
let input =
DateTimeVector::from_slice(&[DateTime::new(1), DateTime::new(2), DateTime::new(3)]);
let mut builder = DateTimeType::default().create_mutable_vector(3);
builder
.push_value_ref(ValueRef::DateTime(DateTime::new(5)))
.unwrap();
assert!(builder.push_value_ref(ValueRef::Int32(123)).is_err());
builder.extend_slice_of(&input, 1, 2).unwrap();
assert!(builder
.extend_slice_of(&crate::vectors::Int32Vector::from_slice(&[13]), 0, 1)
.is_err());
let vector = builder.to_vector();
let expect: VectorRef = Arc::new(DateTimeVector::from_slice(&[
DateTime::new(5),
DateTime::new(2),
DateTime::new(3),
]));
assert_eq!(expect, vector);
}
#[test]
fn test_datetime_from_arrow() {
let vector = DateTimeVector::from_slice(&[DateTime::new(1), DateTime::new(2)]);
let arrow = vector.as_arrow().slice(0, vector.len());
let vector2 = DateTimeVector::try_from_arrow_array(&arrow).unwrap();
assert_eq!(vector, vector2);
}
}

View File

@@ -1,208 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::sync::Arc;
use crate::data_type::DataType;
use crate::vectors::{
BinaryVector, BooleanVector, ConstantVector, DateTimeVector, DateVector, ListVector,
PrimitiveVector, StringVector, TimestampVector, Vector,
};
use crate::with_match_primitive_type_id;
impl Eq for dyn Vector + '_ {}
impl PartialEq for dyn Vector + '_ {
fn eq(&self, other: &dyn Vector) -> bool {
equal(self, other)
}
}
impl PartialEq<dyn Vector> for Arc<dyn Vector + '_> {
fn eq(&self, other: &dyn Vector) -> bool {
equal(&**self, other)
}
}
macro_rules! is_vector_eq {
($VectorType: ident, $lhs: ident, $rhs: ident) => {{
let lhs = $lhs.as_any().downcast_ref::<$VectorType>().unwrap();
let rhs = $rhs.as_any().downcast_ref::<$VectorType>().unwrap();
lhs == rhs
}};
}
fn equal(lhs: &dyn Vector, rhs: &dyn Vector) -> bool {
if lhs.data_type() != rhs.data_type() || lhs.len() != rhs.len() {
return false;
}
if lhs.is_const() || rhs.is_const() {
// Length has been checked before, so we only need to compare inner
// vector here.
return equal(
&**lhs
.as_any()
.downcast_ref::<ConstantVector>()
.unwrap()
.inner(),
&**lhs
.as_any()
.downcast_ref::<ConstantVector>()
.unwrap()
.inner(),
);
}
use crate::data_type::ConcreteDataType::*;
let lhs_type = lhs.data_type();
match lhs.data_type() {
Null(_) => true,
Boolean(_) => is_vector_eq!(BooleanVector, lhs, rhs),
Binary(_) => is_vector_eq!(BinaryVector, lhs, rhs),
String(_) => is_vector_eq!(StringVector, lhs, rhs),
Date(_) => is_vector_eq!(DateVector, lhs, rhs),
DateTime(_) => is_vector_eq!(DateTimeVector, lhs, rhs),
Timestamp(_) => is_vector_eq!(TimestampVector, lhs, rhs),
List(_) => is_vector_eq!(ListVector, lhs, rhs),
UInt8(_) | UInt16(_) | UInt32(_) | UInt64(_) | Int8(_) | Int16(_) | Int32(_) | Int64(_)
| Float32(_) | Float64(_) => {
with_match_primitive_type_id!(lhs_type.logical_type_id(), |$T| {
let lhs = lhs.as_any().downcast_ref::<PrimitiveVector<$T>>().unwrap();
let rhs = rhs.as_any().downcast_ref::<PrimitiveVector<$T>>().unwrap();
lhs == rhs
},
{
unreachable!("should not compare {} with {}", lhs.vector_type_name(), rhs.vector_type_name())
})
}
}
}
#[cfg(test)]
mod tests {
use arrow::array::{ListArray, MutableListArray, MutablePrimitiveArray, TryExtend};
use super::*;
use crate::vectors::{
Float32Vector, Float64Vector, Int16Vector, Int32Vector, Int64Vector, Int8Vector,
NullVector, TimestampVector, UInt16Vector, UInt32Vector, UInt64Vector, UInt8Vector,
VectorRef,
};
fn assert_vector_ref_eq(vector: VectorRef) {
let rhs = vector.clone();
assert_eq!(vector, rhs);
assert_dyn_vector_eq(&*vector, &*rhs);
}
fn assert_dyn_vector_eq(lhs: &dyn Vector, rhs: &dyn Vector) {
assert_eq!(lhs, rhs);
}
fn assert_vector_ref_ne(lhs: VectorRef, rhs: VectorRef) {
assert_ne!(lhs, rhs);
}
#[test]
fn test_vector_eq() {
assert_vector_ref_eq(Arc::new(BinaryVector::from(vec![
Some(b"hello".to_vec()),
Some(b"world".to_vec()),
])));
assert_vector_ref_eq(Arc::new(BooleanVector::from(vec![true, false])));
assert_vector_ref_eq(Arc::new(ConstantVector::new(
Arc::new(BooleanVector::from(vec![true])),
5,
)));
assert_vector_ref_eq(Arc::new(BooleanVector::from(vec![true, false])));
assert_vector_ref_eq(Arc::new(DateVector::from(vec![Some(100), Some(120)])));
assert_vector_ref_eq(Arc::new(DateTimeVector::from(vec![Some(100), Some(120)])));
assert_vector_ref_eq(Arc::new(TimestampVector::from_values([100, 120])));
let mut arrow_array = MutableListArray::<i32, MutablePrimitiveArray<i64>>::new();
arrow_array
.try_extend(vec![Some(vec![Some(1), Some(2), Some(3)])])
.unwrap();
let arrow_array: ListArray<i32> = arrow_array.into();
assert_vector_ref_eq(Arc::new(ListVector::from(arrow_array)));
assert_vector_ref_eq(Arc::new(NullVector::new(4)));
assert_vector_ref_eq(Arc::new(StringVector::from(vec![
Some("hello"),
Some("world"),
])));
assert_vector_ref_eq(Arc::new(Int8Vector::from_slice(&[1, 2, 3, 4])));
assert_vector_ref_eq(Arc::new(UInt8Vector::from_slice(&[1, 2, 3, 4])));
assert_vector_ref_eq(Arc::new(Int16Vector::from_slice(&[1, 2, 3, 4])));
assert_vector_ref_eq(Arc::new(UInt16Vector::from_slice(&[1, 2, 3, 4])));
assert_vector_ref_eq(Arc::new(Int32Vector::from_slice(&[1, 2, 3, 4])));
assert_vector_ref_eq(Arc::new(UInt32Vector::from_slice(&[1, 2, 3, 4])));
assert_vector_ref_eq(Arc::new(Int64Vector::from_slice(&[1, 2, 3, 4])));
assert_vector_ref_eq(Arc::new(UInt64Vector::from_slice(&[1, 2, 3, 4])));
assert_vector_ref_eq(Arc::new(Float32Vector::from_slice(&[1.0, 2.0, 3.0, 4.0])));
assert_vector_ref_eq(Arc::new(Float64Vector::from_slice(&[1.0, 2.0, 3.0, 4.0])));
}
#[test]
fn test_vector_ne() {
assert_vector_ref_ne(
Arc::new(Int32Vector::from_slice(&[1, 2, 3, 4])),
Arc::new(Int32Vector::from_slice(&[1, 2])),
);
assert_vector_ref_ne(
Arc::new(Int32Vector::from_slice(&[1, 2, 3, 4])),
Arc::new(Int8Vector::from_slice(&[1, 2, 3, 4])),
);
assert_vector_ref_ne(
Arc::new(Int32Vector::from_slice(&[1, 2, 3, 4])),
Arc::new(BooleanVector::from(vec![true, true])),
);
assert_vector_ref_ne(
Arc::new(ConstantVector::new(
Arc::new(BooleanVector::from(vec![true])),
5,
)),
Arc::new(ConstantVector::new(
Arc::new(BooleanVector::from(vec![true])),
4,
)),
);
assert_vector_ref_ne(
Arc::new(ConstantVector::new(
Arc::new(BooleanVector::from(vec![true])),
5,
)),
Arc::new(ConstantVector::new(
Arc::new(BooleanVector::from(vec![false])),
4,
)),
);
assert_vector_ref_ne(
Arc::new(ConstantVector::new(
Arc::new(BooleanVector::from(vec![true])),
5,
)),
Arc::new(ConstantVector::new(
Arc::new(Int32Vector::from_slice(vec![1])),
4,
)),
);
assert_vector_ref_ne(Arc::new(NullVector::new(5)), Arc::new(NullVector::new(8)));
}
}

View File

@@ -1,304 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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.
//! Vector helper functions, inspired by databend Series mod
use std::any::Any;
use std::sync::Arc;
use arrow::array::Array;
use arrow::compute;
use arrow::datatypes::DataType as ArrowDataType;
use datafusion_common::ScalarValue;
use snafu::{OptionExt, ResultExt};
use crate::arrow_array::StringArray;
use crate::error::{ConversionSnafu, Result, UnknownVectorSnafu};
use crate::scalars::*;
use crate::vectors::date::DateVector;
use crate::vectors::datetime::DateTimeVector;
use crate::vectors::*;
pub struct Helper;
impl Helper {
/// Get a pointer to the underlying data of this vectors.
/// Can be useful for fast comparisons.
/// # Safety
/// Assumes that the `vector` is T.
pub unsafe fn static_cast<T: Any>(vector: &VectorRef) -> &T {
let object = vector.as_ref();
debug_assert!(object.as_any().is::<T>());
&*(object as *const dyn Vector as *const T)
}
pub fn check_get_scalar<T: Scalar>(vector: &VectorRef) -> Result<&<T as Scalar>::VectorType> {
let arr = vector
.as_any()
.downcast_ref::<<T as Scalar>::VectorType>()
.with_context(|| UnknownVectorSnafu {
msg: format!(
"downcast vector error, vector type: {:?}, expected vector: {:?}",
vector.vector_type_name(),
std::any::type_name::<T>(),
),
});
arr
}
pub fn check_get<T: 'static + Vector>(vector: &VectorRef) -> Result<&T> {
let arr = vector
.as_any()
.downcast_ref::<T>()
.with_context(|| UnknownVectorSnafu {
msg: format!(
"downcast vector error, vector type: {:?}, expected vector: {:?}",
vector.vector_type_name(),
std::any::type_name::<T>(),
),
});
arr
}
pub fn check_get_mutable_vector<T: 'static + MutableVector>(
vector: &mut dyn MutableVector,
) -> Result<&mut T> {
let ty = vector.data_type();
let arr = vector
.as_mut_any()
.downcast_mut()
.with_context(|| UnknownVectorSnafu {
msg: format!(
"downcast vector error, vector type: {:?}, expected vector: {:?}",
ty,
std::any::type_name::<T>(),
),
});
arr
}
pub fn check_get_scalar_vector<T: Scalar>(
vector: &VectorRef,
) -> Result<&<T as Scalar>::VectorType> {
let arr = vector
.as_any()
.downcast_ref::<<T as Scalar>::VectorType>()
.with_context(|| UnknownVectorSnafu {
msg: format!(
"downcast vector error, vector type: {:?}, expected vector: {:?}",
vector.vector_type_name(),
std::any::type_name::<T>(),
),
});
arr
}
/// Try to cast an arrow scalar value into vector
///
/// # Panics
/// Panic if given scalar value is not supported.
pub fn try_from_scalar_value(value: ScalarValue, length: usize) -> Result<VectorRef> {
let vector = match value {
ScalarValue::Boolean(v) => {
ConstantVector::new(Arc::new(BooleanVector::from(vec![v])), length)
}
ScalarValue::Float32(v) => {
ConstantVector::new(Arc::new(Float32Vector::from(vec![v])), length)
}
ScalarValue::Float64(v) => {
ConstantVector::new(Arc::new(Float64Vector::from(vec![v])), length)
}
ScalarValue::Int8(v) => {
ConstantVector::new(Arc::new(Int8Vector::from(vec![v])), length)
}
ScalarValue::Int16(v) => {
ConstantVector::new(Arc::new(Int16Vector::from(vec![v])), length)
}
ScalarValue::Int32(v) => {
ConstantVector::new(Arc::new(Int32Vector::from(vec![v])), length)
}
ScalarValue::Int64(v) => {
ConstantVector::new(Arc::new(Int64Vector::from(vec![v])), length)
}
ScalarValue::UInt8(v) => {
ConstantVector::new(Arc::new(UInt8Vector::from(vec![v])), length)
}
ScalarValue::UInt16(v) => {
ConstantVector::new(Arc::new(UInt16Vector::from(vec![v])), length)
}
ScalarValue::UInt32(v) => {
ConstantVector::new(Arc::new(UInt32Vector::from(vec![v])), length)
}
ScalarValue::UInt64(v) => {
ConstantVector::new(Arc::new(UInt64Vector::from(vec![v])), length)
}
ScalarValue::Utf8(v) => {
ConstantVector::new(Arc::new(StringVector::from(vec![v])), length)
}
ScalarValue::LargeUtf8(v) => {
ConstantVector::new(Arc::new(StringVector::from(vec![v])), length)
}
ScalarValue::Binary(v) => {
ConstantVector::new(Arc::new(BinaryVector::from(vec![v])), length)
}
ScalarValue::LargeBinary(v) => {
ConstantVector::new(Arc::new(BinaryVector::from(vec![v])), length)
}
ScalarValue::Date32(v) => {
ConstantVector::new(Arc::new(DateVector::from(vec![v])), length)
}
ScalarValue::Date64(v) => {
ConstantVector::new(Arc::new(DateTimeVector::from(vec![v])), length)
}
_ => {
return ConversionSnafu {
from: format!("Unsupported scalar value: {}", value),
}
.fail()
}
};
Ok(Arc::new(vector))
}
/// Try to cast an arrow array into vector
///
/// # Panics
/// Panic if given arrow data type is not supported.
pub fn try_into_vector(array: impl AsRef<dyn Array>) -> Result<VectorRef> {
Ok(match array.as_ref().data_type() {
ArrowDataType::Null => Arc::new(NullVector::try_from_arrow_array(array)?),
ArrowDataType::Boolean => Arc::new(BooleanVector::try_from_arrow_array(array)?),
ArrowDataType::Binary | ArrowDataType::LargeBinary => {
Arc::new(BinaryVector::try_from_arrow_array(array)?)
}
ArrowDataType::Int8 => Arc::new(Int8Vector::try_from_arrow_array(array)?),
ArrowDataType::Int16 => Arc::new(Int16Vector::try_from_arrow_array(array)?),
ArrowDataType::Int32 => Arc::new(Int32Vector::try_from_arrow_array(array)?),
ArrowDataType::Int64 => Arc::new(Int64Vector::try_from_arrow_array(array)?),
ArrowDataType::UInt8 => Arc::new(UInt8Vector::try_from_arrow_array(array)?),
ArrowDataType::UInt16 => Arc::new(UInt16Vector::try_from_arrow_array(array)?),
ArrowDataType::UInt32 => Arc::new(UInt32Vector::try_from_arrow_array(array)?),
ArrowDataType::UInt64 => Arc::new(UInt64Vector::try_from_arrow_array(array)?),
ArrowDataType::Float32 => Arc::new(Float32Vector::try_from_arrow_array(array)?),
ArrowDataType::Float64 => Arc::new(Float64Vector::try_from_arrow_array(array)?),
ArrowDataType::Utf8 | ArrowDataType::LargeUtf8 => {
Arc::new(StringVector::try_from_arrow_array(array)?)
}
ArrowDataType::Date32 => Arc::new(DateVector::try_from_arrow_array(array)?),
ArrowDataType::Date64 => Arc::new(DateTimeVector::try_from_arrow_array(array)?),
ArrowDataType::List(_) => Arc::new(ListVector::try_from_arrow_array(array)?),
ArrowDataType::Timestamp(_, _) => {
Arc::new(TimestampVector::try_from_arrow_array(array)?)
}
_ => unimplemented!("Arrow array datatype: {:?}", array.as_ref().data_type()),
})
}
pub fn try_into_vectors(arrays: &[ArrayRef]) -> Result<Vec<VectorRef>> {
arrays.iter().map(Self::try_into_vector).collect()
}
pub fn like_utf8(names: Vec<String>, s: &str) -> Result<VectorRef> {
let array = StringArray::from_slice(&names);
let filter =
compute::like::like_utf8_scalar(&array, s).context(error::ArrowComputeSnafu)?;
let result = compute::filter::filter(&array, &filter).context(error::ArrowComputeSnafu)?;
Helper::try_into_vector(result)
}
}
#[cfg(test)]
mod tests {
use arrow::array::Int32Array;
use common_time::date::Date;
use common_time::datetime::DateTime;
use super::*;
#[test]
fn test_try_into_vectors() {
let arrays: Vec<ArrayRef> = vec![
Arc::new(Int32Array::from_vec(vec![1])),
Arc::new(Int32Array::from_vec(vec![2])),
Arc::new(Int32Array::from_vec(vec![3])),
];
let vectors = Helper::try_into_vectors(&arrays);
assert!(vectors.is_ok());
let vectors = vectors.unwrap();
vectors.iter().for_each(|v| assert_eq!(1, v.len()));
assert_eq!(Value::Int32(1), vectors[0].get(0));
assert_eq!(Value::Int32(2), vectors[1].get(0));
assert_eq!(Value::Int32(3), vectors[2].get(0));
}
#[test]
pub fn test_try_into_date_vector() {
let vector = DateVector::from(vec![Some(1), Some(2), None]);
let arrow_array = vector.to_arrow_array();
assert_eq!(&arrow::datatypes::DataType::Date32, arrow_array.data_type());
let vector_converted = Helper::try_into_vector(arrow_array).unwrap();
assert_eq!(vector.len(), vector_converted.len());
for i in 0..vector_converted.len() {
assert_eq!(vector.get(i), vector_converted.get(i));
}
}
#[test]
pub fn test_try_from_scalar_date_value() {
let vector = Helper::try_from_scalar_value(ScalarValue::Date32(Some(42)), 3).unwrap();
assert_eq!(ConcreteDataType::date_datatype(), vector.data_type());
assert_eq!(3, vector.len());
for i in 0..vector.len() {
assert_eq!(Value::Date(Date::new(42)), vector.get(i));
}
}
#[test]
pub fn test_try_from_scalar_datetime_value() {
let vector = Helper::try_from_scalar_value(ScalarValue::Date64(Some(42)), 3).unwrap();
assert_eq!(ConcreteDataType::datetime_datatype(), vector.data_type());
assert_eq!(3, vector.len());
for i in 0..vector.len() {
assert_eq!(Value::DateTime(DateTime::new(42)), vector.get(i));
}
}
#[test]
fn test_like_utf8() {
fn assert_vector(expected: Vec<&str>, actual: &VectorRef) {
let actual = actual.as_any().downcast_ref::<StringVector>().unwrap();
assert_eq!(*actual, StringVector::from(expected));
}
let names: Vec<String> = vec!["greptime", "hello", "public", "world"]
.into_iter()
.map(|x| x.to_string())
.collect();
let ret = Helper::like_utf8(names.clone(), "%ll%").unwrap();
assert_vector(vec!["hello"], &ret);
let ret = Helper::like_utf8(names.clone(), "%time").unwrap();
assert_vector(vec!["greptime"], &ret);
let ret = Helper::like_utf8(names.clone(), "%ld").unwrap();
assert_vector(vec!["world"], &ret);
let ret = Helper::like_utf8(names, "%").unwrap();
assert_vector(vec!["greptime", "hello", "public", "world"], &ret);
}
}

View File

@@ -1,639 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use std::ops::Range;
use std::sync::Arc;
use arrow::array::{Array, ArrayRef, ListArray};
use arrow::bitmap::utils::ZipValidity;
use arrow::bitmap::MutableBitmap;
use arrow::datatypes::DataType as ArrowDataType;
use serde_json::Value as JsonValue;
use snafu::prelude::*;
use crate::error::Result;
use crate::prelude::*;
use crate::serialize::Serializable;
use crate::types::ListType;
use crate::value::{ListValue, ListValueRef};
use crate::vectors::{impl_try_from_arrow_array_for_vector, impl_validity_for_vector};
type ArrowListArray = ListArray<i32>;
/// Vector of Lists, basically backed by Arrow's `ListArray`.
#[derive(Debug, Clone, PartialEq)]
pub struct ListVector {
array: ArrowListArray,
inner_datatype: ConcreteDataType,
}
impl ListVector {
/// Only iterate values in the [ListVector].
///
/// Be careful to use this method as it would ignore validity and replace null
/// by empty vector.
pub fn values_iter(&self) -> Box<dyn Iterator<Item = Result<VectorRef>> + '_> {
Box::new(self.array.values_iter().map(VectorHelper::try_into_vector))
}
pub(crate) fn as_arrow(&self) -> &dyn Array {
&self.array
}
}
impl Vector for ListVector {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::List(ListType::new(self.inner_datatype.clone()))
}
fn vector_type_name(&self) -> String {
"ListVector".to_string()
}
fn as_any(&self) -> &dyn Any {
self
}
fn len(&self) -> usize {
self.array.len()
}
fn to_arrow_array(&self) -> ArrayRef {
Arc::new(self.array.clone())
}
fn to_boxed_arrow_array(&self) -> Box<dyn Array> {
Box::new(self.array.clone())
}
fn validity(&self) -> Validity {
impl_validity_for_vector!(self.array)
}
fn memory_size(&self) -> usize {
let offsets_bytes = self.array.offsets().len() * std::mem::size_of::<i64>();
let value_refs_bytes = self.array.values().len() * std::mem::size_of::<Arc<dyn Array>>();
offsets_bytes + value_refs_bytes
}
fn is_null(&self, row: usize) -> bool {
self.array.is_null(row)
}
fn slice(&self, offset: usize, length: usize) -> VectorRef {
Arc::new(ListVector::from(self.array.slice(offset, length)))
}
fn get(&self, index: usize) -> Value {
if !self.array.is_valid(index) {
return Value::Null;
}
let array = &self.array.value(index);
let vector = VectorHelper::try_into_vector(array).unwrap_or_else(|_| {
panic!(
"arrow array with datatype {:?} cannot converted to our vector",
array.data_type()
)
});
let values = (0..vector.len())
.map(|i| vector.get(i))
.collect::<Vec<Value>>();
Value::List(ListValue::new(
Some(Box::new(values)),
self.inner_datatype.clone(),
))
}
fn get_ref(&self, index: usize) -> ValueRef {
ValueRef::List(ListValueRef::Indexed {
vector: self,
idx: index,
})
}
}
impl Serializable for ListVector {
fn serialize_to_json(&self) -> Result<Vec<JsonValue>> {
self.array
.iter()
.map(|v| match v {
None => Ok(JsonValue::Null),
Some(v) => VectorHelper::try_into_vector(v)
.and_then(|v| v.serialize_to_json())
.map(JsonValue::Array),
})
.collect()
}
}
impl From<ArrowListArray> for ListVector {
fn from(array: ArrowListArray) -> Self {
let inner_datatype = ConcreteDataType::from_arrow_type(match array.data_type() {
ArrowDataType::List(field) => &field.data_type,
_ => unreachable!(),
});
Self {
array,
inner_datatype,
}
}
}
impl_try_from_arrow_array_for_vector!(ArrowListArray, ListVector);
pub struct ListVectorIter<'a> {
vector: &'a ListVector,
iter: ZipValidity<'a, usize, Range<usize>>,
}
impl<'a> ListVectorIter<'a> {
pub fn new(vector: &'a ListVector) -> ListVectorIter<'a> {
let iter = ZipValidity::new(
0..vector.len(),
vector.array.validity().as_ref().map(|x| x.iter()),
);
Self { vector, iter }
}
}
impl<'a> Iterator for ListVectorIter<'a> {
type Item = Option<ListValueRef<'a>>;
#[inline]
fn next(&mut self) -> Option<Self::Item> {
self.iter.next().map(|idx_opt| {
idx_opt.map(|idx| ListValueRef::Indexed {
vector: self.vector,
idx,
})
})
}
#[inline]
fn size_hint(&self) -> (usize, Option<usize>) {
self.iter.size_hint()
}
#[inline]
fn nth(&mut self, n: usize) -> Option<Self::Item> {
self.iter.nth(n).map(|idx_opt| {
idx_opt.map(|idx| ListValueRef::Indexed {
vector: self.vector,
idx,
})
})
}
}
impl ScalarVector for ListVector {
type OwnedItem = ListValue;
type RefItem<'a> = ListValueRef<'a>;
type Iter<'a> = ListVectorIter<'a>;
type Builder = ListVectorBuilder;
fn get_data(&self, idx: usize) -> Option<Self::RefItem<'_>> {
if self.array.is_valid(idx) {
Some(ListValueRef::Indexed { vector: self, idx })
} else {
None
}
}
fn iter_data(&self) -> Self::Iter<'_> {
ListVectorIter::new(self)
}
}
// Some codes are ported from arrow2's MutableListArray.
pub struct ListVectorBuilder {
inner_type: ConcreteDataType,
offsets: Vec<i32>,
values: Box<dyn MutableVector>,
validity: Option<MutableBitmap>,
}
impl ListVectorBuilder {
pub fn with_type_capacity(inner_type: ConcreteDataType, capacity: usize) -> ListVectorBuilder {
let mut offsets = Vec::with_capacity(capacity + 1);
offsets.push(0);
// The actual required capacity might greater than the capacity of the `ListVector`
// if there exists child vector that has more than one element.
let values = inner_type.create_mutable_vector(capacity);
ListVectorBuilder {
inner_type,
offsets,
values,
validity: None,
}
}
#[inline]
fn last_offset(&self) -> i32 {
*self.offsets.last().unwrap()
}
fn push_null(&mut self) {
self.offsets.push(self.last_offset());
match &mut self.validity {
Some(validity) => validity.push(false),
None => self.init_validity(),
}
}
fn init_validity(&mut self) {
let len = self.offsets.len() - 1;
let mut validity = MutableBitmap::with_capacity(self.offsets.capacity());
validity.extend_constant(len, true);
validity.set(len - 1, false);
self.validity = Some(validity)
}
fn push_list_value(&mut self, list_value: &ListValue) -> Result<()> {
if let Some(items) = list_value.items() {
for item in &**items {
self.values.push_value_ref(item.as_value_ref())?;
}
}
self.push_valid();
Ok(())
}
/// Needs to be called when a valid value was extended to this builder.
fn push_valid(&mut self) {
let size = self.values.len();
let size = i32::try_from(size).unwrap();
assert!(size >= *self.offsets.last().unwrap());
self.offsets.push(size);
if let Some(validity) = &mut self.validity {
validity.push(true)
}
}
}
impl MutableVector for ListVectorBuilder {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::list_datatype(self.inner_type.clone())
}
fn len(&self) -> usize {
self.offsets.len() - 1
}
fn as_any(&self) -> &dyn Any {
self
}
fn as_mut_any(&mut self) -> &mut dyn Any {
self
}
fn to_vector(&mut self) -> VectorRef {
Arc::new(self.finish())
}
fn push_value_ref(&mut self, value: ValueRef) -> Result<()> {
if let Some(list_ref) = value.as_list()? {
match list_ref {
ListValueRef::Indexed { vector, idx } => match vector.get(idx).as_list()? {
Some(list_value) => self.push_list_value(list_value)?,
None => self.push_null(),
},
ListValueRef::Ref { val } => self.push_list_value(val)?,
}
} else {
self.push_null();
}
Ok(())
}
fn extend_slice_of(&mut self, vector: &dyn Vector, offset: usize, length: usize) -> Result<()> {
for idx in offset..offset + length {
let value = vector.get_ref(idx);
self.push_value_ref(value)?;
}
Ok(())
}
}
impl ScalarVectorBuilder for ListVectorBuilder {
type VectorType = ListVector;
fn with_capacity(_capacity: usize) -> Self {
panic!("Must use ListVectorBuilder::with_type_capacity()");
}
fn push(&mut self, value: Option<<Self::VectorType as ScalarVector>::RefItem<'_>>) {
// We expect the input ListValue has the same inner type as the builder when using
// push(), so just panic if `push_value_ref()` returns error, which indicate an
// invalid input value type.
self.push_value_ref(value.into()).unwrap_or_else(|e| {
panic!(
"Failed to push value, expect value type {:?}, err:{}",
self.inner_type, e
);
});
}
fn finish(&mut self) -> Self::VectorType {
let array = ArrowListArray::try_new(
ConcreteDataType::list_datatype(self.inner_type.clone()).as_arrow_type(),
std::mem::take(&mut self.offsets).into(),
self.values.to_vector().to_arrow_array(),
std::mem::take(&mut self.validity).map(|x| x.into()),
)
.unwrap(); // The `ListVectorBuilder` itself should ensure it always builds a valid array.
ListVector {
array,
inner_datatype: self.inner_type.clone(),
}
}
}
#[cfg(test)]
mod tests {
use arrow::array::{MutableListArray, MutablePrimitiveArray, TryExtend};
use serde_json::json;
use super::*;
use crate::types::ListType;
#[test]
fn test_list_vector() {
let data = vec![
Some(vec![Some(1i32), Some(2), Some(3)]),
None,
Some(vec![Some(4), None, Some(6)]),
];
let mut arrow_array = MutableListArray::<i32, MutablePrimitiveArray<i32>>::new();
arrow_array.try_extend(data).unwrap();
let arrow_array: ArrowListArray = arrow_array.into();
let list_vector = ListVector {
array: arrow_array.clone(),
inner_datatype: ConcreteDataType::int32_datatype(),
};
assert_eq!(
ConcreteDataType::List(ListType::new(ConcreteDataType::int32_datatype())),
list_vector.data_type()
);
assert_eq!("ListVector", list_vector.vector_type_name());
assert_eq!(3, list_vector.len());
assert!(!list_vector.is_null(0));
assert!(list_vector.is_null(1));
assert!(!list_vector.is_null(2));
assert_eq!(
arrow_array,
list_vector
.to_arrow_array()
.as_any()
.downcast_ref::<ArrowListArray>()
.unwrap()
.clone()
);
assert_eq!(
Validity::Slots(arrow_array.validity().unwrap()),
list_vector.validity()
);
assert_eq!(
arrow_array.offsets().len() * std::mem::size_of::<i64>()
+ arrow_array.values().len() * std::mem::size_of::<Arc<dyn Array>>(),
list_vector.memory_size()
);
let slice = list_vector.slice(0, 2);
assert_eq!(
"ListArray[[1, 2, 3], None]",
format!("{:?}", slice.to_arrow_array())
);
assert_eq!(
Value::List(ListValue::new(
Some(Box::new(vec![
Value::Int32(1),
Value::Int32(2),
Value::Int32(3)
])),
ConcreteDataType::int32_datatype()
)),
list_vector.get(0)
);
let value_ref = list_vector.get_ref(0);
assert!(matches!(
value_ref,
ValueRef::List(ListValueRef::Indexed { .. })
));
let value_ref = list_vector.get_ref(1);
if let ValueRef::List(ListValueRef::Indexed { idx, .. }) = value_ref {
assert_eq!(1, idx);
} else {
unreachable!()
}
assert_eq!(Value::Null, list_vector.get(1));
assert_eq!(
Value::List(ListValue::new(
Some(Box::new(vec![
Value::Int32(4),
Value::Null,
Value::Int32(6)
])),
ConcreteDataType::int32_datatype()
)),
list_vector.get(2)
);
}
#[test]
fn test_from_arrow_array() {
let data = vec![
Some(vec![Some(1u32), Some(2), Some(3)]),
None,
Some(vec![Some(4), None, Some(6)]),
];
let mut arrow_array = MutableListArray::<i32, MutablePrimitiveArray<u32>>::new();
arrow_array.try_extend(data).unwrap();
let arrow_array: ArrowListArray = arrow_array.into();
let array_ref: ArrayRef = Arc::new(arrow_array);
let list_vector = ListVector::try_from_arrow_array(array_ref).unwrap();
assert_eq!(
"ListVector { array: ListArray[[1, 2, 3], None, [4, None, 6]], inner_datatype: UInt32(UInt32) }",
format!("{:?}", list_vector)
);
}
#[test]
fn test_iter_list_vector_values() {
let data = vec![
Some(vec![Some(1i64), Some(2), Some(3)]),
None,
Some(vec![Some(4), None, Some(6)]),
];
let mut arrow_array = MutableListArray::<i32, MutablePrimitiveArray<i64>>::new();
arrow_array.try_extend(data).unwrap();
let arrow_array: ArrowListArray = arrow_array.into();
let list_vector = ListVector::from(arrow_array);
assert_eq!(
ConcreteDataType::List(ListType::new(ConcreteDataType::int64_datatype())),
list_vector.data_type()
);
let mut iter = list_vector.values_iter();
assert_eq!(
"Int64[1, 2, 3]",
format!("{:?}", iter.next().unwrap().unwrap().to_arrow_array())
);
assert_eq!(
"Int64[]",
format!("{:?}", iter.next().unwrap().unwrap().to_arrow_array())
);
assert_eq!(
"Int64[4, None, 6]",
format!("{:?}", iter.next().unwrap().unwrap().to_arrow_array())
);
assert!(iter.next().is_none())
}
#[test]
fn test_serialize_to_json() {
let data = vec![
Some(vec![Some(1i64), Some(2), Some(3)]),
None,
Some(vec![Some(4), None, Some(6)]),
];
let mut arrow_array = MutableListArray::<i32, MutablePrimitiveArray<i64>>::new();
arrow_array.try_extend(data).unwrap();
let arrow_array: ArrowListArray = arrow_array.into();
let list_vector = ListVector::from(arrow_array);
assert_eq!(
vec![json!([1, 2, 3]), json!(null), json!([4, null, 6]),],
list_vector.serialize_to_json().unwrap()
);
}
fn new_list_vector(data: Vec<Option<Vec<Option<i32>>>>) -> ListVector {
let mut arrow_array = MutableListArray::<i32, MutablePrimitiveArray<i32>>::new();
arrow_array.try_extend(data).unwrap();
let arrow_array: ArrowListArray = arrow_array.into();
ListVector::from(arrow_array)
}
#[test]
fn test_list_vector_builder() {
let mut builder =
ListType::new(ConcreteDataType::int32_datatype()).create_mutable_vector(3);
builder
.push_value_ref(ValueRef::List(ListValueRef::Ref {
val: &ListValue::new(
Some(Box::new(vec![
Value::Int32(4),
Value::Null,
Value::Int32(6),
])),
ConcreteDataType::int32_datatype(),
),
}))
.unwrap();
assert!(builder.push_value_ref(ValueRef::Int32(123)).is_err());
let data = vec![
Some(vec![Some(1), Some(2), Some(3)]),
None,
Some(vec![Some(7), Some(8), None]),
];
let input = new_list_vector(data);
builder.extend_slice_of(&input, 1, 2).unwrap();
assert!(builder
.extend_slice_of(&crate::vectors::Int32Vector::from_slice(&[13]), 0, 1)
.is_err());
let vector = builder.to_vector();
let expect: VectorRef = Arc::new(new_list_vector(vec![
Some(vec![Some(4), None, Some(6)]),
None,
Some(vec![Some(7), Some(8), None]),
]));
assert_eq!(expect, vector);
}
#[test]
fn test_list_vector_for_scalar() {
let mut builder =
ListVectorBuilder::with_type_capacity(ConcreteDataType::int32_datatype(), 2);
builder.push(None);
builder.push(Some(ListValueRef::Ref {
val: &ListValue::new(
Some(Box::new(vec![
Value::Int32(4),
Value::Null,
Value::Int32(6),
])),
ConcreteDataType::int32_datatype(),
),
}));
let vector = builder.finish();
let expect = new_list_vector(vec![None, Some(vec![Some(4), None, Some(6)])]);
assert_eq!(expect, vector);
assert!(vector.get_data(0).is_none());
assert_eq!(
ListValueRef::Indexed {
vector: &vector,
idx: 1
},
vector.get_data(1).unwrap()
);
assert_eq!(
*vector.get(1).as_list().unwrap().unwrap(),
vector.get_data(1).unwrap().to_owned_scalar()
);
let mut iter = vector.iter_data();
assert!(iter.next().unwrap().is_none());
assert_eq!(
ListValueRef::Indexed {
vector: &vector,
idx: 1
},
iter.next().unwrap().unwrap()
);
assert!(iter.next().is_none());
let mut iter = vector.iter_data();
assert_eq!(2, iter.size_hint().0);
assert_eq!(
ListValueRef::Indexed {
vector: &vector,
idx: 1
},
iter.nth(1).unwrap().unwrap()
);
}
}

View File

@@ -1,54 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use crate::error::Result;
use crate::prelude::*;
/// Mutable vector that could be used to build an immutable vector.
pub trait MutableVector: Send + Sync {
/// Returns the data type of the vector.
fn data_type(&self) -> ConcreteDataType;
/// Returns the length of the vector.
fn len(&self) -> usize;
/// Returns whether the vector is empty.
fn is_empty(&self) -> bool {
self.len() == 0
}
/// Convert to Any, to enable dynamic casting.
fn as_any(&self) -> &dyn Any;
/// Convert to mutable Any, to enable dynamic casting.
fn as_mut_any(&mut self) -> &mut dyn Any;
/// Convert `self` to an (immutable) [VectorRef] and reset `self`.
fn to_vector(&mut self) -> VectorRef;
/// Push value ref to this mutable vector.
///
/// Returns error if data type unmatch.
fn push_value_ref(&mut self, value: ValueRef) -> Result<()>;
/// Extend this mutable vector by slice of `vector`.
///
/// Returns error if data type unmatch.
///
/// # Panics
/// Panics if `offset + length > vector.len()`.
fn extend_slice_of(&mut self, vector: &dyn Vector, offset: usize, length: usize) -> Result<()>;
}

View File

@@ -1,269 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use std::fmt;
use std::sync::Arc;
use arrow::array::{Array, ArrayRef, NullArray};
use arrow::datatypes::DataType as ArrowDataType;
use snafu::{ensure, OptionExt};
use crate::data_type::ConcreteDataType;
use crate::error::{self, Result};
use crate::serialize::Serializable;
use crate::types::NullType;
use crate::value::{Value, ValueRef};
use crate::vectors::{self, MutableVector, Validity, Vector, VectorRef};
#[derive(PartialEq)]
pub struct NullVector {
array: NullArray,
}
impl NullVector {
pub fn new(n: usize) -> Self {
Self {
array: NullArray::new(ArrowDataType::Null, n),
}
}
pub(crate) fn as_arrow(&self) -> &dyn Array {
&self.array
}
}
impl From<NullArray> for NullVector {
fn from(array: NullArray) -> Self {
Self { array }
}
}
impl Vector for NullVector {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::Null(NullType::default())
}
fn vector_type_name(&self) -> String {
"NullVector".to_string()
}
fn as_any(&self) -> &dyn Any {
self
}
fn len(&self) -> usize {
self.array.len()
}
fn to_arrow_array(&self) -> ArrayRef {
Arc::new(self.array.clone())
}
fn to_boxed_arrow_array(&self) -> Box<dyn Array> {
Box::new(self.array.clone())
}
fn validity(&self) -> Validity {
Validity::AllNull
}
fn memory_size(&self) -> usize {
0
}
fn is_null(&self, _row: usize) -> bool {
true
}
fn only_null(&self) -> bool {
true
}
fn slice(&self, _offset: usize, length: usize) -> VectorRef {
Arc::new(Self::new(length))
}
fn get(&self, _index: usize) -> Value {
// Skips bound check for null array.
Value::Null
}
fn get_ref(&self, _index: usize) -> ValueRef {
// Skips bound check for null array.
ValueRef::Null
}
}
impl fmt::Debug for NullVector {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "NullVector({})", self.len())
}
}
impl Serializable for NullVector {
fn serialize_to_json(&self) -> Result<Vec<serde_json::Value>> {
Ok(std::iter::repeat(serde_json::Value::Null)
.take(self.len())
.collect())
}
}
vectors::impl_try_from_arrow_array_for_vector!(NullArray, NullVector);
#[derive(Default)]
pub struct NullVectorBuilder {
length: usize,
}
impl MutableVector for NullVectorBuilder {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::null_datatype()
}
fn len(&self) -> usize {
self.length
}
fn as_any(&self) -> &dyn Any {
self
}
fn as_mut_any(&mut self) -> &mut dyn Any {
self
}
fn to_vector(&mut self) -> VectorRef {
let vector = Arc::new(NullVector::new(self.length));
self.length = 0;
vector
}
fn push_value_ref(&mut self, value: ValueRef) -> Result<()> {
ensure!(
value.is_null(),
error::CastTypeSnafu {
msg: format!("Failed to cast value ref {:?} to null", value),
}
);
self.length += 1;
Ok(())
}
fn extend_slice_of(&mut self, vector: &dyn Vector, offset: usize, length: usize) -> Result<()> {
vector
.as_any()
.downcast_ref::<NullVector>()
.with_context(|| error::CastTypeSnafu {
msg: format!(
"Failed to convert vector from {} to NullVector",
vector.vector_type_name()
),
})?;
assert!(
offset + length <= vector.len(),
"offset {} + length {} must less than {}",
offset,
length,
vector.len()
);
self.length += length;
Ok(())
}
}
pub(crate) fn replicate_null(vector: &NullVector, offsets: &[usize]) -> VectorRef {
assert_eq!(offsets.len(), vector.len());
if offsets.is_empty() {
return vector.slice(0, 0);
}
Arc::new(NullVector::new(*offsets.last().unwrap()))
}
#[cfg(test)]
mod tests {
use serde_json;
use super::*;
use crate::data_type::DataType;
#[test]
fn test_null_vector_misc() {
let v = NullVector::new(32);
assert_eq!(v.len(), 32);
assert_eq!(0, v.memory_size());
let arrow_arr = v.to_arrow_array();
assert_eq!(arrow_arr.null_count(), 32);
let array2 = arrow_arr.slice(8, 16);
assert_eq!(array2.len(), 16);
assert_eq!(array2.null_count(), 16);
assert_eq!("NullVector", v.vector_type_name());
assert!(!v.is_const());
assert_eq!(Validity::AllNull, v.validity());
assert!(v.only_null());
for i in 0..32 {
assert!(v.is_null(i));
assert_eq!(Value::Null, v.get(i));
assert_eq!(ValueRef::Null, v.get_ref(i));
}
}
#[test]
fn test_debug_null_vector() {
let array = NullVector::new(1024 * 1024);
assert_eq!(format!("{:?}", array), "NullVector(1048576)");
}
#[test]
fn test_serialize_json() {
let vector = NullVector::new(3);
let json_value = vector.serialize_to_json().unwrap();
assert_eq!(
"[null,null,null]",
serde_json::to_string(&json_value).unwrap()
);
}
#[test]
fn test_null_vector_validity() {
let vector = NullVector::new(5);
assert_eq!(Validity::AllNull, vector.validity());
assert_eq!(5, vector.null_count());
}
#[test]
fn test_null_vector_builder() {
let mut builder = NullType::default().create_mutable_vector(3);
builder.push_value_ref(ValueRef::Null).unwrap();
assert!(builder.push_value_ref(ValueRef::Int32(123)).is_err());
let input = NullVector::new(3);
builder.extend_slice_of(&input, 1, 2).unwrap();
assert!(builder
.extend_slice_of(&crate::vectors::Int32Vector::from_slice(&[13]), 0, 1)
.is_err());
let vector = builder.to_vector();
let expect: VectorRef = Arc::new(input);
assert_eq!(expect, vector);
}
}

View File

@@ -1,137 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 filter;
mod find_unique;
mod replicate;
use common_base::BitVec;
use crate::error::Result;
use crate::types::PrimitiveElement;
use crate::vectors::{
BinaryVector, BooleanVector, ConstantVector, DateTimeVector, DateVector, ListVector,
NullVector, PrimitiveVector, StringVector, TimestampVector, Vector, VectorRef,
};
/// Vector compute operations.
pub trait VectorOp {
/// Copies each element according `offsets` parameter.
/// - `i-th` element should be copied `offsets[i] - offsets[i - 1]` times
/// - `0-th` element would be copied `offsets[0]` times
///
/// # Panics
/// Panics if `offsets.len() != self.len()`.
fn replicate(&self, offsets: &[usize]) -> VectorRef;
/// Mark `i-th` bit of `selected` to `true` if the `i-th` element of `self` is unique, which
/// means there is no elements behind it have same value as it.
///
/// The caller should ensure
/// 1. the length of `selected` bitmap is equal to `vector.len()`.
/// 2. `vector` and `prev_vector` are sorted.
///
/// If there are multiple duplicate elements, this function retains the **first** element.
/// The first element is considered as unique if the first element of `self` is different
/// from its previous element, that is the last element of `prev_vector`.
///
/// # Panics
/// Panics if
/// - `selected.len() < self.len()`.
/// - `prev_vector` and `self` have different data types.
fn find_unique(&self, selected: &mut BitVec, prev_vector: Option<&dyn Vector>);
/// Filters the vector, returns elements matching the `filter` (i.e. where the values are true).
///
/// Note that the nulls of `filter` are interpreted as `false` will lead to these elements being masked out.
fn filter(&self, filter: &BooleanVector) -> Result<VectorRef>;
}
macro_rules! impl_scalar_vector_op {
($( { $VectorType: ident, $replicate: ident } ),+) => {$(
impl VectorOp for $VectorType {
fn replicate(&self, offsets: &[usize]) -> VectorRef {
replicate::$replicate(self, offsets)
}
fn find_unique(&self, selected: &mut BitVec, prev_vector: Option<&dyn Vector>) {
let prev_vector = prev_vector.map(|pv| pv.as_any().downcast_ref::<$VectorType>().unwrap());
find_unique::find_unique_scalar(self, selected, prev_vector);
}
fn filter(&self, filter: &BooleanVector) -> Result<VectorRef> {
filter::filter_non_constant!(self, $VectorType, filter)
}
}
)+};
}
impl_scalar_vector_op!(
{ BinaryVector, replicate_scalar },
{ BooleanVector, replicate_scalar },
{ ListVector, replicate_scalar },
{ StringVector, replicate_scalar },
{ DateVector, replicate_date },
{ DateTimeVector, replicate_datetime },
{ TimestampVector, replicate_timestamp }
);
impl VectorOp for ConstantVector {
fn replicate(&self, offsets: &[usize]) -> VectorRef {
replicate::replicate_constant(self, offsets)
}
fn find_unique(&self, selected: &mut BitVec, prev_vector: Option<&dyn Vector>) {
let prev_vector = prev_vector.and_then(|pv| pv.as_any().downcast_ref::<ConstantVector>());
find_unique::find_unique_constant(self, selected, prev_vector);
}
fn filter(&self, filter: &BooleanVector) -> Result<VectorRef> {
filter::filter_constant(self, filter)
}
}
impl VectorOp for NullVector {
fn replicate(&self, offsets: &[usize]) -> VectorRef {
replicate::replicate_null(self, offsets)
}
fn find_unique(&self, selected: &mut BitVec, prev_vector: Option<&dyn Vector>) {
let prev_vector = prev_vector.and_then(|pv| pv.as_any().downcast_ref::<NullVector>());
find_unique::find_unique_null(self, selected, prev_vector);
}
fn filter(&self, filter: &BooleanVector) -> Result<VectorRef> {
filter::filter_non_constant!(self, NullVector, filter)
}
}
impl<T> VectorOp for PrimitiveVector<T>
where
T: PrimitiveElement,
{
fn replicate(&self, offsets: &[usize]) -> VectorRef {
replicate::replicate_primitive(self, offsets)
}
fn find_unique(&self, selected: &mut BitVec, prev_vector: Option<&dyn Vector>) {
let prev_vector =
prev_vector.and_then(|pv| pv.as_any().downcast_ref::<PrimitiveVector<T>>());
find_unique::find_unique_scalar(self, selected, prev_vector);
}
fn filter(&self, filter: &BooleanVector) -> Result<VectorRef> {
filter::filter_non_constant!(self, PrimitiveVector<T>, filter)
}
}

View File

@@ -1,128 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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.
pub(crate) use crate::vectors::constant::filter_constant;
macro_rules! filter_non_constant {
($vector: expr, $VectorType: ty, $filter: ident) => {{
use std::sync::Arc;
use snafu::ResultExt;
let arrow_array = $vector.as_arrow();
let filtered = arrow::compute::filter::filter(arrow_array, $filter.as_boolean_array())
.context(crate::error::ArrowComputeSnafu)?;
Ok(Arc::new(<$VectorType>::try_from_arrow_array(filtered)?))
}};
}
pub(crate) use filter_non_constant;
#[cfg(test)]
mod tests {
use std::sync::Arc;
use crate::scalars::ScalarVector;
use crate::vectors::{
BooleanVector, ConstantVector, Int32Vector, NullVector, StringVector, VectorOp, VectorRef,
};
fn check_filter_primitive(expect: &[i32], input: &[i32], filter: &[bool]) {
let v = Int32Vector::from_slice(&input);
let filter = BooleanVector::from_slice(filter);
let out = v.filter(&filter).unwrap();
let expect: VectorRef = Arc::new(Int32Vector::from_slice(&expect));
assert_eq!(expect, out);
}
#[test]
fn test_filter_primitive() {
check_filter_primitive(&[], &[], &[]);
check_filter_primitive(&[5], &[5], &[true]);
check_filter_primitive(&[], &[5], &[false]);
check_filter_primitive(&[], &[5, 6], &[false, false]);
check_filter_primitive(&[5, 6], &[5, 6], &[true, true]);
check_filter_primitive(&[], &[5, 6, 7], &[false, false, false]);
check_filter_primitive(&[5], &[5, 6, 7], &[true, false, false]);
check_filter_primitive(&[6], &[5, 6, 7], &[false, true, false]);
check_filter_primitive(&[7], &[5, 6, 7], &[false, false, true]);
check_filter_primitive(&[5, 7], &[5, 6, 7], &[true, false, true]);
}
fn check_filter_constant(expect_length: usize, input_length: usize, filter: &[bool]) {
let v = ConstantVector::new(Arc::new(Int32Vector::from_slice(&[123])), input_length);
let filter = BooleanVector::from_slice(filter);
let out = v.filter(&filter).unwrap();
assert!(out.is_const());
assert_eq!(expect_length, out.len());
}
#[test]
fn test_filter_constant() {
check_filter_constant(0, 0, &[]);
check_filter_constant(1, 1, &[true]);
check_filter_constant(0, 1, &[false]);
check_filter_constant(1, 2, &[false, true]);
check_filter_constant(2, 2, &[true, true]);
check_filter_constant(1, 4, &[false, false, false, true]);
check_filter_constant(2, 4, &[false, true, false, true]);
}
#[test]
fn test_filter_scalar() {
let v = StringVector::from_slice(&["0", "1", "2", "3"]);
let filter = BooleanVector::from_slice(&[false, true, false, true]);
let out = v.filter(&filter).unwrap();
let expect: VectorRef = Arc::new(StringVector::from_slice(&["1", "3"]));
assert_eq!(expect, out);
}
#[test]
fn test_filter_null() {
let v = NullVector::new(5);
let filter = BooleanVector::from_slice(&[false, true, false, true, true]);
let out = v.filter(&filter).unwrap();
let expect: VectorRef = Arc::new(NullVector::new(3));
assert_eq!(expect, out);
}
macro_rules! impl_filter_date_like_test {
($VectorType: ident, $ValueType: ident, $method: ident) => {{
use std::sync::Arc;
use common_time::$ValueType;
use $crate::vectors::{$VectorType, VectorRef};
let v = $VectorType::from_iterator((0..5).map($ValueType::$method));
let filter = BooleanVector::from_slice(&[false, true, false, true, true]);
let out = v.filter(&filter).unwrap();
let expect: VectorRef = Arc::new($VectorType::from_iterator(
[1, 3, 4].into_iter().map($ValueType::$method),
));
assert_eq!(expect, out);
}};
}
#[test]
fn test_filter_date_like() {
impl_filter_date_like_test!(DateVector, Date, new);
impl_filter_date_like_test!(DateTimeVector, DateTime, new);
impl_filter_date_like_test!(TimestampVector, Timestamp, from_millis);
}
}

View File

@@ -1,361 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 common_base::BitVec;
use crate::scalars::ScalarVector;
use crate::vectors::{ConstantVector, NullVector, Vector};
// To implement `find_unique()` correctly, we need to keep in mind that always marks an element as
// selected when it is different from the previous one, and leaves the `selected` unchanged
// in any other case.
pub(crate) fn find_unique_scalar<'a, T: ScalarVector>(
vector: &'a T,
selected: &'a mut BitVec,
prev_vector: Option<&'a T>,
) where
T::RefItem<'a>: PartialEq,
{
assert!(selected.len() >= vector.len());
if vector.is_empty() {
return;
}
for ((i, current), next) in vector
.iter_data()
.enumerate()
.zip(vector.iter_data().skip(1))
{
if current != next {
// If next element is a different element, we mark it as selected.
selected.set(i + 1, true);
}
}
// Marks first element as selected if it is different from previous element, otherwise
// keep selected bitmap unchanged.
let is_first_not_duplicate = prev_vector
.map(|pv| {
if pv.is_empty() {
true
} else {
let last = pv.get_data(pv.len() - 1);
last != vector.get_data(0)
}
})
.unwrap_or(true);
if is_first_not_duplicate {
selected.set(0, true);
}
}
pub(crate) fn find_unique_null(
vector: &NullVector,
selected: &mut BitVec,
prev_vector: Option<&NullVector>,
) {
if vector.is_empty() {
return;
}
let is_first_not_duplicate = prev_vector.map(|pv| pv.is_empty()).unwrap_or(true);
if is_first_not_duplicate {
selected.set(0, true);
}
}
pub(crate) fn find_unique_constant(
vector: &ConstantVector,
selected: &mut BitVec,
prev_vector: Option<&ConstantVector>,
) {
if vector.is_empty() {
return;
}
let is_first_not_duplicate = prev_vector
.map(|pv| {
if pv.is_empty() {
true
} else {
vector.get_constant_ref() != pv.get_constant_ref()
}
})
.unwrap_or(true);
if is_first_not_duplicate {
selected.set(0, true);
}
}
#[cfg(test)]
mod tests {
use std::sync::Arc;
use super::*;
use crate::vectors::{Int32Vector, StringVector, VectorOp};
fn check_bitmap(expect: &[bool], selected: &BitVec) {
let actual = selected.iter().collect::<Vec<_>>();
assert_eq!(expect, actual);
}
fn check_find_unique_scalar(expect: &[bool], input: &[i32], prev: Option<&[i32]>) {
check_find_unique_scalar_opt(expect, input.iter().map(|v| Some(*v)), prev);
}
fn check_find_unique_scalar_opt(
expect: &[bool],
input: impl Iterator<Item = Option<i32>>,
prev: Option<&[i32]>,
) {
let input = Int32Vector::from_iter(input);
let prev = prev.map(Int32Vector::from_slice);
let mut selected = BitVec::repeat(false, input.len());
input.find_unique(&mut selected, prev.as_ref().map(|v| v as _));
check_bitmap(expect, &selected);
}
#[test]
fn test_find_unique_scalar() {
check_find_unique_scalar(&[], &[], None);
check_find_unique_scalar(&[true], &[1], None);
check_find_unique_scalar(&[true, false], &[1, 1], None);
check_find_unique_scalar(&[true, true], &[1, 2], None);
check_find_unique_scalar(&[true, true, true, true], &[1, 2, 3, 4], None);
check_find_unique_scalar(&[true, false, true, false], &[1, 1, 3, 3], None);
check_find_unique_scalar(&[true, false, false, false, true], &[2, 2, 2, 2, 3], None);
check_find_unique_scalar(&[true], &[5], Some(&[]));
check_find_unique_scalar(&[true], &[5], Some(&[3]));
check_find_unique_scalar(&[false], &[5], Some(&[5]));
check_find_unique_scalar(&[false], &[5], Some(&[4, 5]));
check_find_unique_scalar(&[false, true], &[5, 6], Some(&[4, 5]));
check_find_unique_scalar(&[false, true, false], &[5, 6, 6], Some(&[4, 5]));
check_find_unique_scalar(
&[false, true, false, true, true],
&[5, 6, 6, 7, 8],
Some(&[4, 5]),
);
check_find_unique_scalar_opt(
&[true, true, false, true, false],
[Some(1), Some(2), Some(2), None, None].into_iter(),
None,
);
}
#[test]
fn test_find_unique_scalar_multi_times_with_prev() {
let prev = Int32Vector::from_slice(&[1]);
let v1 = Int32Vector::from_slice(&[2, 3, 4]);
let mut selected = BitVec::repeat(false, v1.len());
v1.find_unique(&mut selected, Some(&prev));
// Though element in v2 are the same as prev, but we should still keep them.
let v2 = Int32Vector::from_slice(&[1, 1, 1]);
v2.find_unique(&mut selected, Some(&prev));
check_bitmap(&[true, true, true], &selected);
}
fn new_bitmap(bits: &[bool]) -> BitVec {
BitVec::from_iter(bits)
}
#[test]
fn test_find_unique_scalar_with_prev() {
let prev = Int32Vector::from_slice(&[1]);
let mut selected = new_bitmap(&[true, false, true, false]);
let v = Int32Vector::from_slice(&[2, 3, 4, 5]);
v.find_unique(&mut selected, Some(&prev));
// All elements are different.
check_bitmap(&[true, true, true, true], &selected);
let mut selected = new_bitmap(&[true, false, true, false]);
let v = Int32Vector::from_slice(&[1, 2, 3, 4]);
v.find_unique(&mut selected, Some(&prev));
// Though first element is duplicate, but we keep the flag unchanged.
check_bitmap(&[true, true, true, true], &selected);
// Same case as above, but now `prev` is None.
let mut selected = new_bitmap(&[true, false, true, false]);
let v = Int32Vector::from_slice(&[1, 2, 3, 4]);
v.find_unique(&mut selected, None);
check_bitmap(&[true, true, true, true], &selected);
// Same case as above, but now `prev` is empty.
let mut selected = new_bitmap(&[true, false, true, false]);
let v = Int32Vector::from_slice(&[1, 2, 3, 4]);
v.find_unique(&mut selected, Some(&Int32Vector::from_slice(&[])));
check_bitmap(&[true, true, true, true], &selected);
let mut selected = new_bitmap(&[false, false, false, false]);
let v = Int32Vector::from_slice(&[2, 2, 4, 5]);
v.find_unique(&mut selected, Some(&prev));
// only v[1] is duplicate.
check_bitmap(&[true, false, true, true], &selected);
}
fn check_find_unique_null(len: usize) {
let input = NullVector::new(len);
let mut selected = BitVec::repeat(false, input.len());
input.find_unique(&mut selected, None);
let mut expect = vec![false; len];
if !expect.is_empty() {
expect[0] = true;
}
check_bitmap(&expect, &selected);
let mut selected = BitVec::repeat(false, input.len());
let prev = Some(NullVector::new(1));
input.find_unique(&mut selected, prev.as_ref().map(|v| v as _));
let expect = vec![false; len];
check_bitmap(&expect, &selected);
}
#[test]
fn test_find_unique_null() {
for len in 0..5 {
check_find_unique_null(len);
}
}
#[test]
fn test_find_unique_null_with_prev() {
let prev = NullVector::new(1);
// Keep flags unchanged.
let mut selected = new_bitmap(&[true, false, true, false]);
let v = NullVector::new(4);
v.find_unique(&mut selected, Some(&prev));
check_bitmap(&[true, false, true, false], &selected);
// Keep flags unchanged.
let mut selected = new_bitmap(&[false, false, true, false]);
v.find_unique(&mut selected, Some(&prev));
check_bitmap(&[false, false, true, false], &selected);
// Prev is None, select first element.
let mut selected = new_bitmap(&[false, false, true, false]);
v.find_unique(&mut selected, None);
check_bitmap(&[true, false, true, false], &selected);
// Prev is empty, select first element.
let mut selected = new_bitmap(&[false, false, true, false]);
v.find_unique(&mut selected, Some(&NullVector::new(0)));
check_bitmap(&[true, false, true, false], &selected);
}
fn check_find_unique_constant(len: usize) {
let input = ConstantVector::new(Arc::new(Int32Vector::from_slice(&[8])), len);
let mut selected = BitVec::repeat(false, len);
input.find_unique(&mut selected, None);
let mut expect = vec![false; len];
if !expect.is_empty() {
expect[0] = true;
}
check_bitmap(&expect, &selected);
let mut selected = BitVec::repeat(false, len);
let prev = Some(ConstantVector::new(
Arc::new(Int32Vector::from_slice(&[8])),
1,
));
input.find_unique(&mut selected, prev.as_ref().map(|v| v as _));
let expect = vec![false; len];
check_bitmap(&expect, &selected);
}
#[test]
fn test_find_unique_constant() {
for len in 0..5 {
check_find_unique_constant(len);
}
}
#[test]
fn test_find_unique_constant_with_prev() {
let prev = ConstantVector::new(Arc::new(Int32Vector::from_slice(&[1])), 1);
// Keep flags unchanged.
let mut selected = new_bitmap(&[true, false, true, false]);
let v = ConstantVector::new(Arc::new(Int32Vector::from_slice(&[1])), 4);
v.find_unique(&mut selected, Some(&prev));
check_bitmap(&[true, false, true, false], &selected);
// Keep flags unchanged.
let mut selected = new_bitmap(&[false, false, true, false]);
v.find_unique(&mut selected, Some(&prev));
check_bitmap(&[false, false, true, false], &selected);
// Prev is None, select first element.
let mut selected = new_bitmap(&[false, false, true, false]);
v.find_unique(&mut selected, None);
check_bitmap(&[true, false, true, false], &selected);
// Prev is empty, select first element.
let mut selected = new_bitmap(&[false, false, true, false]);
v.find_unique(
&mut selected,
Some(&ConstantVector::new(
Arc::new(Int32Vector::from_slice(&[1])),
0,
)),
);
check_bitmap(&[true, false, true, false], &selected);
// Different constant vector.
let mut selected = new_bitmap(&[false, false, true, false]);
let v = ConstantVector::new(Arc::new(Int32Vector::from_slice(&[2])), 4);
v.find_unique(&mut selected, Some(&prev));
check_bitmap(&[true, false, true, false], &selected);
}
#[test]
fn test_find_unique_string() {
let input = StringVector::from_slice(&["a", "a", "b", "c"]);
let mut selected = BitVec::repeat(false, 4);
input.find_unique(&mut selected, None);
let expect = vec![true, false, true, true];
check_bitmap(&expect, &selected);
}
macro_rules! impl_find_unique_date_like_test {
($VectorType: ident, $ValueType: ident, $method: ident) => {{
use common_time::$ValueType;
use $crate::vectors::$VectorType;
let v = $VectorType::from_iterator([8, 8, 9, 10].into_iter().map($ValueType::$method));
let mut selected = BitVec::repeat(false, 4);
v.find_unique(&mut selected, None);
let expect = vec![true, false, true, true];
check_bitmap(&expect, &selected);
}};
}
#[test]
fn test_find_unique_date_like() {
impl_find_unique_date_like_test!(DateVector, Date, new);
impl_find_unique_date_like_test!(DateTimeVector, DateTime, new);
impl_find_unique_date_like_test!(TimestampVector, Timestamp, from_millis);
}
}

View File

@@ -1,147 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 crate::prelude::*;
pub(crate) use crate::vectors::constant::replicate_constant;
pub(crate) use crate::vectors::date::replicate_date;
pub(crate) use crate::vectors::datetime::replicate_datetime;
pub(crate) use crate::vectors::null::replicate_null;
pub(crate) use crate::vectors::primitive::replicate_primitive;
pub(crate) use crate::vectors::timestamp::replicate_timestamp;
pub(crate) fn replicate_scalar<C: ScalarVector>(c: &C, offsets: &[usize]) -> VectorRef {
assert_eq!(offsets.len(), c.len());
if offsets.is_empty() {
return c.slice(0, 0);
}
let mut builder = <<C as ScalarVector>::Builder>::with_capacity(c.len());
let mut previous_offset = 0;
for (i, offset) in offsets.iter().enumerate() {
let data = c.get_data(i);
for _ in previous_offset..*offset {
builder.push(data);
}
previous_offset = *offset;
}
builder.to_vector()
}
#[cfg(test)]
mod tests {
use std::sync::Arc;
use super::*;
use crate::vectors::{ConstantVector, Int32Vector, NullVector, StringVector, VectorOp};
#[test]
fn test_replicate_primitive() {
let v = Int32Vector::from_iterator(0..5);
let offsets = [0, 1, 2, 3, 4];
let v = v.replicate(&offsets);
assert_eq!(4, v.len());
for i in 0..4 {
assert_eq!(Value::Int32(i as i32 + 1), v.get(i));
}
}
#[test]
fn test_replicate_nullable_primitive() {
let v = Int32Vector::from(vec![None, Some(1), None, Some(2)]);
let offsets = [2, 4, 6, 8];
let v = v.replicate(&offsets);
assert_eq!(8, v.len());
let expect: VectorRef = Arc::new(Int32Vector::from(vec![
None,
None,
Some(1),
Some(1),
None,
None,
Some(2),
Some(2),
]));
assert_eq!(expect, v);
}
#[test]
fn test_replicate_scalar() {
let v = StringVector::from_slice(&["0", "1", "2", "3"]);
let offsets = [1, 3, 5, 6];
let v = v.replicate(&offsets);
assert_eq!(6, v.len());
let expect: VectorRef = Arc::new(StringVector::from_slice(&["0", "1", "1", "2", "2", "3"]));
assert_eq!(expect, v);
}
#[test]
fn test_replicate_constant() {
let v = Arc::new(StringVector::from_slice(&["hello"]));
let cv = ConstantVector::new(v.clone(), 2);
let offsets = [1, 4];
let cv = cv.replicate(&offsets);
assert_eq!(4, cv.len());
let expect: VectorRef = Arc::new(ConstantVector::new(v, 4));
assert_eq!(expect, cv);
}
#[test]
fn test_replicate_null() {
let v = NullVector::new(0);
let offsets = [];
let v = v.replicate(&offsets);
assert!(v.is_empty());
let v = NullVector::new(3);
let offsets = [1, 3, 5];
let v = v.replicate(&offsets);
assert_eq!(5, v.len());
}
macro_rules! impl_replicate_date_like_test {
($VectorType: ident, $ValueType: ident, $method: ident) => {{
use common_time::$ValueType;
use $crate::vectors::$VectorType;
let v = $VectorType::from_iterator((0..5).map($ValueType::$method));
let offsets = [0, 1, 2, 3, 4];
let v = v.replicate(&offsets);
assert_eq!(4, v.len());
for i in 0..4 {
assert_eq!(
Value::$ValueType($ValueType::$method((i as i32 + 1).into())),
v.get(i)
);
}
}};
}
#[test]
fn test_replicate_date_like() {
impl_replicate_date_like_test!(DateVector, Date, new);
impl_replicate_date_like_test!(DateTimeVector, DateTime, new);
impl_replicate_date_like_test!(TimestampVector, Timestamp, from_millis);
}
}

View File

@@ -1,492 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use std::iter::FromIterator;
use std::slice::Iter;
use std::sync::Arc;
use arrow::array::{Array, ArrayRef, MutableArray, MutablePrimitiveArray, PrimitiveArray};
use arrow::bitmap::utils::ZipValidity;
use serde_json::Value as JsonValue;
use snafu::{OptionExt, ResultExt};
use crate::data_type::{ConcreteDataType, DataType};
use crate::error::{ConversionSnafu, Result, SerializeSnafu};
use crate::scalars::{Scalar, ScalarRef, ScalarVector, ScalarVectorBuilder};
use crate::serialize::Serializable;
use crate::types::{Primitive, PrimitiveElement};
use crate::value::{Value, ValueRef};
use crate::vectors::{self, MutableVector, Validity, Vector, VectorRef};
/// Vector for primitive data types.
#[derive(Debug, Clone, PartialEq)]
pub struct PrimitiveVector<T: Primitive> {
pub(crate) array: PrimitiveArray<T>,
}
impl<T: Primitive> PrimitiveVector<T> {
pub fn new(array: PrimitiveArray<T>) -> Self {
Self { array }
}
pub fn try_from_arrow_array(array: impl AsRef<dyn Array>) -> Result<Self> {
Ok(Self::new(
array
.as_ref()
.as_any()
.downcast_ref::<PrimitiveArray<T>>()
.with_context(|| ConversionSnafu {
from: format!("{:?}", array.as_ref().data_type()),
})?
.clone(),
))
}
pub fn from_slice<P: AsRef<[T]>>(slice: P) -> Self {
Self {
array: PrimitiveArray::from_slice(slice),
}
}
pub fn from_vec(array: Vec<T>) -> Self {
Self {
array: PrimitiveArray::from_vec(array),
}
}
pub fn from_values<I: IntoIterator<Item = T>>(iter: I) -> Self {
Self {
array: PrimitiveArray::from_values(iter),
}
}
pub(crate) fn as_arrow(&self) -> &dyn Array {
&self.array
}
fn slice(&self, offset: usize, length: usize) -> Self {
Self::from(self.array.slice(offset, length))
}
}
impl<T: PrimitiveElement> Vector for PrimitiveVector<T> {
fn data_type(&self) -> ConcreteDataType {
T::build_data_type()
}
fn vector_type_name(&self) -> String {
format!("{}Vector", T::type_name())
}
fn as_any(&self) -> &dyn Any {
self
}
fn len(&self) -> usize {
self.array.len()
}
fn to_arrow_array(&self) -> ArrayRef {
Arc::new(self.array.clone())
}
fn to_boxed_arrow_array(&self) -> Box<dyn Array> {
Box::new(self.array.clone())
}
fn validity(&self) -> Validity {
vectors::impl_validity_for_vector!(self.array)
}
fn memory_size(&self) -> usize {
self.array.values().len() * std::mem::size_of::<T>()
}
fn is_null(&self, row: usize) -> bool {
self.array.is_null(row)
}
fn slice(&self, offset: usize, length: usize) -> VectorRef {
Arc::new(self.slice(offset, length))
}
fn get(&self, index: usize) -> Value {
vectors::impl_get_for_vector!(self.array, index)
}
fn get_ref(&self, index: usize) -> ValueRef {
if self.array.is_valid(index) {
// Safety: The index have been checked by `is_valid()`.
unsafe { self.array.value_unchecked(index).into_value_ref() }
} else {
ValueRef::Null
}
}
}
impl<T: Primitive> From<PrimitiveArray<T>> for PrimitiveVector<T> {
fn from(array: PrimitiveArray<T>) -> Self {
Self { array }
}
}
impl<T: Primitive> From<Vec<Option<T>>> for PrimitiveVector<T> {
fn from(v: Vec<Option<T>>) -> Self {
Self {
array: PrimitiveArray::<T>::from(v),
}
}
}
impl<T: Primitive, Ptr: std::borrow::Borrow<Option<T>>> FromIterator<Ptr> for PrimitiveVector<T> {
fn from_iter<I: IntoIterator<Item = Ptr>>(iter: I) -> Self {
Self {
array: MutablePrimitiveArray::<T>::from_iter(iter).into(),
}
}
}
impl<T> ScalarVector for PrimitiveVector<T>
where
T: PrimitiveElement,
{
type OwnedItem = T;
type RefItem<'a> = T;
type Iter<'a> = PrimitiveIter<'a, T>;
type Builder = PrimitiveVectorBuilder<T>;
fn get_data(&self, idx: usize) -> Option<Self::RefItem<'_>> {
if self.array.is_valid(idx) {
Some(self.array.value(idx))
} else {
None
}
}
fn iter_data(&self) -> Self::Iter<'_> {
PrimitiveIter {
iter: self.array.iter(),
}
}
}
pub type UInt8Vector = PrimitiveVector<u8>;
pub type UInt16Vector = PrimitiveVector<u16>;
pub type UInt32Vector = PrimitiveVector<u32>;
pub type UInt64Vector = PrimitiveVector<u64>;
pub type Int8Vector = PrimitiveVector<i8>;
pub type Int16Vector = PrimitiveVector<i16>;
pub type Int32Vector = PrimitiveVector<i32>;
pub type Int64Vector = PrimitiveVector<i64>;
pub type Float32Vector = PrimitiveVector<f32>;
pub type Float64Vector = PrimitiveVector<f64>;
pub struct PrimitiveIter<'a, T> {
iter: ZipValidity<'a, &'a T, Iter<'a, T>>,
}
impl<'a, T: Copy> Iterator for PrimitiveIter<'a, T> {
type Item = Option<T>;
fn next(&mut self) -> Option<Option<T>> {
self.iter.next().map(|v| v.copied())
}
}
impl<T: PrimitiveElement> Serializable for PrimitiveVector<T> {
fn serialize_to_json(&self) -> Result<Vec<JsonValue>> {
self.array
.iter()
.map(serde_json::to_value)
.collect::<serde_json::Result<_>>()
.context(SerializeSnafu)
}
}
pub struct PrimitiveVectorBuilder<T: PrimitiveElement> {
pub(crate) mutable_array: MutablePrimitiveArray<T>,
}
pub type UInt8VectorBuilder = PrimitiveVectorBuilder<u8>;
pub type UInt16VectorBuilder = PrimitiveVectorBuilder<u16>;
pub type UInt32VectorBuilder = PrimitiveVectorBuilder<u32>;
pub type UInt64VectorBuilder = PrimitiveVectorBuilder<u64>;
pub type Int8VectorBuilder = PrimitiveVectorBuilder<i8>;
pub type Int16VectorBuilder = PrimitiveVectorBuilder<i16>;
pub type Int32VectorBuilder = PrimitiveVectorBuilder<i32>;
pub type Int64VectorBuilder = PrimitiveVectorBuilder<i64>;
pub type Float32VectorBuilder = PrimitiveVectorBuilder<f32>;
pub type Float64VectorBuilder = PrimitiveVectorBuilder<f64>;
impl<T: PrimitiveElement> MutableVector for PrimitiveVectorBuilder<T> {
fn data_type(&self) -> ConcreteDataType {
T::build_data_type()
}
fn len(&self) -> usize {
self.mutable_array.len()
}
fn as_any(&self) -> &dyn Any {
self
}
fn as_mut_any(&mut self) -> &mut dyn Any {
self
}
fn to_vector(&mut self) -> VectorRef {
Arc::new(self.finish())
}
fn push_value_ref(&mut self, value: ValueRef) -> Result<()> {
let primitive = T::cast_value_ref(value)?;
self.mutable_array.push(primitive);
Ok(())
}
fn extend_slice_of(&mut self, vector: &dyn Vector, offset: usize, length: usize) -> Result<()> {
let primitive = T::cast_vector(vector)?;
// Slice the underlying array to avoid creating a new Arc.
let slice = primitive.slice(offset, length);
self.mutable_array.extend_trusted_len(slice.iter());
Ok(())
}
}
impl<T> ScalarVectorBuilder for PrimitiveVectorBuilder<T>
where
T: Scalar<VectorType = PrimitiveVector<T>> + PrimitiveElement,
for<'a> T: ScalarRef<'a, ScalarType = T, VectorType = PrimitiveVector<T>>,
for<'a> T: Scalar<RefType<'a> = T>,
{
type VectorType = PrimitiveVector<T>;
fn with_capacity(capacity: usize) -> Self {
Self {
mutable_array: MutablePrimitiveArray::with_capacity(capacity),
}
}
fn push(&mut self, value: Option<<Self::VectorType as ScalarVector>::RefItem<'_>>) {
self.mutable_array.push(value);
}
fn finish(&mut self) -> Self::VectorType {
PrimitiveVector {
array: std::mem::take(&mut self.mutable_array).into(),
}
}
}
impl<T: PrimitiveElement> PrimitiveVectorBuilder<T> {
fn with_type_capacity(data_type: ConcreteDataType, capacity: usize) -> Self {
Self {
mutable_array: MutablePrimitiveArray::with_capacity_from(
capacity,
data_type.as_arrow_type(),
),
}
}
}
pub(crate) fn replicate_primitive<T: PrimitiveElement>(
vector: &PrimitiveVector<T>,
offsets: &[usize],
) -> VectorRef {
Arc::new(replicate_primitive_with_type(
vector,
offsets,
T::build_data_type(),
))
}
pub(crate) fn replicate_primitive_with_type<T: PrimitiveElement>(
vector: &PrimitiveVector<T>,
offsets: &[usize],
data_type: ConcreteDataType,
) -> PrimitiveVector<T> {
assert_eq!(offsets.len(), vector.len());
if offsets.is_empty() {
return vector.slice(0, 0);
}
let mut builder = PrimitiveVectorBuilder::<T>::with_type_capacity(
data_type,
*offsets.last().unwrap() as usize,
);
let mut previous_offset = 0;
for (offset, value) in offsets.iter().zip(vector.array.iter()) {
let repeat_times = *offset - previous_offset;
match value {
Some(data) => {
builder.mutable_array.extend_trusted_len(
std::iter::repeat(*data)
.take(repeat_times)
.map(Option::Some),
);
}
None => {
builder.mutable_array.extend_constant(repeat_times, None);
}
}
previous_offset = *offset;
}
builder.finish()
}
#[cfg(test)]
mod tests {
use arrow::datatypes::DataType as ArrowDataType;
use serde_json;
use super::*;
use crate::data_type::DataType;
use crate::serialize::Serializable;
use crate::types::Int64Type;
fn check_vec(v: PrimitiveVector<i32>) {
assert_eq!(4, v.len());
assert_eq!("Int32Vector", v.vector_type_name());
assert!(!v.is_const());
assert_eq!(Validity::AllValid, v.validity());
assert!(!v.only_null());
for i in 0..4 {
assert!(!v.is_null(i));
assert_eq!(Value::Int32(i as i32 + 1), v.get(i));
assert_eq!(ValueRef::Int32(i as i32 + 1), v.get_ref(i));
}
let json_value = v.serialize_to_json().unwrap();
assert_eq!("[1,2,3,4]", serde_json::to_string(&json_value).unwrap(),);
let arrow_arr = v.to_arrow_array();
assert_eq!(4, arrow_arr.len());
assert_eq!(&ArrowDataType::Int32, arrow_arr.data_type());
}
#[test]
fn test_from_values() {
let v = PrimitiveVector::<i32>::from_values(vec![1, 2, 3, 4]);
check_vec(v);
}
#[test]
fn test_from_vec() {
let v = PrimitiveVector::<i32>::from_vec(vec![1, 2, 3, 4]);
check_vec(v);
}
#[test]
fn test_from_slice() {
let v = PrimitiveVector::<i32>::from_slice(vec![1, 2, 3, 4]);
check_vec(v);
}
#[test]
fn test_serialize_primitive_vector_with_null_to_json() {
let input = [Some(1i32), Some(2i32), None, Some(4i32), None];
let mut builder = PrimitiveVectorBuilder::with_capacity(input.len());
for v in input {
builder.push(v);
}
let vector = builder.finish();
let json_value = vector.serialize_to_json().unwrap();
assert_eq!(
"[1,2,null,4,null]",
serde_json::to_string(&json_value).unwrap(),
);
}
#[test]
fn test_from_arrow_array() {
let arrow_array = PrimitiveArray::from_slice(vec![1, 2, 3, 4]);
let v = PrimitiveVector::from(arrow_array);
check_vec(v);
}
#[test]
fn test_primitive_vector_build_get() {
let input = [Some(1i32), Some(2i32), None, Some(4i32), None];
let mut builder = PrimitiveVectorBuilder::with_capacity(input.len());
for v in input {
builder.push(v);
}
let vector = builder.finish();
assert_eq!(input.len(), vector.len());
for (i, v) in input.into_iter().enumerate() {
assert_eq!(v, vector.get_data(i));
assert_eq!(Value::from(v), vector.get(i));
}
let res: Vec<_> = vector.iter_data().collect();
assert_eq!(input, &res[..]);
}
#[test]
fn test_primitive_vector_validity() {
let input = [Some(1i32), Some(2i32), None, None];
let mut builder = PrimitiveVectorBuilder::with_capacity(input.len());
for v in input {
builder.push(v);
}
let vector = builder.finish();
assert_eq!(2, vector.null_count());
let validity = vector.validity();
let slots = validity.slots().unwrap();
assert_eq!(2, slots.null_count());
assert!(!slots.get_bit(2));
assert!(!slots.get_bit(3));
let vector = PrimitiveVector::<i32>::from_slice(vec![1, 2, 3, 4]);
assert_eq!(0, vector.null_count());
assert_eq!(Validity::AllValid, vector.validity());
}
#[test]
fn test_memory_size() {
let v = PrimitiveVector::<i32>::from_slice((0..5).collect::<Vec<i32>>());
assert_eq!(20, v.memory_size());
let v = PrimitiveVector::<i64>::from(vec![Some(0i64), Some(1i64), Some(2i64), None, None]);
assert_eq!(40, v.memory_size());
}
#[test]
fn test_primitive_vector_builder() {
let mut builder = Int64Type::default().create_mutable_vector(3);
builder.push_value_ref(ValueRef::Int64(123)).unwrap();
assert!(builder.push_value_ref(ValueRef::Int32(123)).is_err());
let input = Int64Vector::from_slice(&[7, 8, 9]);
builder.extend_slice_of(&input, 1, 2).unwrap();
assert!(builder
.extend_slice_of(&Int32Vector::from_slice(&[13]), 0, 1)
.is_err());
let vector = builder.to_vector();
let expect: VectorRef = Arc::new(Int64Vector::from_slice(&[123, 8, 9]));
assert_eq!(expect, vector);
}
}

View File

@@ -1,327 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use std::sync::Arc;
use arrow::array::{Array, ArrayRef, MutableArray, Utf8ValuesIter};
use arrow::bitmap::utils::ZipValidity;
use serde_json::Value as JsonValue;
use snafu::{OptionExt, ResultExt};
use crate::arrow_array::{MutableStringArray, StringArray};
use crate::data_type::ConcreteDataType;
use crate::error::{Result, SerializeSnafu};
use crate::scalars::{ScalarVector, ScalarVectorBuilder};
use crate::serialize::Serializable;
use crate::types::StringType;
use crate::value::{Value, ValueRef};
use crate::vectors::{self, MutableVector, Validity, Vector, VectorRef};
/// String array wrapper
#[derive(Debug, Clone, PartialEq)]
pub struct StringVector {
array: StringArray,
}
impl StringVector {
pub(crate) fn as_arrow(&self) -> &dyn Array {
&self.array
}
}
impl From<StringArray> for StringVector {
fn from(array: StringArray) -> Self {
Self { array }
}
}
impl From<Vec<Option<String>>> for StringVector {
fn from(data: Vec<Option<String>>) -> Self {
Self {
array: StringArray::from(data),
}
}
}
impl From<Vec<String>> for StringVector {
fn from(data: Vec<String>) -> Self {
Self {
array: StringArray::from(
data.into_iter()
.map(Option::Some)
.collect::<Vec<Option<String>>>(),
),
}
}
}
impl From<Vec<Option<&str>>> for StringVector {
fn from(data: Vec<Option<&str>>) -> Self {
Self {
array: StringArray::from(data),
}
}
}
impl From<Vec<&str>> for StringVector {
fn from(data: Vec<&str>) -> Self {
Self {
array: StringArray::from(
data.into_iter()
.map(Option::Some)
.collect::<Vec<Option<&str>>>(),
),
}
}
}
impl Vector for StringVector {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::String(StringType::default())
}
fn vector_type_name(&self) -> String {
"StringVector".to_string()
}
fn as_any(&self) -> &dyn Any {
self
}
fn len(&self) -> usize {
self.array.len()
}
fn to_arrow_array(&self) -> ArrayRef {
Arc::new(self.array.clone())
}
fn to_boxed_arrow_array(&self) -> Box<dyn Array> {
Box::new(self.array.clone())
}
fn validity(&self) -> Validity {
vectors::impl_validity_for_vector!(self.array)
}
fn memory_size(&self) -> usize {
self.len() * std::mem::size_of::<i64>() + self.array.values().len()
}
fn is_null(&self, row: usize) -> bool {
self.array.is_null(row)
}
fn slice(&self, offset: usize, length: usize) -> VectorRef {
Arc::new(Self::from(self.array.slice(offset, length)))
}
fn get(&self, index: usize) -> Value {
vectors::impl_get_for_vector!(self.array, index)
}
fn get_ref(&self, index: usize) -> ValueRef {
vectors::impl_get_ref_for_vector!(self.array, index)
}
}
impl ScalarVector for StringVector {
type OwnedItem = String;
type RefItem<'a> = &'a str;
type Iter<'a> = ZipValidity<'a, &'a str, Utf8ValuesIter<'a, i32>>;
type Builder = StringVectorBuilder;
fn get_data(&self, idx: usize) -> Option<Self::RefItem<'_>> {
if self.array.is_valid(idx) {
Some(self.array.value(idx))
} else {
None
}
}
fn iter_data(&self) -> Self::Iter<'_> {
self.array.iter()
}
}
pub struct StringVectorBuilder {
buffer: MutableStringArray,
}
impl MutableVector for StringVectorBuilder {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::string_datatype()
}
fn len(&self) -> usize {
self.buffer.len()
}
fn as_any(&self) -> &dyn Any {
self
}
fn as_mut_any(&mut self) -> &mut dyn Any {
self
}
fn to_vector(&mut self) -> VectorRef {
Arc::new(self.finish())
}
fn push_value_ref(&mut self, value: ValueRef) -> Result<()> {
self.buffer.push(value.as_string()?);
Ok(())
}
fn extend_slice_of(&mut self, vector: &dyn Vector, offset: usize, length: usize) -> Result<()> {
vectors::impl_extend_for_builder!(self.buffer, vector, StringVector, offset, length)
}
}
impl ScalarVectorBuilder for StringVectorBuilder {
type VectorType = StringVector;
fn with_capacity(capacity: usize) -> Self {
Self {
buffer: MutableStringArray::with_capacity(capacity),
}
}
fn push(&mut self, value: Option<<Self::VectorType as ScalarVector>::RefItem<'_>>) {
self.buffer.push(value)
}
fn finish(&mut self) -> Self::VectorType {
Self::VectorType {
array: std::mem::take(&mut self.buffer).into(),
}
}
}
impl Serializable for StringVector {
fn serialize_to_json(&self) -> crate::error::Result<Vec<JsonValue>> {
self.iter_data()
.map(|v| match v {
None => Ok(serde_json::Value::Null),
Some(s) => serde_json::to_value(s),
})
.collect::<serde_json::Result<_>>()
.context(SerializeSnafu)
}
}
vectors::impl_try_from_arrow_array_for_vector!(StringArray, StringVector);
#[cfg(test)]
mod tests {
use arrow::datatypes::DataType as ArrowDataType;
use serde_json;
use super::*;
use crate::data_type::DataType;
#[test]
fn test_string_vector_misc() {
let strs = vec!["hello", "greptime", "rust"];
let v = StringVector::from(strs.clone());
assert_eq!(3, v.len());
assert_eq!("StringVector", v.vector_type_name());
assert!(!v.is_const());
assert_eq!(Validity::AllValid, v.validity());
assert!(!v.only_null());
assert_eq!(41, v.memory_size());
for (i, s) in strs.iter().enumerate() {
assert_eq!(Value::from(*s), v.get(i));
assert_eq!(ValueRef::from(*s), v.get_ref(i));
assert_eq!(Value::from(*s), v.try_get(i).unwrap());
}
let arrow_arr = v.to_arrow_array();
assert_eq!(3, arrow_arr.len());
assert_eq!(&ArrowDataType::Utf8, arrow_arr.data_type());
}
#[test]
fn test_serialize_string_vector() {
let mut builder = StringVectorBuilder::with_capacity(3);
builder.push(Some("hello"));
builder.push(None);
builder.push(Some("world"));
let string_vector = builder.finish();
let serialized =
serde_json::to_string(&string_vector.serialize_to_json().unwrap()).unwrap();
assert_eq!(r#"["hello",null,"world"]"#, serialized);
}
#[test]
fn test_from_arrow_array() {
let mut builder = MutableStringArray::new();
builder.push(Some("A"));
builder.push(Some("B"));
builder.push::<&str>(None);
builder.push(Some("D"));
let string_array: StringArray = builder.into();
let vector = StringVector::from(string_array);
assert_eq!(
r#"["A","B",null,"D"]"#,
serde_json::to_string(&vector.serialize_to_json().unwrap()).unwrap(),
);
}
#[test]
fn test_string_vector_build_get() {
let mut builder = StringVectorBuilder::with_capacity(4);
builder.push(Some("hello"));
builder.push(None);
builder.push(Some("world"));
let vector = builder.finish();
assert_eq!(Some("hello"), vector.get_data(0));
assert_eq!(None, vector.get_data(1));
assert_eq!(Some("world"), vector.get_data(2));
// Get out of bound
assert!(vector.try_get(3).is_err());
assert_eq!(Value::String("hello".into()), vector.get(0));
assert_eq!(Value::Null, vector.get(1));
assert_eq!(Value::String("world".into()), vector.get(2));
let mut iter = vector.iter_data();
assert_eq!("hello", iter.next().unwrap().unwrap());
assert_eq!(None, iter.next().unwrap());
assert_eq!("world", iter.next().unwrap().unwrap());
assert_eq!(None, iter.next());
}
#[test]
fn test_string_vector_builder() {
let mut builder = StringType::default().create_mutable_vector(3);
builder.push_value_ref(ValueRef::String("hello")).unwrap();
assert!(builder.push_value_ref(ValueRef::Int32(123)).is_err());
let input = StringVector::from_slice(&["world", "one", "two"]);
builder.extend_slice_of(&input, 1, 2).unwrap();
assert!(builder
.extend_slice_of(&crate::vectors::Int32Vector::from_slice(&[13]), 0, 1)
.is_err());
let vector = builder.to_vector();
let expect: VectorRef = Arc::new(StringVector::from_slice(&["hello", "one", "two"]));
assert_eq!(expect, vector);
}
}

View File

@@ -1,319 +0,0 @@
// Copyright 2022 Greptime Team
//
// 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 std::any::Any;
use std::sync::Arc;
use arrow::array::{Array, ArrayRef, PrimitiveArray};
use common_time::timestamp::{TimeUnit, Timestamp};
use snafu::OptionExt;
use crate::data_type::{ConcreteDataType, DataType};
use crate::error;
use crate::error::Result;
use crate::prelude::{
MutableVector, ScalarVector, ScalarVectorBuilder, Validity, Value, ValueRef, Vector, VectorRef,
};
use crate::serialize::Serializable;
use crate::types::TimestampType;
use crate::vectors::{PrimitiveIter, PrimitiveVector, PrimitiveVectorBuilder};
/// `TimestampVector` stores timestamp in millisecond since UNIX Epoch.
#[derive(Debug, Clone, PartialEq)]
pub struct TimestampVector {
array: PrimitiveVector<i64>,
}
impl TimestampVector {
pub fn new(array: PrimitiveArray<i64>) -> Self {
Self {
array: PrimitiveVector { array },
}
}
pub fn try_from_arrow_array(array: impl AsRef<dyn Array>) -> Result<Self> {
Ok(Self::new(
array
.as_ref()
.as_any()
.downcast_ref::<PrimitiveArray<i64>>()
.with_context(|| error::ConversionSnafu {
from: format!("{:?}", array.as_ref().data_type()),
})?
.clone(),
))
}
pub fn from_values<I: IntoIterator<Item = i64>>(iter: I) -> Self {
Self {
array: PrimitiveVector {
array: PrimitiveArray::from_values(iter),
},
}
}
pub(crate) fn as_arrow(&self) -> &dyn Array {
self.array.as_arrow()
}
}
impl Vector for TimestampVector {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::timestamp_millis_datatype()
}
fn vector_type_name(&self) -> String {
"TimestampVector".to_string()
}
fn as_any(&self) -> &dyn Any {
self
}
fn len(&self) -> usize {
self.array.len()
}
fn to_arrow_array(&self) -> ArrayRef {
let validity = self.array.array.validity().cloned();
let buffer = self.array.array.values().clone();
Arc::new(PrimitiveArray::new(
TimestampType::new(TimeUnit::Millisecond).as_arrow_type(),
buffer,
validity,
))
}
fn to_boxed_arrow_array(&self) -> Box<dyn Array> {
let validity = self.array.array.validity().cloned();
let values = self.array.array.values().clone();
Box::new(PrimitiveArray::new(
arrow::datatypes::DataType::Timestamp(arrow::datatypes::TimeUnit::Millisecond, None),
values,
validity,
))
}
fn validity(&self) -> Validity {
self.array.validity()
}
fn memory_size(&self) -> usize {
self.array.memory_size()
}
fn is_null(&self, row: usize) -> bool {
self.array.is_null(row)
}
fn slice(&self, offset: usize, length: usize) -> VectorRef {
Arc::new(Self {
array: PrimitiveVector {
array: self.array.array.slice(offset, length),
},
})
}
fn get(&self, index: usize) -> Value {
match self.array.get(index) {
Value::Null => Value::Null,
Value::Int64(v) => Value::Timestamp(Timestamp::from_millis(v)),
_ => {
unreachable!()
}
}
}
fn get_ref(&self, index: usize) -> ValueRef {
match self.array.get(index) {
Value::Int64(v) => ValueRef::Timestamp(Timestamp::from_millis(v)),
Value::Null => ValueRef::Null,
_ => unreachable!(),
}
}
}
impl Serializable for TimestampVector {
fn serialize_to_json(&self) -> Result<Vec<serde_json::Value>> {
Ok(self
.array
.iter_data()
.map(|v| match v {
None => serde_json::Value::Null,
Some(v) => v.into(),
})
.collect::<Vec<_>>())
}
}
impl ScalarVector for TimestampVector {
type OwnedItem = Timestamp;
type RefItem<'a> = Timestamp;
type Iter<'a> = TimestampDataIter<'a>;
type Builder = TimestampVectorBuilder;
fn get_data(&self, idx: usize) -> Option<Self::RefItem<'_>> {
self.array.get_data(idx).map(Timestamp::from_millis)
}
fn iter_data(&self) -> Self::Iter<'_> {
TimestampDataIter {
iter: self.array.iter_data(),
}
}
}
pub struct TimestampDataIter<'a> {
iter: PrimitiveIter<'a, i64>,
}
impl<'a> Iterator for TimestampDataIter<'a> {
type Item = Option<Timestamp>;
fn next(&mut self) -> Option<Self::Item> {
self.iter.next().map(|v| v.map(Timestamp::from_millis))
}
}
pub struct TimestampVectorBuilder {
buffer: PrimitiveVectorBuilder<i64>,
}
impl MutableVector for TimestampVectorBuilder {
fn data_type(&self) -> ConcreteDataType {
ConcreteDataType::timestamp_millisecond_datatype()
}
fn len(&self) -> usize {
self.buffer.len()
}
fn as_any(&self) -> &dyn Any {
self
}
fn as_mut_any(&mut self) -> &mut dyn Any {
self
}
fn to_vector(&mut self) -> VectorRef {
Arc::new(self.finish())
}
fn push_value_ref(&mut self, value: ValueRef) -> Result<()> {
// TODO(hl): vector and vector builder should also support customized time unit.
self.buffer.push(
value
.as_timestamp()?
.map(|t| t.convert_to(TimeUnit::Millisecond)),
);
Ok(())
}
fn extend_slice_of(&mut self, vector: &dyn Vector, offset: usize, length: usize) -> Result<()> {
let concrete_vector = vector
.as_any()
.downcast_ref::<TimestampVector>()
.with_context(|| error::CastTypeSnafu {
msg: format!(
"Failed to convert vector from {} to DateVector",
vector.vector_type_name()
),
})?;
self.buffer
.extend_slice_of(&concrete_vector.array, offset, length)?;
Ok(())
}
}
impl ScalarVectorBuilder for TimestampVectorBuilder {
type VectorType = TimestampVector;
fn with_capacity(capacity: usize) -> Self {
Self {
buffer: PrimitiveVectorBuilder::with_capacity(capacity),
}
}
/// Pushes a Timestamp value into vector builder. The timestamp must be with time unit
/// `Second`/`MilliSecond`/`Microsecond`.
fn push(&mut self, value: Option<<Self::VectorType as ScalarVector>::RefItem<'_>>) {
self.buffer
.push(value.map(|v| v.convert_to(TimeUnit::Millisecond)));
}
fn finish(&mut self) -> Self::VectorType {
Self::VectorType {
array: self.buffer.finish(),
}
}
}
pub(crate) fn replicate_timestamp(vector: &TimestampVector, offsets: &[usize]) -> VectorRef {
let array = crate::vectors::primitive::replicate_primitive_with_type(
&vector.array,
offsets,
vector.data_type(),
);
Arc::new(TimestampVector { array })
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
pub fn test_build_timestamp_vector() {
let mut builder = TimestampVectorBuilder::with_capacity(3);
builder.push(Some(Timestamp::new(1, TimeUnit::Second)));
builder.push(None);
builder.push(Some(Timestamp::new(2, TimeUnit::Millisecond)));
let vector = builder.finish();
assert_eq!(
ConcreteDataType::timestamp_millisecond_datatype(),
vector.data_type()
);
assert_eq!(3, vector.len());
assert_eq!(
Value::Timestamp(Timestamp::new(1000, TimeUnit::Millisecond)),
vector.get(0)
);
assert_eq!(Value::Null, vector.get(1));
assert_eq!(
Value::Timestamp(Timestamp::new(2, TimeUnit::Millisecond)),
vector.get(2)
);
assert_eq!(
vec![
Some(Timestamp::new(1000, TimeUnit::Millisecond)),
None,
Some(Timestamp::new(2, TimeUnit::Millisecond)),
],
vector.iter_data().collect::<Vec<_>>()
);
}
#[test]
fn test_timestamp_from_arrow() {
let vector =
TimestampVector::from_slice(&[Timestamp::from_millis(1), Timestamp::from_millis(2)]);
let arrow = vector.as_arrow().slice(0, vector.len());
let vector2 = TimestampVector::try_from_arrow_array(&arrow).unwrap();
assert_eq!(vector, vector2);
}
}