diff --git a/core/src/common/mod.rs b/core/src/common/mod.rs index 8d5030c02..1b7dfad28 100644 --- a/core/src/common/mod.rs +++ b/core/src/common/mod.rs @@ -35,10 +35,5 @@ trait ValueSetter { fn append_value(&mut self, v: &T::Native); } -mod vector; - mod buffer; pub use buffer::*; - -mod mutable_vector; -pub use mutable_vector::*; diff --git a/core/src/common/mutable_vector.rs b/core/src/common/mutable_vector.rs deleted file mode 100644 index ba29fc01a..000000000 --- a/core/src/common/mutable_vector.rs +++ /dev/null @@ -1,409 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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::ArrayData, buffer::Buffer as ArrowBuffer, datatypes::DataType as ArrowDataType, -}; - -use crate::{ - common::{bit, CometBuffer, ValueGetter, ValueSetter}, - BinaryType, StringType, StringView, TypeTrait, -}; - -const DEFAULT_ARRAY_LEN: usize = 4; - -/// A mutable vector that can be re-used across batches. -#[derive(Debug)] -pub struct MutableVector { - /// The Arrow type for the elements of this vector. - pub(crate) arrow_type: ArrowDataType, - - /// The number of total elements in this vector. - pub(crate) num_values: usize, - - /// The number of null elements in this vector, must <= `num_values`. - pub(crate) num_nulls: usize, - - /// The capacity of the vector - pub(crate) capacity: usize, - - /// How many bits are required to store a single value - pub(crate) bit_width: usize, - - /// The validity buffer of this Arrow vector. A bit set at position `i` indicates the `i`th - /// element is not null. Otherwise, an unset bit at position `i` indicates the `i`th element is - /// null. - pub(crate) validity_buffer: CometBuffer, - - /// The value buffer of this Arrow vector. This could store either offsets if the vector - /// is of list or struct type, or actual values themselves otherwise. - pub(crate) value_buffer: CometBuffer, - - /// Child vectors for non-primitive types (e.g., list, struct). - pub(crate) children: Vec, - - /// Dictionary (i.e., values) associated with this vector. Only set if using dictionary - /// encoding. - pub(crate) dictionary: Option>, - - /// Whether all the values in the vector are not null. - /// - /// This is useful so we can skip setting non-null for each individual value, in the - /// `validity_buffer`. - all_non_null: bool, -} - -impl MutableVector { - pub fn new(capacity: usize, arrow_type: &ArrowDataType) -> Self { - let bit_width = Self::bit_width(arrow_type); - Self::new_with_bit_width(capacity, arrow_type.clone(), bit_width) - } - - pub fn new_with_bit_width( - capacity: usize, - arrow_type: ArrowDataType, - bit_width: usize, - ) -> Self { - let validity_len = bit::ceil(capacity, 8); - let validity_buffer = CometBuffer::new(validity_len); - - let mut value_capacity = capacity; - if Self::is_binary_type(&arrow_type) { - // Arrow offset array needs to have one extra slot - value_capacity += 1; - } - // Make sure the capacity is positive - let len = bit::ceil(value_capacity * bit_width, 8); - let mut value_buffer = CometBuffer::new(len); - - let mut children = Vec::new(); - - match arrow_type { - ArrowDataType::Binary | ArrowDataType::Utf8 => { - children.push(MutableVector::new_with_bit_width( - capacity, - ArrowDataType::Int8, - DEFAULT_ARRAY_LEN * 8, - )); - } - _ => {} - } - - if Self::is_binary_type(&arrow_type) { - // Setup the first offset which is always 0. - let zero: u32 = 0; - bit::memcpy_value(&zero, 4, &mut value_buffer); - } - - Self { - arrow_type, - num_values: 0, - num_nulls: 0, - capacity, - bit_width, - validity_buffer, - value_buffer, - children, - dictionary: None, - all_non_null: false, - } - } - - /// Appends a non-null value `v` to the end of this vector. - #[inline] - pub fn append_value(&mut self, v: &T::Native) { - >::append_value(self, v); - } - - /// Gets the non-null value at `idx` of this vector. - #[inline] - pub fn value(&self, idx: usize) -> T::Native { - >::value(self, idx) - } - - /// Whether the given value at `idx` of this vector is null. - #[inline] - pub fn is_null(&self, idx: usize) -> bool { - unsafe { !bit::get_bit_raw(self.validity_buffer.as_ptr(), idx) } - } - - /// Resets this vector to the initial state. - #[inline] - pub fn reset(&mut self) { - self.num_values = 0; - self.num_nulls = 0; - self.all_non_null = false; - self.validity_buffer.reset(); - if Self::is_binary_type(&self.arrow_type) { - // Reset the first offset to 0 - let zero: u32 = 0; - bit::memcpy_value(&zero, 4, &mut self.value_buffer); - // Also reset the child value vector - let child = &mut self.children[0]; - child.reset(); - } else if Self::should_reset_value_buffer(&self.arrow_type) { - self.value_buffer.reset(); - } - } - - /// Appends a new null value to the end of this vector. - #[inline] - pub fn put_null(&mut self) { - self.put_nulls(1) - } - - /// Appends `n` null values to the end of this vector. - #[inline] - pub fn put_nulls(&mut self, n: usize) { - // We need to update offset buffer for binary. - if Self::is_binary_type(&self.arrow_type) { - let mut offset = self.num_values * 4; - let prev_offset_value = bit::read_num_bytes_u32(4, &self.value_buffer[offset..]); - offset += 4; - (0..n).for_each(|_| { - bit::memcpy_value(&prev_offset_value, 4, &mut self.value_buffer[offset..]); - offset += 4; - }); - } - - self.num_nulls += n; - self.num_values += n; - } - - /// Returns the number of total values (including both null and non-null) of this vector. - #[inline] - pub fn num_values(&self) -> usize { - self.num_values - } - - /// Returns the number of null values of this vector. - #[inline] - pub fn num_nulls(&self) -> usize { - self.num_nulls - } - - #[inline] - pub fn set_not_null(&mut self, i: usize) { - unsafe { - bit::set_bit_raw(self.validity_buffer.as_mut_ptr(), i); - } - } - - /// Sets all values in this vector to be non-null. - #[inline] - pub fn set_all_non_null(&mut self) { - self.all_non_null = true; - } - - /// Sets the content of validity buffer to be `buffer`. - pub fn set_validity_buffer(&mut self, buffer: &ArrowBuffer) { - self.validity_buffer = buffer.into(); - } - - /// Sets the content of value buffer to be `buffer`. - pub fn set_value_buffer(&mut self, buffer: &ArrowBuffer) { - self.value_buffer = buffer.into(); - } - - /// Sets the dictionary of this to be `dict`. - pub fn set_dictionary(&mut self, dict: MutableVector) { - self.dictionary = Some(Box::new(dict)) - } - - /// Clones this into an Arrow [`ArrayData`](arrow::array::ArrayData). Note that the caller of - /// this method MUST make sure the returned `ArrayData` won't live longer than this vector - /// itself. Otherwise, dangling pointer may happen. - /// - /// # Safety - /// - /// This method is highly unsafe since it calls `to_immutable` which leaks raw pointer to the - /// memory region that are tracked by `ArrowMutableBuffer`. Please see comments on - /// `to_immutable` buffer to understand the motivation. - pub fn get_array_data(&mut self) -> ArrayData { - unsafe { - let data_type = if let Some(d) = &self.dictionary { - ArrowDataType::Dictionary( - Box::new(ArrowDataType::Int32), - Box::new(d.arrow_type.clone()), - ) - } else { - self.arrow_type.clone() - }; - let mut builder = ArrayData::builder(data_type) - .len(self.num_values) - .add_buffer(self.value_buffer.to_arrow()); - - builder = if self.all_non_null { - builder.null_count(0) - } else { - builder - .null_bit_buffer(Some(self.validity_buffer.to_arrow())) - .null_count(self.num_nulls) - }; - - if Self::is_binary_type(&self.arrow_type) && self.dictionary.is_none() { - let child = &mut self.children[0]; - builder = builder.add_buffer(child.value_buffer.to_arrow()); - } - - if let Some(d) = &mut self.dictionary { - builder = builder.add_child_data(d.get_array_data()); - } - - builder.build_unchecked() - } - } - - /// Returns the number of bits it takes to store one element of `arrow_type` in the value buffer - /// of this vector. - pub fn bit_width(arrow_type: &ArrowDataType) -> usize { - match arrow_type { - ArrowDataType::Boolean => 1, - ArrowDataType::Int8 => 8, - ArrowDataType::Int16 => 16, - ArrowDataType::Int32 | ArrowDataType::Float32 | ArrowDataType::Date32 => 32, - ArrowDataType::Int64 | ArrowDataType::Float64 | ArrowDataType::Timestamp(_, _) => 64, - ArrowDataType::FixedSizeBinary(type_length) => *type_length as usize * 8, - ArrowDataType::Decimal128(..) => 128, // Arrow stores decimal with 16 bytes - ArrowDataType::Binary | ArrowDataType::Utf8 => 32, // Only count offset size - dt => panic!("Unsupported Arrow data type: {:?}", dt), - } - } - - #[inline] - fn is_binary_type(dt: &ArrowDataType) -> bool { - matches!(dt, ArrowDataType::Binary | ArrowDataType::Utf8) - } - - #[inline] - fn should_reset_value_buffer(dt: &ArrowDataType) -> bool { - // - Boolean type expects have a zeroed value buffer - // - Decimal may pad buffer with 0xff so we need to clear them before a new batch - matches!(dt, ArrowDataType::Boolean | ArrowDataType::Decimal128(_, _)) - } - - /// Creates an immutable reference from a mutable Arrow buffer `buf`. - /// - /// # Safety - /// - /// This function is highly unsafe. Please see documentation of the [`to_arrow`] method for - /// details. - #[inline] - unsafe fn to_immutable(buf: &CometBuffer) -> ArrowBuffer { - buf.to_arrow() - } -} - -impl ValueGetter for MutableVector { - default fn value(&self, idx: usize) -> T::Native { - unsafe { - let ptr = self.value_buffer.as_ptr() as *const T::Native; - *ptr.add(idx) - } - } -} - -impl ValueGetter for MutableVector { - fn value(&self, _: usize) -> StringView { - unimplemented!("'value' on StringType is currently unsupported"); - } -} - -impl ValueGetter for MutableVector { - fn value(&self, _: usize) -> StringView { - unimplemented!("'value' on BinaryType is currently unsupported"); - } -} - -impl ValueSetter for MutableVector { - default fn append_value(&mut self, v: &T::Native) { - unsafe { - let ptr = self.value_buffer.as_mut_ptr() as *mut T::Native; - *ptr.add(self.num_values) = *v; - } - self.num_values += 1; - } -} - -impl ValueSetter for MutableVector { - fn append_value(&mut self, _: &StringView) { - unimplemented!("'append_value' on StringType is currently unsupported"); - } -} - -impl ValueSetter for MutableVector { - fn append_value(&mut self, _: &StringView) { - unimplemented!("'append_value' on BinaryType is currently unsupported"); - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::IntegerType; - - #[test] - fn set_and_get() { - let mut mv = MutableVector::new(1024, &ArrowDataType::Int32); - - for n in 0..100 { - mv.append_value::(&(n * n)); - } - assert_eq!(mv.num_values(), 100); - assert_eq!(mv.num_nulls(), 0); - - for n in 0..100 { - assert_eq!(mv.value::(n) as usize, n * n); - } - - mv.reset(); - - for n in 0..200 { - if n % 2 == 0 { - mv.put_null(); - } else { - mv.append_value::(&(n * 2)); - } - } - - assert_eq!(mv.num_values(), 200); - assert_eq!(mv.num_nulls(), 100); - - for n in 0..200 { - if n % 2 == 0 { - assert!(mv.is_null(n)); - } else { - assert_eq!(mv.value::(n) as usize, n * 2); - } - } - } - - #[test] - #[should_panic] - fn set_string_unsupported() { - let mut mv = MutableVector::new(1024, &ArrowDataType::Utf8); - let sv = StringView::default(); - mv.append_value::(&sv); - } - - #[test] - #[should_panic] - fn get_string_unsupported() { - let mv = MutableVector::new(1024, &ArrowDataType::Utf8); - mv.value::(0); - } -} diff --git a/core/src/common/vector.rs b/core/src/common/vector.rs deleted file mode 100644 index 1afb1e78f..000000000 --- a/core/src/common/vector.rs +++ /dev/null @@ -1,523 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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::{ - common::{bit, ValueGetter}, - BoolType, DataType, TypeTrait, BITS_PER_BYTE, STRING_VIEW_LEN, STRING_VIEW_PREFIX_LEN, -}; -use arrow::{ - array::{Array, ArrayRef}, - buffer::{Buffer, MutableBuffer}, - datatypes::DataType as ArrowDataType, -}; -use arrow_data::ArrayData; - -/// A vector that holds elements of plain types (i.e., no nested type such as list, map, struct). -pub struct PlainVector { - /// The data type for elements in this vector - data_type: DataType, - /// Total number of values in this vector - num_values: usize, - /// Total number of nulls in this vector. Must <= `num_values`. - num_nulls: usize, - /// The value buffer - value_buffer: ValueBuffer, - /// Number of bytes for each element in the vector. For variable length types such as string - /// and binary, this will be the size of [`StringView`] which is always 16 bytes. - value_size: usize, - /// Offsets into buffers - offset: usize, - /// The validity buffer. If empty, all values in this vector are not null. - validity_buffer: Option, - /// Whether this vector is dictionary encoded - is_dictionary: bool, - /// Indices (or dictionary keys) when `is_dictionary` is true. Otherwise, this is always - /// an empty vector. - indices: IndexBuffer, -} - -impl ValueGetter for PlainVector { - default fn value(&self, idx: usize) -> T::Native { - let offset = self.offset(idx); - unsafe { - let ptr = self.value_buffer.as_ptr() as *const T::Native; - *ptr.add(offset) - } - } -} - -impl ValueGetter for PlainVector { - fn value(&self, idx: usize) -> bool { - let offset = self.offset(idx); - unsafe { bit::get_bit_raw(self.value_buffer.as_ptr(), offset) } - } -} - -impl PlainVector { - /// Returns the data type of this vector. - pub fn data_type(&self) -> &DataType { - &self.data_type - } - - /// Returns the total number of elements in this vector. - pub fn num_values(&self) -> usize { - self.num_values - } - - /// Returns the total number of nulls in this vector. - pub fn num_nulls(&self) -> usize { - self.num_nulls - } - - /// Whether there is any null in this vector. - pub fn has_null(&self) -> bool { - self.num_nulls > 0 - } - - /// Whether the element at `idx` is null. - pub fn is_null(&self, idx: usize) -> bool { - if let Some(validity_buffer) = &self.validity_buffer { - unsafe { - return !bit::get_bit_raw(validity_buffer.as_ptr(), self.offset + idx); - } - } - - false - } - - #[inline(always)] - pub fn value(&self, idx: usize) -> T::Native { - >::value(self, idx) - } - - #[inline(always)] - fn offset(&self, idx: usize) -> usize { - let idx = self.offset + idx; - if self.is_dictionary { - self.indices.get(idx) - } else { - idx - } - } -} - -impl From for PlainVector { - fn from(data: ArrayData) -> Self { - assert!(!data.buffers().is_empty(), "expected at least one buffer"); - let arrow_dt = data.data_type(); - let dt: DataType = arrow_dt.into(); - let is_dictionary = matches!(arrow_dt, ArrowDataType::Dictionary(_, _)); - - let mut value_buffers = data.buffers(); - let mut indices = IndexBuffer::empty(); - let validity_buffer = data.nulls().map(|nb| nb.buffer().clone()); - - if is_dictionary { - // in case of dictionary data, the dictionary values are stored in child data, while - // dictionary keys are stored in `value_buffer`. - assert_eq!( - data.child_data().len(), - 1, - "child data should contain a single array" - ); - let child_data = &data.child_data()[0]; - indices = IndexBuffer::new(value_buffers[0].clone(), data.len() + data.offset()); - value_buffers = child_data.buffers(); - } - - let value_size = dt.kind().type_size() / BITS_PER_BYTE; - let value_buffer = ValueBuffer::new(&dt, value_buffers.to_vec(), data.len()); - - Self { - data_type: dt, - num_values: data.len(), - num_nulls: data.null_count(), - value_buffer, - value_size, - offset: data.offset(), - validity_buffer, - is_dictionary, - indices, - } - } -} - -impl From for PlainVector { - fn from(value: ArrayRef) -> Self { - Self::from(value.into_data()) - } -} - -struct ValueBuffer { - ptr: *const u8, - /// Keep the `ptr` alive - original_buffers: Vec, -} - -impl ValueBuffer { - pub fn new(dt: &DataType, buffers: Vec, len: usize) -> Self { - if matches!(dt, DataType::String | DataType::Binary) { - assert_eq!( - 2, - buffers.len(), - "expected two buffers (offset, value) for string/binary" - ); - - let mut string_view_buf = MutableBuffer::from_len_zeroed(len * 16); - let buf_mut = string_view_buf.as_mut_ptr(); - - let offsets = buffers[0].as_ptr() as *const i32; - let values = buffers[1].as_ptr(); - - let mut dst_offset = 0; - let mut start = 0; - unsafe { - for i in 0..len { - // StringView format: - // - length (4 bytes) - // - first 4 bytes of the string/binary (4 bytes) - // - pointer to the string/binary (8 bytes) - let end = *offsets.add(i + 1); - let len = end - start; - let value = values.add(start as usize); - *(buf_mut.add(dst_offset) as *mut i32) = len; - if len >= STRING_VIEW_PREFIX_LEN as i32 { - // only store prefix if the string has at least 4 bytes, otherwise, we'll - // zero pad the bytes. - std::ptr::copy_nonoverlapping( - value, - buf_mut.add(dst_offset + STRING_VIEW_PREFIX_LEN), - STRING_VIEW_PREFIX_LEN, - ); - } - *(buf_mut.add(dst_offset + STRING_VIEW_PREFIX_LEN + 4) as *mut usize) = - value as usize; - start = end; - dst_offset += STRING_VIEW_LEN; - } - } - - let string_buffer: Buffer = string_view_buf.into(); - let ptr = string_buffer.as_ptr(); - - Self { - ptr, - original_buffers: vec![string_buffer, buffers[1].clone()], - } - } else { - let ptr = buffers[0].as_ptr(); - Self { - ptr, - original_buffers: buffers, - } - } - } - - /// Returns the raw pointer for the data in this value buffer. - /// NOTE: caller of this should NOT store the raw pointer to avoid dangling pointers. - pub fn as_ptr(&self) -> *const u8 { - self.ptr - } -} - -struct IndexBuffer { - ptr: *const u8, - /// Keep the `ptr` alive. - buf: Option, - /// Total number of elements in the index buffer - len: usize, -} - -impl IndexBuffer { - pub fn new(buf: Buffer, len: usize) -> Self { - let ptr = buf.as_ptr(); - Self { - buf: Some(buf), - ptr, - len, - } - } - - pub fn empty() -> Self { - Self { - buf: None, - ptr: std::ptr::null(), - len: 0, - } - } - - #[inline] - pub fn get(&self, i: usize) -> usize { - debug_assert!(i < self.len); - unsafe { - let ptr = self.ptr as *const i32; - *ptr.add(i) as usize - } - } -} - -#[cfg(test)] -mod tests { - use crate::{ - BoolType, ByteType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, - NativeEqual, ShortType, StringType, TimestampType, TypeTrait, STRING_VIEW_PREFIX_LEN, - }; - - use crate::common::vector::PlainVector; - use arrow::{ - array::{ - Array, BooleanArray, Date32Array, Decimal128Array, Float32Array, Float64Array, - Int16Array, Int32Array, Int8Array, StringArray, - }, - buffer::Buffer, - datatypes::{DataType as ArrowDataType, ToByteSlice}, - }; - use arrow_array::TimestampMicrosecondArray; - use arrow_data::ArrayData; - - #[test] - fn primitive_no_null() { - let arr = Int32Array::from(vec![0, 1, 2, 3, 4]); - let vector = PlainVector::from(arr.into_data()); - - assert_eq!(5, vector.num_values()); - assert_eq!(0, vector.num_nulls()); - assert_eq!(4, vector.value_size); - assert!(vector.validity_buffer.is_none()); - - for i in 0..5 { - assert!(!vector.is_null(i)); - assert_eq!(i as i32, vector.value::(i)) - } - } - - fn check_answer(expected: &[Option], actual: &PlainVector) { - assert_eq!(expected.len(), actual.num_values()); - let nulls = expected - .iter() - .filter(|v| v.is_none()) - .collect::>>(); - assert_eq!(nulls.len(), actual.num_nulls()); - - for i in 0..expected.len() { - if let Some(v) = expected[i] { - assert!(!actual.is_null(i)); - assert!(v.is_equal(&actual.value::(i))); - } else { - assert!(actual.is_null(i)); - } - } - } - - #[test] - fn primitive_with_nulls() { - let data = vec![Some(0), None, Some(2), None, Some(4)]; - let arr = TimestampMicrosecondArray::from(data.clone()); - let vector = PlainVector::from(arr.into_data()); - - check_answer::(&data, &vector); - } - - #[test] - fn primitive_with_offsets_nulls() { - let arr = Int32Array::from(vec![Some(0), None, Some(2), None, Some(4), None, Some(7)]); - let data = arr.into_data(); - let vector = PlainVector::from(data.slice(2, 3)); - - assert_eq!(3, vector.num_values()); - assert_eq!(1, vector.num_nulls()); - - for i in 0..2 { - if i % 2 == 0 { - assert!(!vector.is_null(i)); - assert_eq!((i + 2) as i32, vector.value::(i)); - } else { - assert!(vector.is_null(i)); - } - } - } - - #[test] - fn primitive_dictionary() { - let value_data = ArrayData::builder(ArrowDataType::Int8) - .len(8) - .add_buffer(Buffer::from( - &[10_i8, 11, 12, 13, 14, 15, 16, 17].to_byte_slice(), - )) - .build() - .unwrap(); - - // Construct a buffer for value offsets, for the nested array: - let keys = Buffer::from(&[2_i32, 3, 4].to_byte_slice()); - - // Construct a dictionary array from the above two - let key_type = ArrowDataType::Int32; - let value_type = ArrowDataType::Int8; - let dict_data_type = ArrowDataType::Dictionary(Box::new(key_type), Box::new(value_type)); - let dict_data = ArrayData::builder(dict_data_type) - .len(3) - .add_buffer(keys) - .add_child_data(value_data) - .build() - .unwrap(); - - let vector = PlainVector::from(dict_data); - - assert_eq!(DataType::Byte, *vector.data_type()); - assert_eq!(3, vector.num_values()); - assert_eq!(0, vector.num_nulls()); - assert!(!vector.has_null()); - assert_eq!(12, vector.value::(0)); - assert_eq!(13, vector.value::(1)); - assert_eq!(14, vector.value::(2)); - } - - #[test] - fn bools() { - let data = vec![Some(true), None, Some(false), None, Some(true)]; - let arr = BooleanArray::from(data.clone()); - let vector = PlainVector::from(arr.into_data()); - - check_answer::(&data, &vector); - } - - #[test] - fn bytes() { - let data = vec![Some(4_i8), None, None, Some(5_i8), Some(7_i8)]; - let arr = Int8Array::from(data.clone()); - let vector = PlainVector::from(arr.into_data()); - - check_answer::(&data, &vector); - } - - #[test] - fn shorts() { - let data = vec![Some(4_i16), None, None, Some(-40_i16), Some(-3_i16)]; - let arr = Int16Array::from(data.clone()); - let vector = PlainVector::from(arr.into_data()); - - check_answer::(&data, &vector); - } - - #[test] - fn floats() { - let data = vec![ - Some(4.0_f32), - Some(-0.0_f32), - Some(-3.0_f32), - Some(0.0_f32), - Some(std::f32::consts::PI), - ]; - let arr = Float32Array::from(data.clone()); - let vector = PlainVector::from(arr.into_data()); - - check_answer::(&data, &vector); - } - - #[test] - fn doubles() { - let data = vec![ - None, - Some(std::f64::consts::PI), - Some(4.0_f64), - Some(f64::NAN), - ]; - let arr = Float64Array::from(data.clone()); - let vector = PlainVector::from(arr.into_data()); - - check_answer::(&data, &vector); - } - - #[test] - fn decimals() { - let data = vec![Some(1_i128), None, None, Some(i128::MAX)]; - let arr = Decimal128Array::from(data.clone()); - let vector = PlainVector::from(arr.into_data()); - - check_answer::(&data, &vector); - } - - #[test] - fn timestamps() { - // 1: 00:00:00.001 - // 37800005: 10:30:00.005 - // 86399210: 23:59:59.210 - let data = vec![Some(1), None, Some(37_800_005), Some(86_399_210)]; - let arr = TimestampMicrosecondArray::from(data.clone()); - let vector = PlainVector::from(arr.into_data()); - - check_answer::(&data, &vector); - } - - #[test] - fn dates() { - let data = vec![Some(100), None, Some(200), None]; - let arr = Date32Array::from(data.clone()); - let vector = PlainVector::from(arr.into_data()); - - check_answer::(&data, &vector); - } - - #[test] - fn string_no_nulls() { - let values: Vec<&str> = vec!["hello", "", "comet"]; - let arr = StringArray::from(values.clone()); - - let vector = PlainVector::from(arr.into_data()); - assert_eq!(3, vector.num_values()); - assert_eq!(0, vector.num_nulls()); - - for i in 0..values.len() { - let expected = values[i]; - let actual = vector.value::(i); - assert_eq!(expected.len(), actual.len as usize); - if expected.len() >= STRING_VIEW_PREFIX_LEN { - assert_eq!( - &expected[..STRING_VIEW_PREFIX_LEN], - String::from_utf8_lossy(&actual.prefix) - ); - } - assert_eq!(expected, actual.as_utf8_str()); - } - } - - #[test] - fn string_with_nulls() { - let data = [Some("hello"), None, Some("comet")]; - let arr = StringArray::from(data.to_vec().clone()); - - let vector = PlainVector::from(arr.into_data()); - assert_eq!(3, vector.num_values()); - assert_eq!(1, vector.num_nulls()); - - for i in 0..data.len() { - if data[i].is_none() { - assert!(vector.is_null(i)); - } else { - let expected = data[i].unwrap(); - let actual = vector.value::(i); - if expected.len() >= STRING_VIEW_PREFIX_LEN { - assert_eq!( - &expected[..STRING_VIEW_PREFIX_LEN], - String::from_utf8_lossy(&actual.prefix) - ); - } - assert_eq!(expected, actual.as_utf8_str()); - } - } - } -} diff --git a/core/src/lib.rs b/core/src/lib.rs index f209859a7..092677274 100644 --- a/core/src/lib.rs +++ b/core/src/lib.rs @@ -21,7 +21,6 @@ #![allow(clippy::upper_case_acronyms)] #![allow(clippy::derive_partial_eq_without_eq)] // For prost generated struct #![cfg_attr(feature = "nightly", feature(core_intrinsics))] -#![feature(specialization)] // Branch prediction hint. This is currently only available on nightly. #[cfg(feature = "nightly")] diff --git a/rust-toolchain b/rust-toolchain index 55d7013e4..2bf5ad044 100644 --- a/rust-toolchain +++ b/rust-toolchain @@ -1 +1 @@ -nightly-2023-09-05 +stable